Myasuka commented on code in PR #20399: URL: https://github.com/apache/flink/pull/20399#discussion_r934271416
########## flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksNativeFullSnapshotStrategy.java: ########## @@ -0,0 +1,410 @@ +/* + * 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.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointStreamWithResultProvider; +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.KeyedBackendSerializationProxy; +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.SnapshotResources; +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.StateUtil; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.Checkpoint; +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.io.IOException; +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, RocksNativeFullSnapshotStrategy.NativeFullRocksDBSnapshotResources> { + + 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 NativeFullRocksDBSnapshotResources syncPrepareResources(long checkpointId) + throws Exception { + + final SnapshotDirectory snapshotDirectory = prepareLocalSnapshotDirectory(checkpointId); + LOG.trace("Local RocksDB checkpoint goes to backup path {}.", snapshotDirectory); + + final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots = + new ArrayList<>(kvStateInformation.size()); + + for (Map.Entry<String, RocksDbKvStateInfo> stateMetaInfoEntry : + kvStateInformation.entrySet()) { + stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().metaInfo.snapshot()); + } + + takeDBNativeCheckpoint(snapshotDirectory); + + return new NativeFullRocksDBSnapshotResources(snapshotDirectory, stateMetaInfoSnapshots); Review Comment: These lines are almost the same as `RocksIncrementalSnapshotStrategy`, and I think we could move such classes (e.g. `NativeFullRocksDBSnapshotResources`) and methods (e.g. `#materializeMetaData`) to `RocksDBSnapshotStrategyBase` to avoid code duplication. -- 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