You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/21 16:05:49 UTC

[GitHub] [flink] aljoscha commented on a change in pull request #14719: [FLINK-21072] Refactor the SnapshotStrategy hierarchy

aljoscha commented on a change in pull request #14719:
URL: https://github.com/apache/flink/pull/14719#discussion_r561991514



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableFactory.java
##########
@@ -22,18 +22,14 @@
 import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo;
 
 /**
- * Interface for synchronicity behavior of heap snapshot strategy.
+ * A factory method for creating a {@link StateTable}.

Review comment:
       ```suggestion
    * A factory for {@link StateTable StateTables}.
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackendSnapshotStrategy.java
##########
@@ -106,119 +99,133 @@ protected DefaultOperatorStateBackendSnapshotStrategy(
             Thread.currentThread().setContextClassLoader(snapshotClassLoader);
         }
 
-        AsyncSnapshotCallable<SnapshotResult<OperatorStateHandle>> snapshotCallable =
-                new AsyncSnapshotCallable<SnapshotResult<OperatorStateHandle>>() {
-
-                    @Override
-                    protected SnapshotResult<OperatorStateHandle> callInternal() throws Exception {
-
-                        CheckpointStreamFactory.CheckpointStateOutputStream localOut =
-                                streamFactory.createCheckpointStateOutputStream(
-                                        CheckpointedStateScope.EXCLUSIVE);
-                        snapshotCloseableRegistry.registerCloseable(localOut);
-
-                        // get the registered operator state infos ...
-                        List<StateMetaInfoSnapshot> operatorMetaInfoSnapshots =
-                                new ArrayList<>(registeredOperatorStatesDeepCopies.size());
-
-                        for (Map.Entry<String, PartitionableListState<?>> entry :
-                                registeredOperatorStatesDeepCopies.entrySet()) {
-                            operatorMetaInfoSnapshots.add(
-                                    entry.getValue().getStateMetaInfo().snapshot());
-                        }
-
-                        // ... get the registered broadcast operator state infos ...
-                        List<StateMetaInfoSnapshot> broadcastMetaInfoSnapshots =
-                                new ArrayList<>(registeredBroadcastStatesDeepCopies.size());
-
-                        for (Map.Entry<String, BackendWritableBroadcastState<?, ?>> entry :
-                                registeredBroadcastStatesDeepCopies.entrySet()) {
-                            broadcastMetaInfoSnapshots.add(
-                                    entry.getValue().getStateMetaInfo().snapshot());
-                        }
-
-                        // ... write them all in the checkpoint stream ...
-                        DataOutputView dov = new DataOutputViewStreamWrapper(localOut);
-
-                        OperatorBackendSerializationProxy backendSerializationProxy =
-                                new OperatorBackendSerializationProxy(
-                                        operatorMetaInfoSnapshots, broadcastMetaInfoSnapshots);
-
-                        backendSerializationProxy.write(dov);
-
-                        // ... and then go for the states ...
-
-                        // we put BOTH normal and broadcast state metadata here
-                        int initialMapCapacity =
-                                registeredOperatorStatesDeepCopies.size()
-                                        + registeredBroadcastStatesDeepCopies.size();
-                        final Map<String, OperatorStateHandle.StateMetaInfo> writtenStatesMetaData =
-                                new HashMap<>(initialMapCapacity);
-
-                        for (Map.Entry<String, PartitionableListState<?>> entry :
-                                registeredOperatorStatesDeepCopies.entrySet()) {
-
-                            PartitionableListState<?> value = entry.getValue();
-                            long[] partitionOffsets = value.write(localOut);
-                            OperatorStateHandle.Mode mode =
-                                    value.getStateMetaInfo().getAssignmentMode();
-                            writtenStatesMetaData.put(
-                                    entry.getKey(),
-                                    new OperatorStateHandle.StateMetaInfo(partitionOffsets, mode));
-                        }
-
-                        // ... and the broadcast states themselves ...
-                        for (Map.Entry<String, BackendWritableBroadcastState<?, ?>> entry :
-                                registeredBroadcastStatesDeepCopies.entrySet()) {
-
-                            BackendWritableBroadcastState<?, ?> value = entry.getValue();
-                            long[] partitionOffsets = {value.write(localOut)};
-                            OperatorStateHandle.Mode mode =
-                                    value.getStateMetaInfo().getAssignmentMode();
-                            writtenStatesMetaData.put(
-                                    entry.getKey(),
-                                    new OperatorStateHandle.StateMetaInfo(partitionOffsets, mode));
-                        }
-
-                        // ... and, finally, create the state handle.
-                        OperatorStateHandle retValue = null;
-
-                        if (snapshotCloseableRegistry.unregisterCloseable(localOut)) {
-
-                            StreamStateHandle stateHandle = localOut.closeAndGetHandle();
-
-                            if (stateHandle != null) {
-                                retValue =
-                                        new OperatorStreamStateHandle(
-                                                writtenStatesMetaData, stateHandle);
-                            }
-
-                            return SnapshotResult.of(retValue);
-                        } else {
-                            throw new IOException("Stream was already unregistered.");
-                        }
-                    }
+        return new DefaultOperatorStateBackendSnapshotResources(
+                registeredOperatorStatesDeepCopies, registeredBroadcastStatesDeepCopies);
+    }
 
-                    @Override
-                    protected void cleanupProvidedResources() {
-                        // nothing to do
-                    }
+    @Override
+    public SupplierWithException<SnapshotResult<OperatorStateHandle>, ? extends Exception>
+            asyncSnapshot(
+                    DefaultOperatorStateBackendSnapshotResources syncPartResource,
+                    long checkpointId,
+                    long timestamp,
+                    @Nonnull CheckpointStreamFactory streamFactory,
+                    @Nonnull CheckpointOptions checkpointOptions) {
+
+        if (syncPartResource.getRegisteredBroadcastStatesDeepCopies().isEmpty()

Review comment:
       You could get the sync-part resources once instead of always calling methods on the "Resources". This would make the diff smaller, at least for me when I'm looking at it in the IDE.
   
   Same holds for the other refactored strategies.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshottable.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.RunnableFuture;
+
+/**
+ * Interface for objects that can snapshot its state (state backends currently). Implementing
+ * classes should ideally be stateless or at least threadsafe, i.e. this is a functional interface
+ * and is can be called in parallel by multiple checkpoints.
+ *
+ * @param <S> type of the returned state object that represents the result of the snapshot
+ *     operation.
+ * @see SnapshotStrategy
+ * @see SnapshotStrategyRunner
+ */
+@Internal
+public interface Snapshottable<S extends StateObject> {

Review comment:
       I would go for `Snapshotable`, that's also what other similar projects seem to use: https://docs.oracle.com/cd/E10926_01/doc/owb.101/b12155/oracle/owb/metadatamgmt/changemgmt/Snapshotable.html

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
##########
@@ -83,7 +83,7 @@
 
     private final Map<String, BackendWritableBroadcastState<?, ?>> accessedBroadcastStatesByName;
 
-    private final AbstractSnapshotStrategy<OperatorStateHandle> snapshotStrategy;
+    private final SnapshotStrategyRunner<OperatorStateHandle, ?> snapshotStrategy;

Review comment:
       In some places this could now be called `snapshotStrategyRunner` to reflect that it's now a runner.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotStrategyRunner.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.util.concurrent.RunnableFuture;
+
+/**
+ * A class to execute a {@link SnapshotStrategy}. It can execute a strategy either synchronously or
+ * asynchronously. It takes care of common logging and resource cleaning.
+ *
+ * @param <T> type of the snapshot result.
+ */
+public final class SnapshotStrategyRunner<T extends StateObject, SR extends SnapshotResources> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SnapshotStrategyRunner.class);
+
+    private static final String LOG_SYNC_COMPLETED_TEMPLATE =
+            "{} ({}, synchronous part) in thread {} took {} ms.";
+    private static final String LOG_ASYNC_COMPLETED_TEMPLATE =
+            "{} ({}, asynchronous part) in thread {} took {} ms.";
+
+    /**
+     * Descriptive name of the snapshot strategy that will appear in the log outputs and {@link
+     * #toString()}.
+     */
+    @Nonnull private final String description;
+
+    @Nonnull private final SnapshotStrategy<T, SR> snapshotStrategy;
+    @Nonnull private final CloseableRegistry cancelStreamRegistry;
+
+    @Nonnull private final boolean isSynchronous;
+
+    public SnapshotStrategyRunner(
+            @Nonnull String description,
+            @Nonnull SnapshotStrategy<T, SR> snapshotStrategy,
+            @Nonnull CloseableRegistry cancelStreamRegistry,
+            boolean isSynchronous) {

Review comment:
       nit: I was already confused about what `true` or `false` meant when I saw callsites. Maybe some enum with `SYNCHRONOUS` `ASYNCHRONOUS` would help.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org