Myasuka commented on a change in pull request #16969:
URL: https://github.com/apache/flink/pull/16969#discussion_r702850957
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java
##########
@@ -0,0 +1,184 @@
+package org.apache.flink.contrib.streaming.state.snapshot;
+
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBResource;
+import org.apache.flink.contrib.streaming.state.RocksDBStateUploader;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.ArrayListSerializer;
+import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
+import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
+import org.apache.flink.runtime.state.StateHandleID;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory;
+import org.apache.flink.util.ResourceGuard;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.flink.core.fs.Path.fromLocalFile;
+import static org.apache.flink.core.fs.local.LocalFileSystem.getSharedInstance;
+import static org.powermock.api.mockito.PowerMockito.spy;
+
+/** Tests for {@link RocksIncrementalSnapshotStrategy}. */
+public class RocksIncrementalSnapshotStrategyTest {
+
+ @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Rule public RocksDBResource rocksDBResource = new RocksDBResource();
+
+ // Verify the next checkpoint is still incremental after a savepoint
completed.
+ @Test
+ public void testCheckpointIsIncremental() throws Exception {
+
+ try (CloseableRegistry closeableRegistry = new CloseableRegistry()) {
+ Tuple2<RocksIncrementalSnapshotStrategy,
FsCheckpointStreamFactory> prepareData =
+ prepareCreateSnapshotStrategy(closeableRegistry);
+
+ try (RocksIncrementalSnapshotStrategy checkpointSnapshotStrategy =
prepareData.f0) {
+
+ FsCheckpointStreamFactory checkpointStreamFactory =
prepareData.f1;
Review comment:
We'd better include `checkpointStreamFactory` within the
try-with-resource statement.
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java
##########
@@ -0,0 +1,184 @@
+package org.apache.flink.contrib.streaming.state.snapshot;
+
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBResource;
+import org.apache.flink.contrib.streaming.state.RocksDBStateUploader;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.ArrayListSerializer;
+import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
+import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
+import org.apache.flink.runtime.state.StateHandleID;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory;
+import org.apache.flink.util.ResourceGuard;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.flink.core.fs.Path.fromLocalFile;
+import static org.apache.flink.core.fs.local.LocalFileSystem.getSharedInstance;
+import static org.powermock.api.mockito.PowerMockito.spy;
+
+/** Tests for {@link RocksIncrementalSnapshotStrategy}. */
+public class RocksIncrementalSnapshotStrategyTest {
+
+ @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Rule public RocksDBResource rocksDBResource = new RocksDBResource();
+
+ // Verify the next checkpoint is still incremental after a savepoint
completed.
+ @Test
+ public void testCheckpointIsIncremental() throws Exception {
+
+ try (CloseableRegistry closeableRegistry = new CloseableRegistry()) {
+ Tuple2<RocksIncrementalSnapshotStrategy,
FsCheckpointStreamFactory> prepareData =
+ prepareCreateSnapshotStrategy(closeableRegistry);
+
+ try (RocksIncrementalSnapshotStrategy checkpointSnapshotStrategy =
prepareData.f0) {
+
+ FsCheckpointStreamFactory checkpointStreamFactory =
prepareData.f1;
+
+ // make and notify checkpoint with id 1
+ snapshot(
+ 1L, checkpointSnapshotStrategy,
checkpointStreamFactory, closeableRegistry);
+ checkpointSnapshotStrategy.notifyCheckpointComplete(1L);
+
+ // notify savepoint with id 2
+ checkpointSnapshotStrategy.notifyCheckpointComplete(2L);
+
+ // make checkpoint with id 3
+ IncrementalRemoteKeyedStateHandle
incrementalRemoteKeyedStateHandle3 =
+ snapshot(
+ 3L,
+ checkpointSnapshotStrategy,
+ checkpointStreamFactory,
+ closeableRegistry);
+
+ // If 3rd checkpoint's placeholderStateHandleCount > 0,it
means 3rd checkpoint is
+ // incremental.
+ Map<StateHandleID, StreamStateHandle> sharedState3 =
+ incrementalRemoteKeyedStateHandle3.getSharedState();
+ long placeholderStateHandleCount =
+ sharedState3.entrySet().stream()
+ .filter(e -> e.getValue() instanceof
PlaceholderStreamStateHandle)
+ .count();
+
+ Assert.assertTrue(placeholderStateHandleCount > 0);
+ }
+ }
+ }
+
+ public Tuple2<RocksIncrementalSnapshotStrategy, FsCheckpointStreamFactory>
+ prepareCreateSnapshotStrategy(CloseableRegistry closeableRegistry)
+ throws IOException, RocksDBException {
+
+ // prepare a rocksDB and put data
Review comment:
This comment could be removed or just say putting some data.
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java
##########
@@ -0,0 +1,184 @@
+package org.apache.flink.contrib.streaming.state.snapshot;
+
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBResource;
+import org.apache.flink.contrib.streaming.state.RocksDBStateUploader;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.ArrayListSerializer;
+import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
+import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
+import org.apache.flink.runtime.state.StateHandleID;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory;
+import org.apache.flink.util.ResourceGuard;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.flink.core.fs.Path.fromLocalFile;
+import static org.apache.flink.core.fs.local.LocalFileSystem.getSharedInstance;
+import static org.powermock.api.mockito.PowerMockito.spy;
+
+/** Tests for {@link RocksIncrementalSnapshotStrategy}. */
+public class RocksIncrementalSnapshotStrategyTest {
+
+ @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Rule public RocksDBResource rocksDBResource = new RocksDBResource();
+
+ // Verify the next checkpoint is still incremental after a savepoint
completed.
+ @Test
+ public void testCheckpointIsIncremental() throws Exception {
+
+ try (CloseableRegistry closeableRegistry = new CloseableRegistry()) {
+ Tuple2<RocksIncrementalSnapshotStrategy,
FsCheckpointStreamFactory> prepareData =
+ prepareCreateSnapshotStrategy(closeableRegistry);
+
+ try (RocksIncrementalSnapshotStrategy checkpointSnapshotStrategy =
prepareData.f0) {
+
+ FsCheckpointStreamFactory checkpointStreamFactory =
prepareData.f1;
+
+ // make and notify checkpoint with id 1
+ snapshot(
+ 1L, checkpointSnapshotStrategy,
checkpointStreamFactory, closeableRegistry);
+ checkpointSnapshotStrategy.notifyCheckpointComplete(1L);
+
+ // notify savepoint with id 2
+ checkpointSnapshotStrategy.notifyCheckpointComplete(2L);
+
+ // make checkpoint with id 3
+ IncrementalRemoteKeyedStateHandle
incrementalRemoteKeyedStateHandle3 =
+ snapshot(
+ 3L,
+ checkpointSnapshotStrategy,
+ checkpointStreamFactory,
+ closeableRegistry);
+
+ // If 3rd checkpoint's placeholderStateHandleCount > 0,it
means 3rd checkpoint is
+ // incremental.
+ Map<StateHandleID, StreamStateHandle> sharedState3 =
+ incrementalRemoteKeyedStateHandle3.getSharedState();
+ long placeholderStateHandleCount =
+ sharedState3.entrySet().stream()
+ .filter(e -> e.getValue() instanceof
PlaceholderStreamStateHandle)
+ .count();
+
+ Assert.assertTrue(placeholderStateHandleCount > 0);
+ }
+ }
+ }
+
+ public Tuple2<RocksIncrementalSnapshotStrategy, FsCheckpointStreamFactory>
+ prepareCreateSnapshotStrategy(CloseableRegistry closeableRegistry)
+ throws IOException, RocksDBException {
+
+ // prepare a rocksDB and put data
+ ColumnFamilyHandle columnFamilyHandle =
rocksDBResource.createNewColumnFamily("test");
+ RocksDB rocksDB = rocksDBResource.getRocksDB();
+ byte[] key = "checkpoint".getBytes();
+ byte[] val = "incrementalTest".getBytes();
+ rocksDB.put(columnFamilyHandle, key, val);
+
+ // construct RocksIncrementalSnapshotStrategy
+ long lastCompletedCheckpointId = -1L;
+ ResourceGuard rocksDBResourceGuard = new ResourceGuard();
+ SortedMap<Long, Set<StateHandleID>> materializedSstFiles = new
TreeMap<>();
+ LinkedHashMap<String, RocksDBKeyedStateBackend.RocksDbKvStateInfo>
kvStateInformation =
+ new LinkedHashMap<>();
+
+ RocksDBStateUploader rocksDBStateUploader =
+ spy(
Review comment:
I think we don't need a `spy` here.
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java
##########
@@ -0,0 +1,184 @@
+package org.apache.flink.contrib.streaming.state.snapshot;
+
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.contrib.streaming.state.RocksDBOptions;
+import org.apache.flink.contrib.streaming.state.RocksDBResource;
+import org.apache.flink.contrib.streaming.state.RocksDBStateUploader;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.ArrayListSerializer;
+import org.apache.flink.runtime.state.CompositeKeySerializationUtils;
+import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.PlaceholderStreamStateHandle;
+import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
+import org.apache.flink.runtime.state.StateHandleID;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.TestLocalRecoveryConfig;
+import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory;
+import org.apache.flink.util.ResourceGuard;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import static org.apache.flink.core.fs.Path.fromLocalFile;
+import static org.apache.flink.core.fs.local.LocalFileSystem.getSharedInstance;
+import static org.powermock.api.mockito.PowerMockito.spy;
+
+/** Tests for {@link RocksIncrementalSnapshotStrategy}. */
+public class RocksIncrementalSnapshotStrategyTest {
+
+ @Rule public final TemporaryFolder tmp = new TemporaryFolder();
+
+ @Rule public RocksDBResource rocksDBResource = new RocksDBResource();
+
+ // Verify the next checkpoint is still incremental after a savepoint
completed.
+ @Test
+ public void testCheckpointIsIncremental() throws Exception {
+
+ try (CloseableRegistry closeableRegistry = new CloseableRegistry()) {
+ Tuple2<RocksIncrementalSnapshotStrategy,
FsCheckpointStreamFactory> prepareData =
+ prepareCreateSnapshotStrategy(closeableRegistry);
Review comment:
I cannot see any relationship that we must return
`RocksIncrementalSnapshotStrategy` and `FsCheckpointStreamFactory` together
within one method. I think you could use differnt methods to create them and
include them all in the try-with-resource statement.
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategyTest.java
##########
@@ -0,0 +1,184 @@
+package org.apache.flink.contrib.streaming.state.snapshot;
Review comment:
This file lacks of apache license.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]