[ 
https://issues.apache.org/jira/browse/FLINK-8679?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16369135#comment-16369135
 ] 

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_r169069584
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ---
    @@ -266,9 +267,16 @@ public RocksDBKeyedStateBackend(
                RocksIterator iterator = db.newIterator(columnInfo.f0);
                iterator.seekToFirst();
     
    -           Iterable<K> iterable = () -> new 
RocksIteratorWrapper<>(iterator, state, keySerializer, keyGroupPrefixBytes);
    -           Stream<K> targetStream = 
StreamSupport.stream(iterable.spliterator(), false);
    -           return targetStream.onClose(iterator::close);
    +           try {
    +                   ByteArrayOutputStream outputStream = new 
ByteArrayOutputStream(8);
    +                   namespaceSerializer.serialize(namespace, new 
DataOutputViewStreamWrapper(outputStream));
    +                   final byte[] namespaceBytes = 
outputStream.toByteArray();
    +                   Iterable<K> iterable = () -> new 
RocksIteratorWrapper<>(iterator, state, keySerializer, keyGroupPrefixBytes, 
namespaceBytes);
    +                   Stream<K> targetStream = 
StreamSupport.stream(iterable.spliterator(), false);
    +                   return targetStream.onClose(iterator::close);
    +           } catch (IOException ex) {
    +                   throw new FlinkRuntimeException("Failed to get keys 
from RocksDB state backend.", ex);
    --- End diff --
    
    This case would not call `iterator.close()`. I suggest to just create 
everything related to `namespaceBytes` before creating the iterator, so that 
the iterator is only created when no further exception should happen.


> 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)

Reply via email to