Myasuka commented on code in PR #20399: URL: https://github.com/apache/flink/pull/20399#discussion_r939670463
########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.snapshot; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.RocksDbKvStateInfo; +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.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.SnapshotDirectory; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.RocksDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.File; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.getUploadedStateSize; + +/** + * Snapshot strategy for {@link RocksDBKeyedStateBackend} based on RocksDB's native checkpoints and + * creates full snapshots. the difference between savepoint is that sst files will be uploaded + * rather than states. + * + * @param <K> type of the backend keys. + */ +public class RocksNativeFullSnapshotStrategy<K> + extends RocksDBSnapshotStrategyBase< + K, RocksDBSnapshotStrategyBase.NativeRocksDBSnapshotResources> { + + private static final Logger LOG = + LoggerFactory.getLogger(RocksNativeFullSnapshotStrategy.class); + + private static final String DESCRIPTION = "Asynchronous full RocksDB snapshot"; + + /** The state handle ids of all sst files materialized in snapshots for previous checkpoints. */ + @Nonnull private final UUID backendUID; + + /** The help class used to upload state files. */ + private final RocksDBStateUploader stateUploader; + + public RocksNativeFullSnapshotStrategy( + @Nonnull RocksDB db, + @Nonnull ResourceGuard rocksDBResourceGuard, + @Nonnull TypeSerializer<K> keySerializer, + @Nonnull LinkedHashMap<String, RocksDbKvStateInfo> kvStateInformation, + @Nonnull KeyGroupRange keyGroupRange, + @Nonnegative int keyGroupPrefixBytes, + @Nonnull LocalRecoveryConfig localRecoveryConfig, + @Nonnull File instanceBasePath, + @Nonnull UUID backendUID, + @Nonnull RocksDBStateUploader rocksDBStateUploader) { + super( + DESCRIPTION, + db, + rocksDBResourceGuard, + keySerializer, + kvStateInformation, + keyGroupRange, + keyGroupPrefixBytes, + localRecoveryConfig, + instanceBasePath, + backendUID.toString().replaceAll("[\\-]", "")); + this.backendUID = backendUID; + this.stateUploader = rocksDBStateUploader; + } + + @Override + public SnapshotResultSupplier<KeyedStateHandle> asyncSnapshot( + NativeRocksDBSnapshotResources snapshotResources, + long checkpointId, + long timestamp, + @Nonnull CheckpointStreamFactory checkpointStreamFactory, + @Nonnull CheckpointOptions checkpointOptions) { + + if (snapshotResources.stateMetaInfoSnapshots.isEmpty()) { + return registry -> SnapshotResult.empty(); + } + + return new RocksDBNativeFullSnapshotOperation( + checkpointId, + checkpointStreamFactory, + snapshotResources.snapshotDirectory, + snapshotResources.stateMetaInfoSnapshots); + } + + @Override + public void notifyCheckpointComplete(long completedCheckpointId) { + // nothing to do + } + + @Override + public void notifyCheckpointAborted(long abortedCheckpointId) { + // nothing to do + } + + @Override + protected PreviousSnapshot snapshotMetaData( + long checkpointId, @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { + for (Map.Entry<String, RocksDbKvStateInfo> stateMetaInfoEntry : + kvStateInformation.entrySet()) { + stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().metaInfo.snapshot()); + } + return EMPTY_PREVIOUS_SNAPSHOT; + } + + @Override + public void close() { + stateUploader.close(); + } + + /** Encapsulates the process to perform a full snapshot of a RocksDBKeyedStateBackend. */ + private final class RocksDBNativeFullSnapshotOperation + implements SnapshotResultSupplier<KeyedStateHandle> { Review Comment: I think the code duplication still looks a bit ugly. We can introduce another abstract `RocksDBSnapshotOperation`. Maybe we can introduce this abstract class below in `RocksDBSnapshotStrategyBase` (also let `RocksDBSnapshotStrategyBase` include `backendUID`): ~~~java protected abstract class RocksDBSnapshotOperation implements SnapshotResultSupplier<KeyedStateHandle> { /** Id for the current checkpoint. */ protected final long checkpointId; @Nonnull protected final CheckpointStreamFactory checkpointStreamFactory; /** The state meta data. */ @Nonnull protected final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots; /** Local directory for the RocksDB native backup. */ @Nonnull protected final SnapshotDirectory localBackupDirectory; protected RocksDBSnapshotOperation( long checkpointId, @Nonnull CheckpointStreamFactory checkpointStreamFactory, @Nonnull SnapshotDirectory localBackupDirectory, @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { this.checkpointId = checkpointId; this.checkpointStreamFactory = checkpointStreamFactory; this.stateMetaInfoSnapshots = stateMetaInfoSnapshots; this.localBackupDirectory = localBackupDirectory; } protected Optional<KeyedStateHandle> getLocalSnapshot( @Nullable StreamStateHandle localStreamStateHandle, Map<StateHandleID, StreamStateHandle> sharedStateHandleIDs) throws IOException { final DirectoryStateHandle directoryStateHandle = localBackupDirectory.completeSnapshotAndGetHandle(); if (directoryStateHandle != null && localStreamStateHandle != null) { return Optional.of( new IncrementalLocalKeyedStateHandle( backendUID, checkpointId, directoryStateHandle, keyGroupRange, localStreamStateHandle, sharedStateHandleIDs)); } else { return Optional.empty(); } } } ~~~ ########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.snapshot; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.RocksDbKvStateInfo; +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.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.SnapshotDirectory; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.RocksDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.File; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.getUploadedStateSize; + +/** + * Snapshot strategy for {@link RocksDBKeyedStateBackend} based on RocksDB's native checkpoints and + * creates full snapshots. the difference between savepoint is that sst files will be uploaded + * rather than states. + * + * @param <K> type of the backend keys. + */ +public class RocksNativeFullSnapshotStrategy<K> + extends RocksDBSnapshotStrategyBase< + K, RocksDBSnapshotStrategyBase.NativeRocksDBSnapshotResources> { + + private static final Logger LOG = + LoggerFactory.getLogger(RocksNativeFullSnapshotStrategy.class); + + private static final String DESCRIPTION = "Asynchronous full RocksDB snapshot"; + + /** The state handle ids of all sst files materialized in snapshots for previous checkpoints. */ + @Nonnull private final UUID backendUID; + + /** The help class used to upload state files. */ + private final RocksDBStateUploader stateUploader; + + public RocksNativeFullSnapshotStrategy( + @Nonnull RocksDB db, + @Nonnull ResourceGuard rocksDBResourceGuard, + @Nonnull TypeSerializer<K> keySerializer, + @Nonnull LinkedHashMap<String, RocksDbKvStateInfo> kvStateInformation, + @Nonnull KeyGroupRange keyGroupRange, + @Nonnegative int keyGroupPrefixBytes, + @Nonnull LocalRecoveryConfig localRecoveryConfig, + @Nonnull File instanceBasePath, + @Nonnull UUID backendUID, + @Nonnull RocksDBStateUploader rocksDBStateUploader) { + super( + DESCRIPTION, + db, + rocksDBResourceGuard, + keySerializer, + kvStateInformation, + keyGroupRange, + keyGroupPrefixBytes, + localRecoveryConfig, + instanceBasePath, + backendUID.toString().replaceAll("[\\-]", "")); + this.backendUID = backendUID; + this.stateUploader = rocksDBStateUploader; + } + + @Override + public SnapshotResultSupplier<KeyedStateHandle> asyncSnapshot( + NativeRocksDBSnapshotResources snapshotResources, + long checkpointId, + long timestamp, + @Nonnull CheckpointStreamFactory checkpointStreamFactory, + @Nonnull CheckpointOptions checkpointOptions) { + + if (snapshotResources.stateMetaInfoSnapshots.isEmpty()) { + return registry -> SnapshotResult.empty(); + } + + return new RocksDBNativeFullSnapshotOperation( + checkpointId, + checkpointStreamFactory, + snapshotResources.snapshotDirectory, + snapshotResources.stateMetaInfoSnapshots); + } + + @Override + public void notifyCheckpointComplete(long completedCheckpointId) { + // nothing to do + } + + @Override + public void notifyCheckpointAborted(long abortedCheckpointId) { + // nothing to do + } + + @Override + protected PreviousSnapshot snapshotMetaData( + long checkpointId, @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { + for (Map.Entry<String, RocksDbKvStateInfo> stateMetaInfoEntry : + kvStateInformation.entrySet()) { + stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().metaInfo.snapshot()); + } + return EMPTY_PREVIOUS_SNAPSHOT; + } + + @Override + public void close() { + stateUploader.close(); + } + + /** Encapsulates the process to perform a full snapshot of a RocksDBKeyedStateBackend. */ + private final class RocksDBNativeFullSnapshotOperation + implements SnapshotResultSupplier<KeyedStateHandle> { Review Comment: By doing so, we can continue to reduce the code duplication in `RocksNativeFullSnapshotStrategy` and `RocksNativeFullSnapshotStrategy`. ########## flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java: ########## @@ -244,7 +298,8 @@ private RocksDBStateBackend createRocksDBStateBackend( private static void testExternalizedCheckpoints( File checkpointDir, String zooKeeperQuorum, - StateBackend backend, + StateBackend previousBackend, + StateBackend newBackend, Review Comment: I prefer to introduce three state-backend for different three runs with different configurations. And we can actually use the same configuration for `run1` and `run3`. ########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.snapshot; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.RocksDbKvStateInfo; +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.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.SnapshotDirectory; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.RocksDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.File; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.getUploadedStateSize; + +/** + * Snapshot strategy for {@link RocksDBKeyedStateBackend} based on RocksDB's native checkpoints and + * creates full snapshots. the difference between savepoint is that sst files will be uploaded + * rather than states. + * + * @param <K> type of the backend keys. + */ +public class RocksNativeFullSnapshotStrategy<K> + extends RocksDBSnapshotStrategyBase< + K, RocksDBSnapshotStrategyBase.NativeRocksDBSnapshotResources> { + + private static final Logger LOG = Review Comment: It seems the `LOG` is never used. ########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.snapshot; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; +import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.RocksDbKvStateInfo; +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.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.DirectoryStateHandle; +import org.apache.flink.runtime.state.IncrementalLocalKeyedStateHandle; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.SnapshotDirectory; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.RocksDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.File; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.apache.flink.contrib.streaming.state.snapshot.RocksSnapshotUtil.getUploadedStateSize; + +/** + * Snapshot strategy for {@link RocksDBKeyedStateBackend} based on RocksDB's native checkpoints and + * creates full snapshots. the difference between savepoint is that sst files will be uploaded + * rather than states. + * + * @param <K> type of the backend keys. + */ +public class RocksNativeFullSnapshotStrategy<K> + extends RocksDBSnapshotStrategyBase< + K, RocksDBSnapshotStrategyBase.NativeRocksDBSnapshotResources> { + + private static final Logger LOG = + LoggerFactory.getLogger(RocksNativeFullSnapshotStrategy.class); + + private static final String DESCRIPTION = "Asynchronous full RocksDB snapshot"; + + /** The state handle ids of all sst files materialized in snapshots for previous checkpoints. */ + @Nonnull private final UUID backendUID; + + /** The help class used to upload state files. */ + private final RocksDBStateUploader stateUploader; + + public RocksNativeFullSnapshotStrategy( + @Nonnull RocksDB db, + @Nonnull ResourceGuard rocksDBResourceGuard, + @Nonnull TypeSerializer<K> keySerializer, + @Nonnull LinkedHashMap<String, RocksDbKvStateInfo> kvStateInformation, + @Nonnull KeyGroupRange keyGroupRange, + @Nonnegative int keyGroupPrefixBytes, + @Nonnull LocalRecoveryConfig localRecoveryConfig, + @Nonnull File instanceBasePath, + @Nonnull UUID backendUID, + @Nonnull RocksDBStateUploader rocksDBStateUploader) { + super( + DESCRIPTION, + db, + rocksDBResourceGuard, + keySerializer, + kvStateInformation, + keyGroupRange, + keyGroupPrefixBytes, + localRecoveryConfig, + instanceBasePath, + backendUID.toString().replaceAll("[\\-]", "")); + this.backendUID = backendUID; + this.stateUploader = rocksDBStateUploader; + } + + @Override + public SnapshotResultSupplier<KeyedStateHandle> asyncSnapshot( + NativeRocksDBSnapshotResources snapshotResources, + long checkpointId, + long timestamp, + @Nonnull CheckpointStreamFactory checkpointStreamFactory, + @Nonnull CheckpointOptions checkpointOptions) { + + if (snapshotResources.stateMetaInfoSnapshots.isEmpty()) { + return registry -> SnapshotResult.empty(); + } + + return new RocksDBNativeFullSnapshotOperation( + checkpointId, + checkpointStreamFactory, + snapshotResources.snapshotDirectory, + snapshotResources.stateMetaInfoSnapshots); + } + + @Override + public void notifyCheckpointComplete(long completedCheckpointId) { + // nothing to do + } + + @Override + public void notifyCheckpointAborted(long abortedCheckpointId) { + // nothing to do + } + + @Override + protected PreviousSnapshot snapshotMetaData( + long checkpointId, @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { + for (Map.Entry<String, RocksDbKvStateInfo> stateMetaInfoEntry : + kvStateInformation.entrySet()) { + stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().metaInfo.snapshot()); + } + return EMPTY_PREVIOUS_SNAPSHOT; + } + + @Override + public void close() { + stateUploader.close(); + } + + /** Encapsulates the process to perform a full snapshot of a RocksDBKeyedStateBackend. */ + private final class RocksDBNativeFullSnapshotOperation + implements SnapshotResultSupplier<KeyedStateHandle> { + + /** Id for the current checkpoint. */ + private final long checkpointId; + + /** Stream factory that creates the output streams to DFS. */ + @Nonnull private final CheckpointStreamFactory checkpointStreamFactory; + + /** The state meta data. */ + @Nonnull private final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots; + + /** Local directory for the RocksDB native backup. */ + @Nonnull private final SnapshotDirectory localBackupDirectory; + + private RocksDBNativeFullSnapshotOperation( + long checkpointId, + @Nonnull CheckpointStreamFactory checkpointStreamFactory, + @Nonnull SnapshotDirectory localBackupDirectory, + @Nonnull List<StateMetaInfoSnapshot> stateMetaInfoSnapshots) { + this.checkpointStreamFactory = checkpointStreamFactory; + this.checkpointId = checkpointId; + this.localBackupDirectory = localBackupDirectory; + this.stateMetaInfoSnapshots = stateMetaInfoSnapshots; + } + + @Override + public SnapshotResult<KeyedStateHandle> get(CloseableRegistry snapshotCloseableRegistry) + throws Exception { + + boolean completed = false; + + // Handle to the meta data file + SnapshotResult<StreamStateHandle> metaStateHandle = null; + // Handles to the misc files in the current snapshot will go here + final Map<StateHandleID, StreamStateHandle> privateFiles = new HashMap<>(); + + try { + + metaStateHandle = + materializeMetaData( + snapshotCloseableRegistry, + stateMetaInfoSnapshots, + checkpointId, + checkpointStreamFactory); + + // Sanity checks - they should never fail + Preconditions.checkNotNull(metaStateHandle, "Metadata was not properly created."); + Preconditions.checkNotNull( + metaStateHandle.getJobManagerOwnedSnapshot(), + "Metadata for job manager was not properly created."); + + uploadSstFiles(privateFiles, snapshotCloseableRegistry); + long checkpointedSize = metaStateHandle.getStateSize(); + checkpointedSize += getUploadedStateSize(privateFiles.values()); + + final IncrementalRemoteKeyedStateHandle jmIncrementalKeyedStateHandle = + new IncrementalRemoteKeyedStateHandle( + backendUID, + keyGroupRange, + checkpointId, + Collections.emptyMap(), + privateFiles, + metaStateHandle.getJobManagerOwnedSnapshot(), + checkpointedSize); + + final DirectoryStateHandle directoryStateHandle = + localBackupDirectory.completeSnapshotAndGetHandle(); + final SnapshotResult<KeyedStateHandle> snapshotResult; + if (directoryStateHandle != null + && metaStateHandle.getTaskLocalSnapshot() != null) { + + IncrementalLocalKeyedStateHandle localDirKeyedStateHandle = + new IncrementalLocalKeyedStateHandle( + backendUID, + checkpointId, + directoryStateHandle, + keyGroupRange, + metaStateHandle.getTaskLocalSnapshot(), + Collections.emptyMap()); + snapshotResult = + SnapshotResult.withLocalState( + jmIncrementalKeyedStateHandle, localDirKeyedStateHandle); + } else { + snapshotResult = SnapshotResult.of(jmIncrementalKeyedStateHandle); + } + + completed = true; + + return snapshotResult; + } finally { + if (!completed) { + final List<StateObject> statesToDiscard = + new ArrayList<>(1 + privateFiles.size()); + statesToDiscard.add(metaStateHandle); + statesToDiscard.addAll(privateFiles.values()); + cleanupIncompleteSnapshot(statesToDiscard, localBackupDirectory); + } + } + } + + protected void uploadSstFiles( Review Comment: This method should be `private`. ########## flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java: ########## @@ -94,71 +94,63 @@ public static Object[] parameters() { @Test public void testExternalizedIncrementalRocksDBCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); + StateBackend stateBackend = createRocksDBStateBackend(checkpointDir, true); testExternalizedCheckpoints( - checkpointDir, - null, - createRocksDBStateBackend(checkpointDir, true), - false, - restoreMode); + checkpointDir, null, stateBackend, stateBackend, false, restoreMode); Review Comment: Actually, we don't need to change these existing tests. We can introduce another static method below to keep previous tests stay as before. ~~~java private static void testExternalizedCheckpoints( File checkpointDir, String zooKeeperQuorum, boolean localRecovery, RestoreMode restoreMode) throws Exception { StateBackend stateBackend = createRocksDBStateBackend(checkpointDir, true); testExternalizedCheckpoints ... } ~~~ -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org