AlexYinHan commented on code in PR #25924: URL: https://github.com/apache/flink/pull/25924#discussion_r1919963613
########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/datatransfer/ReusableDataTransferStrategy.java: ########## @@ -0,0 +1,183 @@ +/* + * 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.datatransfer; + +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointedStateScope; +import org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filemerging.SegmentFileStateHandle; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.state.forst.fs.ForStFlinkFileSystem; +import org.apache.flink.state.forst.fs.filemapping.FileOwnership; +import org.apache.flink.state.forst.fs.filemapping.FileOwnershipDecider; +import org.apache.flink.state.forst.fs.filemapping.MappingEntry; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * Data transfer strategy for ForSt DB with a remote DB path. When transferring data between + * Checkpoint and DB, this strategy MAY reuse the file from Checkpoint storage. + */ +public class ReusableDataTransferStrategy extends CopyDataTransferStrategy { + + private final FileOwnershipDecider fileOwnershipDecider; + + protected ReusableDataTransferStrategy( + @Nonnull FileOwnershipDecider fileOwnershipDecider, FileSystem dbFileSystem) { + super(dbFileSystem); + + Preconditions.checkArgument( + dbFileSystem instanceof ForStFlinkFileSystem, + "Unexpected dbFileSystem type: " + + dbFileSystem.getClass() + + ", expected: " + + ForStFlinkFileSystem.class); + + this.fileOwnershipDecider = fileOwnershipDecider; + } + + private ForStFlinkFileSystem getForStFlinkFileSystem() { + // We do not check the type here, because it is checked in the constructor. + return (ForStFlinkFileSystem) dbFileSystem; + } + + private @Nullable HandleAndLocalPath reuseFileToCheckpoint( + Path dbFilePath, ForStFlinkFileSystem forStFs) throws IOException { + LOG.trace("Reuse file to checkpoint: {}", dbFilePath); + + // Find the real path of the file + Path sourcePath = forStFs.srcPath(dbFilePath); + if (sourcePath != null) { + dbFilePath = sourcePath; + } + + // Create a StateHandle with the real path + FileStatus fileStatus = forStFs.getFileStatus(dbFilePath); + StreamStateHandle realSourceHandle = new FileStateHandle(sourcePath, fileStatus.getLen()); + + // Give file ownership to JM, i.e. DB will not delete it from now on + forStFs.giveUpOwnership(dbFilePath); + + return HandleAndLocalPath.of(realSourceHandle, dbFilePath.getName()); + } + + @Override + public HandleAndLocalPath transferToCheckpoint( + Path dbFilePath, + long maxTransferBytes, + CheckpointStreamFactory checkpointStreamFactory, + CheckpointedStateScope stateScope, + CloseableRegistry closeableRegistry, + CloseableRegistry tmpResourcesRegistry) + throws IOException { + if (maxTransferBytes < 0) { + // Means transfer whole file to checkpoint storage. + maxTransferBytes = Long.MAX_VALUE; + } + + ForStFlinkFileSystem forStFs = getForStFlinkFileSystem(); + MappingEntry mappingEntry = forStFs.getMappingEntry(dbFilePath); + Preconditions.checkNotNull(mappingEntry); + FileOwnership fileOwnership = mappingEntry.getFileOwnership(); + if (fileOwnership == FileOwnership.PRIVATE_OWNED_BY_DB) { + LOG.trace( + "Do not reuse file to checkpoint because the file is privately owned by DB: {}", + dbFilePath); + return super.transferToCheckpoint( + dbFilePath, + maxTransferBytes, + checkpointStreamFactory, + stateScope, + closeableRegistry, + tmpResourcesRegistry); + } + return reuseFileToCheckpoint(dbFilePath, forStFs); + } + + @Override + public void transferFromCheckpoint( + StreamStateHandle sourceHandle, Path targetPath, CloseableRegistry closeableRegistry) + throws IOException { + + FileOwnership fileOwnership = fileOwnershipDecider.decideForRestoredFile(targetPath); + if (fileOwnership != FileOwnership.PRIVATE_OWNED_BY_DB) { + if (tryReuseFileFromCheckpoint(sourceHandle, targetPath)) { + return; + } + } else { + LOG.trace( + "Do not reuse file from checkpoint because the file is privately owned by DB: {}", + targetPath); + } + super.transferFromCheckpoint(sourceHandle, targetPath, closeableRegistry); + } + + private boolean tryReuseFileFromCheckpoint(StreamStateHandle sourceHandle, Path targetPath) + throws IOException { + Path sourcePath = prepareSourcePathFromStateHandle(sourceHandle); + if (sourcePath == null) { + LOG.trace( + "Do not reuse file from checkpoint because the StateHandle is not backed by a file: {}, {}", + sourceHandle, + targetPath); + return false; + } else { + reuseFileFromCheckpoint(sourcePath, targetPath); + return true; + } + } + + private @Nullable Path prepareSourcePathFromStateHandle(StreamStateHandle sourceHandle) { + Path sourcePath = null; + if (sourceHandle instanceof FileStateHandle) { + sourcePath = ((FileStateHandle) sourceHandle).getFilePath(); + } else if (sourceHandle instanceof SegmentFileStateHandle) { + sourcePath = ((SegmentFileStateHandle) sourceHandle).getFilePath(); + } + return sourcePath; + } Review Comment: +1. I've added ```MappingEntrySource``` in ```MappingEntry```. Each entry can now be backed by either a file or a ```StreamStateHandle```. -- 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