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

ASF GitHub Bot commented on FLINK-8679:
---------------------------------------

Github user sihuazhou commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5518#discussion_r169990779
  
    --- Diff: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBRocksIteratorWrapperTest.java
 ---
    @@ -0,0 +1,147 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.contrib.streaming.state;
    +
    +import org.apache.flink.api.common.JobID;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.base.IntSerializer;
    +import org.apache.flink.api.common.typeutils.base.StringSerializer;
    +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.execution.Environment;
    +import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
    +import org.apache.flink.runtime.query.TaskKvStateRegistry;
    +import org.apache.flink.runtime.state.KeyGroupRange;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +
    +import org.junit.Assert;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import org.rocksdb.ColumnFamilyHandle;
    +import org.rocksdb.RocksIterator;
    +
    +import java.util.ArrayList;
    +import java.util.Comparator;
    +import java.util.List;
    +import java.util.function.Function;
    +
    +import static org.mockito.Mockito.mock;
    +
    +/**
    + * Tests for the RocksIteratorWrapper.
    + */
    +public class RocksDBRocksIteratorWrapperTest {
    +
    +   @Rule
    +   public final TemporaryFolder tmp = new TemporaryFolder();
    +
    +   @Test
    +   public void testIterator() throws Exception{
    +
    +           // test for keyGroupPrefixBytes == 1 && ambiguousKeyPossible == 
false
    +           testIteratorHelper(IntSerializer.INSTANCE, 
StringSerializer.INSTANCE, 128, i -> i);
    +
    +           // test for keyGroupPrefixBytes == 1 && ambiguousKeyPossible == 
true
    +           testIteratorHelper(StringSerializer.INSTANCE, 
StringSerializer.INSTANCE, 128, i -> String.valueOf(i));
    +
    +           // test for keyGroupPrefixBytes == 2 && ambiguousKeyPossible == 
false
    +           testIteratorHelper(IntSerializer.INSTANCE, 
StringSerializer.INSTANCE, 256, i -> i);
    +
    +           // test for keyGroupPrefixBytes == 2 && ambiguousKeyPossible == 
true
    +           testIteratorHelper(StringSerializer.INSTANCE, 
StringSerializer.INSTANCE, 256, i -> String.valueOf(i));
    +   }
    +
    +   <K> void testIteratorHelper(
    +           TypeSerializer<K> keySerializer,
    +           TypeSerializer namespaceSerializer,
    +           int maxKeyGroupNumber,
    +           Function<Integer, K> getKeyFunc) throws Exception {
    +
    +           String testStateName = "aha";
    +           String namespace = "ns";
    +
    +           String dbPath = tmp.newFolder().getAbsolutePath();
    +           String checkpointPath = tmp.newFolder().toURI().toString();
    +           RocksDBStateBackend backend = new RocksDBStateBackend(new 
FsStateBackend(checkpointPath), true);
    +           backend.setDbStoragePath(dbPath);
    +
    +           Environment env = new DummyEnvironment("TestTask", 1, 0);
    +
    +           RocksDBKeyedStateBackend<K> keyedStateBackend = 
(RocksDBKeyedStateBackend<K>) backend.createKeyedStateBackend(
    +                   env,
    +                   new JobID(),
    +                   "Test",
    +                   keySerializer,
    +                   maxKeyGroupNumber,
    +                   new KeyGroupRange(0, maxKeyGroupNumber - 1),
    +                   mock(TaskKvStateRegistry.class));
    +
    +           keyedStateBackend.restore(null);
    +
    +           ValueState<String> testState = 
keyedStateBackend.getPartitionedState(
    +                   namespace,
    +                   namespaceSerializer,
    +                   new ValueStateDescriptor<String>(testStateName, 
String.class));
    +
    +           // insert record
    +           for (int i = 0; i < 1000; ++i) {
    +                   keyedStateBackend.setCurrentKey(getKeyFunc.apply(i));
    +                   testState.update(String.valueOf(i));
    +           }
    +
    +           ColumnFamilyHandle handle = 
keyedStateBackend.getColumnFamilyHandle(testStateName);
    --- End diff --
    
    Nice catch! Addressing


> 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