Zakelly commented on code in PR #25397:
URL: https://github.com/apache/flink/pull/25397#discussion_r1777423164


##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/restore/ForStIncrementalRestoreOperation.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.restore;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
+import 
org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath;
+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.RegisteredStateMetaInfoBase;
+import org.apache.flink.runtime.state.StateBackend.CustomInitializationMetrics;
+import org.apache.flink.runtime.state.StateSerializerProvider;
+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.ForStNativeMetricOptions;
+import org.apache.flink.state.forst.ForStOperationUtils;
+import org.apache.flink.state.forst.ForStResourceContainer;
+import org.apache.flink.state.forst.ForStStateDataTransfer;
+import org.apache.flink.state.forst.StateHandleTransferSpec;
+import org.apache.flink.util.StateMigrationException;
+import org.apache.flink.util.clock.SystemClock;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.runtime.metrics.MetricNames.DOWNLOAD_STATE_DURATION;
+import static 
org.apache.flink.runtime.metrics.MetricNames.RESTORE_STATE_DURATION;
+
+/** Encapsulates the process of restoring a ForSt instance from an incremental 
snapshot. */
+public class ForStIncrementalRestoreOperation<K> implements 
ForStRestoreOperation {
+
+    private static final Logger logger =
+            LoggerFactory.getLogger(ForStIncrementalRestoreOperation.class);
+
+    private final String operatorIdentifier;
+    private final SortedMap<Long, Collection<HandleAndLocalPath>> 
restoredSstFiles;
+    private final ForStHandle forstHandle;
+    private final Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles;
+    private final CloseableRegistry cancelStreamRegistry;
+    private final KeyGroupRange keyGroupRange;
+    private final Path forstBasePath;
+    private final StateSerializerProvider<K> keySerializerProvider;
+    private final ClassLoader userCodeClassLoader;
+    private final CustomInitializationMetrics customInitializationMetrics;
+    private long lastCompletedCheckpointId;
+    private UUID backendUID;
+
+    private boolean isKeySerializerCompatibilityChecked;
+
+    public ForStIncrementalRestoreOperation(
+            String operatorIdentifier,
+            KeyGroupRange keyGroupRange,
+            CloseableRegistry cancelStreamRegistry,
+            ClassLoader userCodeClassLoader,
+            Map<String, ForStKvStateInfo> kvStateInformation,
+            StateSerializerProvider<K> keySerializerProvider,
+            Path forstBasePath,
+            File instanceRocksDBPath,
+            DBOptions dbOptions,
+            Function<String, ColumnFamilyOptions> columnFamilyOptionsFactory,
+            ForStNativeMetricOptions nativeMetricOptions,
+            MetricGroup metricGroup,
+            CustomInitializationMetrics customInitializationMetrics,
+            @Nonnull Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles) {
+        this.forstHandle =
+                new ForStHandle(
+                        kvStateInformation,
+                        instanceRocksDBPath,
+                        dbOptions,
+                        columnFamilyOptionsFactory,
+                        nativeMetricOptions,
+                        metricGroup);
+        this.operatorIdentifier = operatorIdentifier;
+        this.restoredSstFiles = new TreeMap<>();
+        this.lastCompletedCheckpointId = -1L;
+        this.backendUID = UUID.randomUUID();
+        this.customInitializationMetrics = customInitializationMetrics;
+        this.restoreStateHandles = restoreStateHandles;
+        this.cancelStreamRegistry = cancelStreamRegistry;
+        this.keyGroupRange = keyGroupRange;
+        this.forstBasePath = forstBasePath;
+        this.keySerializerProvider = keySerializerProvider;
+        this.userCodeClassLoader = userCodeClassLoader;
+    }
+
+    /**
+     * Root method that branches for different implementations of {@link
+     * IncrementalKeyedStateHandle}.
+     */
+    @Override
+    public ForStRestoreResult restore() throws Exception {
+
+        if (restoreStateHandles == null || restoreStateHandles.isEmpty()) {
+            return null;
+        }
+
+        logger.info(
+                "Starting ForSt incremental recovery in operator {}, target 
key-group range {}. State Handles={}",
+                operatorIdentifier,
+                keyGroupRange.prettyPrintInterval(),
+                restoreStateHandles);
+
+        List<IncrementalRemoteKeyedStateHandle> otherHandles =
+                new ArrayList<>(restoreStateHandles.size() - 1);
+        IncrementalRemoteKeyedStateHandle mainHandle =
+                chooseMainHandleAndCollectOthers(otherHandles);
+
+        StateHandleTransferSpec mainSpec =
+                new StateHandleTransferSpec(
+                        mainHandle, new Path(forstBasePath, 
ForStResourceContainer.DB_DIR_STRING));
+
+        List<StateHandleTransferSpec> otherSpecs =
+                otherHandles.stream()
+                        .map(
+                                handle ->
+                                        new StateHandleTransferSpec(
+                                                handle,
+                                                new Path(
+                                                        forstBasePath,
+                                                        
UUID.randomUUID().toString())))
+                        .collect(Collectors.toList());
+
+        try {
+            runAndReportDuration(
+                    () -> transferAllStateHandles(mainSpec, otherSpecs),
+                    // TODO: use new metric name, such as 
"TransferStateDurationMs"
+                    DOWNLOAD_STATE_DURATION);
+
+            runAndReportDuration(
+                    () -> restoreFromTransferredHandles(mainSpec, otherSpecs),
+                    RESTORE_STATE_DURATION);
+
+            return new ForStRestoreResult(
+                    this.forstHandle.getDb(),
+                    this.forstHandle.getDefaultColumnFamilyHandle(),
+                    this.forstHandle.getNativeMetricMonitor(),
+                    lastCompletedCheckpointId,
+                    backendUID,
+                    restoredSstFiles);
+        } finally {
+            // Delete the transfer destination quietly.
+            otherSpecs.stream()
+                    .map(StateHandleTransferSpec::getTransferDestination)
+                    .forEach(
+                            dir -> {
+                                try {
+                                    FileSystem fs = dir.getFileSystem();
+                                    fs.delete(dir, true);
+                                } catch (IOException ignored) {
+
+                                }
+                            });
+        }
+    }
+
+    private IncrementalRemoteKeyedStateHandle chooseMainHandleAndCollectOthers(
+            final List<IncrementalRemoteKeyedStateHandle> 
otherHandlesCollector) {
+        IncrementalRemoteKeyedStateHandle mainHandle = null;
+
+        // Just choose the handle with the biggest size
+        for (IncrementalRemoteKeyedStateHandle handle : restoreStateHandles) {
+            if (mainHandle == null) {
+                mainHandle = handle;
+            } else {
+                if (handle.getStateSize() > mainHandle.getStateSize()) {
+                    otherHandlesCollector.add(mainHandle);
+                    mainHandle = handle;
+                } else {
+                    otherHandlesCollector.add(handle);
+                }
+            }
+        }
+        return mainHandle;
+    }
+
+    private void transferAllStateHandles(
+            StateHandleTransferSpec mainSpecs, List<StateHandleTransferSpec> 
otherSpecs)
+            throws Exception {
+        try (ForStStateDataTransfer transfer =
+                new 
ForStStateDataTransfer(ForStStateDataTransfer.DEFAULT_THREAD_NUM)) {
+            List<StateHandleTransferSpec> specs = new 
ArrayList<>(otherSpecs.size() + 1);
+            specs.add(mainSpecs);
+            specs.addAll(otherSpecs);
+            transfer.transferAllStateDataToDirectory(specs, 
cancelStreamRegistry);
+        }
+    }
+
+    private void restoreFromTransferredHandles(
+            StateHandleTransferSpec mainHandle, List<StateHandleTransferSpec> 
temporaryHandles)
+            throws Exception {
+
+        restoreFromMainTransferredHandle(mainHandle);
+
+        mergeOtherTransferredHandles(temporaryHandles);
+    }
+
+    private void restoreFromMainTransferredHandle(StateHandleTransferSpec 
mainHandel)

Review Comment:
   typo: mainHandel -> mainHandle



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/restore/ForStIncrementalRestoreOperation.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.restore;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
+import 
org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath;
+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.RegisteredStateMetaInfoBase;
+import org.apache.flink.runtime.state.StateBackend.CustomInitializationMetrics;
+import org.apache.flink.runtime.state.StateSerializerProvider;
+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.ForStNativeMetricOptions;
+import org.apache.flink.state.forst.ForStOperationUtils;
+import org.apache.flink.state.forst.ForStResourceContainer;
+import org.apache.flink.state.forst.ForStStateDataTransfer;
+import org.apache.flink.state.forst.StateHandleTransferSpec;
+import org.apache.flink.util.StateMigrationException;
+import org.apache.flink.util.clock.SystemClock;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.runtime.metrics.MetricNames.DOWNLOAD_STATE_DURATION;
+import static 
org.apache.flink.runtime.metrics.MetricNames.RESTORE_STATE_DURATION;
+
+/** Encapsulates the process of restoring a ForSt instance from an incremental 
snapshot. */
+public class ForStIncrementalRestoreOperation<K> implements 
ForStRestoreOperation {
+
+    private static final Logger logger =
+            LoggerFactory.getLogger(ForStIncrementalRestoreOperation.class);
+
+    private final String operatorIdentifier;
+    private final SortedMap<Long, Collection<HandleAndLocalPath>> 
restoredSstFiles;
+    private final ForStHandle forstHandle;
+    private final Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles;
+    private final CloseableRegistry cancelStreamRegistry;
+    private final KeyGroupRange keyGroupRange;
+    private final Path forstBasePath;
+    private final StateSerializerProvider<K> keySerializerProvider;
+    private final ClassLoader userCodeClassLoader;
+    private final CustomInitializationMetrics customInitializationMetrics;
+    private long lastCompletedCheckpointId;
+    private UUID backendUID;
+
+    private boolean isKeySerializerCompatibilityChecked;
+
+    public ForStIncrementalRestoreOperation(
+            String operatorIdentifier,
+            KeyGroupRange keyGroupRange,
+            CloseableRegistry cancelStreamRegistry,
+            ClassLoader userCodeClassLoader,
+            Map<String, ForStKvStateInfo> kvStateInformation,
+            StateSerializerProvider<K> keySerializerProvider,
+            Path forstBasePath,
+            File instanceRocksDBPath,
+            DBOptions dbOptions,
+            Function<String, ColumnFamilyOptions> columnFamilyOptionsFactory,
+            ForStNativeMetricOptions nativeMetricOptions,
+            MetricGroup metricGroup,
+            CustomInitializationMetrics customInitializationMetrics,
+            @Nonnull Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles) {
+        this.forstHandle =
+                new ForStHandle(
+                        kvStateInformation,
+                        instanceRocksDBPath,
+                        dbOptions,
+                        columnFamilyOptionsFactory,
+                        nativeMetricOptions,
+                        metricGroup);
+        this.operatorIdentifier = operatorIdentifier;
+        this.restoredSstFiles = new TreeMap<>();
+        this.lastCompletedCheckpointId = -1L;
+        this.backendUID = UUID.randomUUID();
+        this.customInitializationMetrics = customInitializationMetrics;
+        this.restoreStateHandles = restoreStateHandles;
+        this.cancelStreamRegistry = cancelStreamRegistry;
+        this.keyGroupRange = keyGroupRange;
+        this.forstBasePath = forstBasePath;
+        this.keySerializerProvider = keySerializerProvider;
+        this.userCodeClassLoader = userCodeClassLoader;
+    }
+
+    /**
+     * Root method that branches for different implementations of {@link
+     * IncrementalKeyedStateHandle}.
+     */
+    @Override
+    public ForStRestoreResult restore() throws Exception {
+
+        if (restoreStateHandles == null || restoreStateHandles.isEmpty()) {
+            return null;
+        }
+
+        logger.info(
+                "Starting ForSt incremental recovery in operator {}, target 
key-group range {}. State Handles={}",
+                operatorIdentifier,
+                keyGroupRange.prettyPrintInterval(),
+                restoreStateHandles);
+
+        List<IncrementalRemoteKeyedStateHandle> otherHandles =
+                new ArrayList<>(restoreStateHandles.size() - 1);
+        IncrementalRemoteKeyedStateHandle mainHandle =
+                chooseMainHandleAndCollectOthers(otherHandles);
+
+        StateHandleTransferSpec mainSpec =
+                new StateHandleTransferSpec(
+                        mainHandle, new Path(forstBasePath, 
ForStResourceContainer.DB_DIR_STRING));

Review Comment:
   Why should the base one be downloaded into "db" folder?



##########
flink-state-backends/flink-statebackend-forst/src/main/java/org/apache/flink/state/forst/restore/ForStIncrementalRestoreOperation.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.restore;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
+import 
org.apache.flink.runtime.state.IncrementalKeyedStateHandle.HandleAndLocalPath;
+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.RegisteredStateMetaInfoBase;
+import org.apache.flink.runtime.state.StateBackend.CustomInitializationMetrics;
+import org.apache.flink.runtime.state.StateSerializerProvider;
+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.ForStNativeMetricOptions;
+import org.apache.flink.state.forst.ForStOperationUtils;
+import org.apache.flink.state.forst.ForStResourceContainer;
+import org.apache.flink.state.forst.ForStStateDataTransfer;
+import org.apache.flink.state.forst.StateHandleTransferSpec;
+import org.apache.flink.util.StateMigrationException;
+import org.apache.flink.util.clock.SystemClock;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.runtime.metrics.MetricNames.DOWNLOAD_STATE_DURATION;
+import static 
org.apache.flink.runtime.metrics.MetricNames.RESTORE_STATE_DURATION;
+
+/** Encapsulates the process of restoring a ForSt instance from an incremental 
snapshot. */
+public class ForStIncrementalRestoreOperation<K> implements 
ForStRestoreOperation {
+
+    private static final Logger logger =
+            LoggerFactory.getLogger(ForStIncrementalRestoreOperation.class);
+
+    private final String operatorIdentifier;
+    private final SortedMap<Long, Collection<HandleAndLocalPath>> 
restoredSstFiles;
+    private final ForStHandle forstHandle;
+    private final Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles;
+    private final CloseableRegistry cancelStreamRegistry;
+    private final KeyGroupRange keyGroupRange;
+    private final Path forstBasePath;
+    private final StateSerializerProvider<K> keySerializerProvider;
+    private final ClassLoader userCodeClassLoader;
+    private final CustomInitializationMetrics customInitializationMetrics;
+    private long lastCompletedCheckpointId;
+    private UUID backendUID;
+
+    private boolean isKeySerializerCompatibilityChecked;
+
+    public ForStIncrementalRestoreOperation(
+            String operatorIdentifier,
+            KeyGroupRange keyGroupRange,
+            CloseableRegistry cancelStreamRegistry,
+            ClassLoader userCodeClassLoader,
+            Map<String, ForStKvStateInfo> kvStateInformation,
+            StateSerializerProvider<K> keySerializerProvider,
+            Path forstBasePath,
+            File instanceRocksDBPath,
+            DBOptions dbOptions,
+            Function<String, ColumnFamilyOptions> columnFamilyOptionsFactory,
+            ForStNativeMetricOptions nativeMetricOptions,
+            MetricGroup metricGroup,
+            CustomInitializationMetrics customInitializationMetrics,
+            @Nonnull Collection<IncrementalRemoteKeyedStateHandle> 
restoreStateHandles) {
+        this.forstHandle =
+                new ForStHandle(
+                        kvStateInformation,
+                        instanceRocksDBPath,
+                        dbOptions,
+                        columnFamilyOptionsFactory,
+                        nativeMetricOptions,
+                        metricGroup);
+        this.operatorIdentifier = operatorIdentifier;
+        this.restoredSstFiles = new TreeMap<>();
+        this.lastCompletedCheckpointId = -1L;
+        this.backendUID = UUID.randomUUID();
+        this.customInitializationMetrics = customInitializationMetrics;
+        this.restoreStateHandles = restoreStateHandles;
+        this.cancelStreamRegistry = cancelStreamRegistry;
+        this.keyGroupRange = keyGroupRange;
+        this.forstBasePath = forstBasePath;
+        this.keySerializerProvider = keySerializerProvider;
+        this.userCodeClassLoader = userCodeClassLoader;
+    }
+
+    /**
+     * Root method that branches for different implementations of {@link
+     * IncrementalKeyedStateHandle}.
+     */
+    @Override
+    public ForStRestoreResult restore() throws Exception {
+
+        if (restoreStateHandles == null || restoreStateHandles.isEmpty()) {
+            return null;
+        }
+
+        logger.info(
+                "Starting ForSt incremental recovery in operator {}, target 
key-group range {}. State Handles={}",
+                operatorIdentifier,
+                keyGroupRange.prettyPrintInterval(),
+                restoreStateHandles);
+
+        List<IncrementalRemoteKeyedStateHandle> otherHandles =
+                new ArrayList<>(restoreStateHandles.size() - 1);
+        IncrementalRemoteKeyedStateHandle mainHandle =
+                chooseMainHandleAndCollectOthers(otherHandles);
+
+        StateHandleTransferSpec mainSpec =
+                new StateHandleTransferSpec(
+                        mainHandle, new Path(forstBasePath, 
ForStResourceContainer.DB_DIR_STRING));
+
+        List<StateHandleTransferSpec> otherSpecs =
+                otherHandles.stream()
+                        .map(
+                                handle ->
+                                        new StateHandleTransferSpec(
+                                                handle,
+                                                new Path(
+                                                        forstBasePath,
+                                                        
UUID.randomUUID().toString())))
+                        .collect(Collectors.toList());
+
+        try {
+            runAndReportDuration(
+                    () -> transferAllStateHandles(mainSpec, otherSpecs),
+                    // TODO: use new metric name, such as 
"TransferStateDurationMs"
+                    DOWNLOAD_STATE_DURATION);
+
+            runAndReportDuration(
+                    () -> restoreFromTransferredHandles(mainSpec, otherSpecs),
+                    RESTORE_STATE_DURATION);
+
+            return new ForStRestoreResult(
+                    this.forstHandle.getDb(),
+                    this.forstHandle.getDefaultColumnFamilyHandle(),
+                    this.forstHandle.getNativeMetricMonitor(),
+                    lastCompletedCheckpointId,
+                    backendUID,
+                    restoredSstFiles);
+        } finally {
+            // Delete the transfer destination quietly.
+            otherSpecs.stream()
+                    .map(StateHandleTransferSpec::getTransferDestination)
+                    .forEach(
+                            dir -> {
+                                try {
+                                    FileSystem fs = dir.getFileSystem();
+                                    fs.delete(dir, true);
+                                } catch (IOException ignored) {
+
+                                }
+                            });
+        }
+    }
+
+    private IncrementalRemoteKeyedStateHandle chooseMainHandleAndCollectOthers(
+            final List<IncrementalRemoteKeyedStateHandle> 
otherHandlesCollector) {
+        IncrementalRemoteKeyedStateHandle mainHandle = null;
+
+        // Just choose the handle with the biggest size
+        for (IncrementalRemoteKeyedStateHandle handle : restoreStateHandles) {
+            if (mainHandle == null) {
+                mainHandle = handle;
+            } else {
+                if (handle.getStateSize() > mainHandle.getStateSize()) {
+                    otherHandlesCollector.add(mainHandle);
+                    mainHandle = handle;
+                } else {
+                    otherHandlesCollector.add(handle);
+                }
+            }
+        }

Review Comment:
   Why not do something like 
`RocksDBIncrementalCheckpointUtils.findTheBestStateHandleForInitial` does?



-- 
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

Reply via email to