private UV deserializeUserValue(byte[] rawValueBytes) throws IOException { ByteArrayInputStreamWithPos bais = new ByteArrayInputStreamWithPos(rawValueBytes); DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais); boolean isNull = in.readBoolean(); return isNull ? null : userValueSerializer.deserialize(in); }
@Override public void flatMap(KeyedStateRow row, Collector<V> out) throws Exception { if (!stateName.equals(row.getStateName())) { return; } byte[] valueBytes = row.getValueBytes(); V value = null; try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); if (!iw.readBoolean()) { skipTimestampIfTtlEnabled(iw); value = valueDeserializer.deserialize(iw); } } if (value == null) { throw new RuntimeException("MapStates with null values are not supported at the moment."); } else { out.collect(value); } } }
@SuppressWarnings("unchecked") @Override public void flatMap(KeyedStateRow row, Collector<Tuple3<K, MK, V>> out) throws Exception { if (!stateName.equals(row.getStateName())) { return; } byte[] keyAndNamespaceBytes = row.getKeyAndNamespaceBytes(); byte[] valueBytes = row.getValueBytes(); K key; MK mapKey; try (ByteArrayInputStreamWithPos keyIs = new ByteArrayInputStreamWithPos(keyAndNamespaceBytes)) { DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(keyIs); iw.skipBytesToRead(keygroupPrefixBytes); key = RocksDBUtils.readKey(keyDeserializer, keyIs, iw, false); VoidNamespaceSerializer.INSTANCE.deserialize(iw); mapKey = (MK) mapKeySerializer.deserialize(iw); } V value = null; try (ByteArrayInputStreamWithPos valIs = new ByteArrayInputStreamWithPos(valueBytes)) { DataInputViewStreamWrapper iw = new DataInputViewStreamWrapper(valIs); if (!iw.readBoolean()) { skipTimestampIfTtlEnabled(iw); value = valueDeserializer.deserialize(iw); } } if (value == null) { throw new RuntimeException("MapStates with null values are not supported at the moment."); } out.collect(Tuple3.of(key, mapKey, value)); }