zoltar9264 commented on code in PR #24879: URL: https://github.com/apache/flink/pull/24879#discussion_r1753111352
########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/snapshot/ForStSnapshotStrategyBase.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.state.forst.snapshot; + +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.checkpoint.filemerging.FileMergingSnapshotManager; +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.IncrementalKeyedStateHandle.HandleAndLocalPath; +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.PlaceholderStreamStateHandle; +import org.apache.flink.runtime.state.SnapshotResources; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.SnapshotStrategy; +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.state.forst.ForStKeyedStateBackend.ForStKvStateInfo; +import org.apache.flink.state.forst.ForStResourceContainer; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * Abstract base class for {@link SnapshotStrategy} implementations for ForSt state backend. + * + * @param <K> type of the backend keys. + */ +public abstract class ForStSnapshotStrategyBase<K, R extends SnapshotResources> + implements CheckpointListener, + SnapshotStrategy< + KeyedStateHandle, ForStSnapshotStrategyBase.ForStNativeSnapshotResources>, + AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(ForStSnapshotStrategyBase.class); + + @Nonnull private final String description; + + /** ForSt instance from the backend. */ + @Nonnull protected final RocksDB db; + + /** Resource guard for the ForSt instance. */ + @Nonnull protected final ResourceGuard resourceGuard; + + @Nonnull protected final ForStResourceContainer resourceContainer; + + /** The key serializer of the backend. */ + @Nonnull protected final TypeSerializer<K> keySerializer; + + /** Key/Value state meta info from the backend. */ + @Nonnull protected final LinkedHashMap<String, ForStKvStateInfo> kvStateInformation; + + /** The key-group range for the task. */ + @Nonnull protected final KeyGroupRange keyGroupRange; + + /** Number of bytes in the key-group prefix. */ + @Nonnegative protected final int keyGroupPrefixBytes; + + /** The state handle ids of all sst files materialized in snapshots for previous checkpoints. */ + @Nonnull protected final UUID backendUID; + + public ForStSnapshotStrategyBase( + @Nonnull String description, + @Nonnull RocksDB db, + @Nonnull ResourceGuard resourceGuard, + @Nonnull ForStResourceContainer resourceContainer, + @Nonnull TypeSerializer<K> keySerializer, + @Nonnull LinkedHashMap<String, ForStKvStateInfo> kvStateInformation, + @Nonnull KeyGroupRange keyGroupRange, + @Nonnegative int keyGroupPrefixBytes, + @Nonnull UUID backendUID) { + this.db = db; + this.resourceGuard = resourceGuard; + this.resourceContainer = resourceContainer; + this.keySerializer = keySerializer; + this.kvStateInformation = kvStateInformation; + this.keyGroupRange = keyGroupRange; + this.keyGroupPrefixBytes = keyGroupPrefixBytes; + this.description = description; + this.backendUID = backendUID; + } + + @Nonnull + public String getDescription() { + return description; + } + + @Override + public ForStNativeSnapshotResources syncPrepareResources(long checkpointId) throws Exception { + + final List<StateMetaInfoSnapshot> stateMetaInfoSnapshots = + new ArrayList<>(kvStateInformation.size()); + final PreviousSnapshot previousSnapshot = + snapshotMetaData(checkpointId, stateMetaInfoSnapshots); + + // Disable file deletion for file transformation. ForSt will decide whether to allow file + // deletion based on the number of calls to disableFileDeletions() and + // enableFileDeletions(), so disableFileDeletions() should be call only once. + db.disableFileDeletions(); + + // get live files with flush memtable + RocksDB.LiveFiles liveFiles = db.getLiveFiles(true); + List<Path> liveFilesPath = + liveFiles.files.stream() + .map(file -> new Path(resourceContainer.getDbPath(), file)) + .collect(Collectors.toList()); + + logLiveFiles(checkpointId, liveFiles.manifestFileSize, liveFilesPath); + + return new ForStNativeSnapshotResources( + stateMetaInfoSnapshots, + liveFiles.manifestFileSize, + liveFilesPath, + previousSnapshot, + () -> { + try { + db.enableFileDeletions(false); + LOG.info( + "Release one file deletion lock with ForStNativeSnapshotResources, backendUID:{}, checkpointId:{}.", + backendUID, + checkpointId); + } catch (RocksDBException e) { + LOG.error( + "Enable file deletion failed, backendUID:{}, checkpointId:{}.", + backendUID, + checkpointId, + e); + // TODO: 2024/5/28 wangfeifan - Add more robust exception handling + } + }); + } Review Comment: If call enableFileDeletions after any exception thrown in disableFileDeletions, consider parallel checkpointing, file deletion may be allowed prematurely. WDYT ? -- 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