[ https://issues.apache.org/jira/browse/FLINK-8679?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16370209#comment-16370209 ]
ASF GitHub Bot commented on FLINK-8679: --------------------------------------- Github user StefanRRichter commented on a diff in the pull request: https://github.com/apache/flink/pull/5518#discussion_r169371872 --- Diff: flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java --- @@ -1991,43 +1999,71 @@ public int numStateEntries() { return count; } + /** + * This class is not thread safety. + */ private static class RocksIteratorWrapper<K> implements Iterator<K> { private final RocksIterator iterator; private final String state; private final TypeSerializer<K> keySerializer; private final int keyGroupPrefixBytes; + private final byte[] namespaceBytes; + private K nextKey; public RocksIteratorWrapper( RocksIterator iterator, String state, TypeSerializer<K> keySerializer, - int keyGroupPrefixBytes) { + int keyGroupPrefixBytes, + byte[] namespaceBytes) { this.iterator = Preconditions.checkNotNull(iterator); this.state = Preconditions.checkNotNull(state); this.keySerializer = Preconditions.checkNotNull(keySerializer); this.keyGroupPrefixBytes = Preconditions.checkNotNull(keyGroupPrefixBytes); + this.namespaceBytes = Preconditions.checkNotNull(namespaceBytes); + this.nextKey = null; } @Override public boolean hasNext() { - return iterator.isValid(); + final int namespaceBytesLength = namespaceBytes.length; + while (nextKey == null && iterator.isValid()) { + try { + boolean namespaceValid = true; + byte[] key = iterator.key(); + if (key.length >= namespaceBytesLength + keyGroupPrefixBytes) { + for (int i = 1; i <= namespaceBytesLength; ++i) { + if (key[key.length - i] != namespaceBytes[namespaceBytesLength - i]) { + namespaceValid = false; + break; + } + } + if (namespaceValid) { + DataInputViewStreamWrapper dataInput = new DataInputViewStreamWrapper( --- End diff -- Oh, yes sorry you are right, I was confused :-) > RocksDBKeyedBackend.getKeys(stateName, namespace) doesn't filter data with > namespace > ------------------------------------------------------------------------------------ > > Key: FLINK-8679 > URL: https://issues.apache.org/jira/browse/FLINK-8679 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing > Affects Versions: 1.5.0 > Reporter: Sihua Zhou > Assignee: Sihua Zhou > Priority: Blocker > Fix For: 1.5.0 > > > Currently, `RocksDBKeyedBackend.getKeys(stateName, namespace)` is odds. It > doesn't use the namespace to filter data. And > `HeapKeyedBackend.getKeys(stateName, namespace)` has done that, I think they > should be consistent at least. -- This message was sent by Atlassian JIRA (v7.6.3#76005)