Zakelly commented on code in PR #25924: URL: https://github.com/apache/flink/pull/25924#discussion_r1914688767
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageAccess.java: ########## @@ -124,10 +124,14 @@ public FsCheckpointStorageAccess( // ------------------------------------------------------------------------ @VisibleForTesting Review Comment: If you make it public and use it in normal code other than test, please remove this annotation. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileType.java: ########## @@ -0,0 +1,25 @@ +/* + * 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.fs.filemapping; + +/** Indicates the type of file in ForSt DB. */ +public enum FileType { Review Comment: Do we really need to introduce this? Is `FileOwnership.LOCAL_OWNED_BY_DB` is enough for `MISC_FILE`? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java: ########## @@ -364,11 +444,31 @@ public void prepareDirectories() throws Exception { } } + public void buildDataTransferStrategy() { Review Comment: I'd suggest: - Build `DataTransferStrategy` in constructor. - make `isDbPathUnderCheckpointPath` as a member of implementation of `DataTransferStrategy`, and calculate this in the builder. ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/datatransfer/DataTransferStrategyBuilder.java: ########## @@ -0,0 +1,95 @@ +/* + * 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.execution.RecoveryClaimMode; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Builder for {@link DataTransferStrategy}. */ +public class DataTransferStrategyBuilder { + private static final Logger LOG = LoggerFactory.getLogger(DataTransferStrategyBuilder.class); + + private Path dbRemotePath; + + private FileSystem dbDelegateFileSystem; + + private boolean isDbPathUnderCheckpointPath; + + private RecoveryClaimMode claimMode; + + public void withoutDbRemotePath() { + this.dbRemotePath = null; + this.dbDelegateFileSystem = null; + this.claimMode = RecoveryClaimMode.DEFAULT; + } + + public void withDbRemotePath(FileSystem dbDelegateFileSystem, Path dbRemotePath) { + this.dbRemotePath = dbRemotePath; + this.dbDelegateFileSystem = dbDelegateFileSystem; + } + + public void withIsDbPathUnderCheckpointPath(Boolean isDbPathUnderCheckpointPath) { + this.isDbPathUnderCheckpointPath = isDbPathUnderCheckpointPath; + } + + public void withClaimMode(RecoveryClaimMode claimMode) { + this.claimMode = claimMode; + } + + public DataTransferStrategyBuilder() { + this.dbRemotePath = null; + this.dbDelegateFileSystem = null; + this.isDbPathUnderCheckpointPath = false; + } + + public DataTransferStrategy build() { + DataTransferStrategy strategy; + if (dbDelegateFileSystem == null) { + strategy = new LocalFsDataTransferStrategy(claimMode); + LOG.info("Build DataTransferStrategy: {}", strategy); + return strategy; + } + + RemoteFsDataTransferStrategy.FileTransferStyle snapshotStyle; + RemoteFsDataTransferStrategy.FileTransferStyle restoreStyle; + if (isDbPathUnderCheckpointPath) { + snapshotStyle = RemoteFsDataTransferStrategy.FileTransferStyle.REUSE; + restoreStyle = + claimMode == RecoveryClaimMode.CLAIM + ? RemoteFsDataTransferStrategy.FileTransferStyle.REUSE + : RemoteFsDataTransferStrategy.FileTransferStyle.COPY; + } else { + snapshotStyle = RemoteFsDataTransferStrategy.FileTransferStyle.COPY; + restoreStyle = RemoteFsDataTransferStrategy.FileTransferStyle.COPY; + } + + strategy = + new RemoteFsDataTransferStrategy( + snapshotStyle, + restoreStyle, + claimMode, + dbDelegateFileSystem, + isDbPathUnderCheckpointPath); + LOG.info("Build DataTransferStrategy: {}", strategy); + return strategy; Review Comment: IIUC, there should be two types of strategy, one is for copying by stream while another is for file reuse (in best effort). But it seems we devide by Local/Remote, is there any reason for this? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java: ########## @@ -378,12 +478,21 @@ private static void prepareDirectories(Path basePath, Path dbPath) throws IOExce throw new IOException( String.format("Could not create ForSt directory at %s.", basePath)); } - if (fileSystem.exists(dbPath)) { - fileSystem.delete(dbPath, true); - } - if (!fileSystem.mkdirs(dbPath)) { - throw new IOException( - String.format("Could not create ForSt db directory at %s.", dbPath)); + if (clearDbPath) { Review Comment: Why we need `clearDbPath` here? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/fs/filemapping/FileMappingManager.java: ########## @@ -231,14 +284,29 @@ private boolean isParentDir(String path, String dir) { } } - /** A wrapper of real path. */ - public static class RealPath { Review Comment: Why removing this class? ########## flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/ForStResourceContainer.java: ########## @@ -167,6 +210,43 @@ public ForStResourceContainer( this.metricGroup = metricGroup; } + private static boolean isDbPathUnderCheckpointPath( + @Nullable CheckpointStorageAccess checkpointStorageAccess, + @Nullable Path dbRemotePath) { + if (dbRemotePath == null) { + return false; + } + + Preconditions.checkArgument( + checkpointStorageAccess instanceof FsCheckpointStorageAccess, + "Unexpected checkpointStorageAccess type: " + + (checkpointStorageAccess == null + ? "null" + : checkpointStorageAccess.getClass())); Review Comment: So we only support `FsCheckpointStorageAccess`, right? Can we support `MemoryBackendCheckpointStorageAccess` by writing via stream? And what if it is `FsMergingCheckpointStorageAccess`? -- 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