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/10/18 10:26:16 UTC

[GitHub] [flink] rkhachatryan commented on a change in pull request #16606: [FLINK-21357][runtime/statebackend]Periodic materialization for generalized incremental checkpoints

rkhachatryan commented on a change in pull request #16606:
URL: https://github.com/apache/flink/pull/16606#discussion_r730782566



##########
File path: flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.changelog;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.core.fs.FileSystemSafetyNet;
+import org.apache.flink.runtime.state.KeyedStateHandle;
+import org.apache.flink.runtime.state.SnapshotResult;
+import org.apache.flink.runtime.state.StateObject;
+import org.apache.flink.runtime.state.changelog.SequenceNumber;
+import org.apache.flink.runtime.taskmanager.AsyncExceptionHandler;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Stateless Materialization Manager. */
+public class PeriodicMaterializationManager implements Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(PeriodicMaterializationManager.class);
+
+    /** task mailbox executor, execute from Task Thread. */
+    private final MailboxExecutor mailboxExecutor;
+
+    /** Async thread pool, to complete async phase of materialization. */
+    private final ExecutorService asyncOperationsThreadPool;
+
+    /** scheduled executor, periodically trigger materialization. */
+    private final ScheduledExecutorService periodicExecutor;
+
+    private final AsyncExceptionHandler asyncExceptionHandler;
+
+    private final String subtaskName;
+
+    private final long periodicMaterializeDelay;
+
+    /** Allowed number of consecutive materialization failures. */
+    private final int allowedNumberOfFailures;
+
+    /** Number of consecutive materialization failures. */
+    private final AtomicInteger numberOfConsecutiveFailures;
+
+    private final ChangelogKeyedStateBackend<?> keyedStateBackend;
+
+    private boolean started = false;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            ChangelogKeyedStateBackend<?> keyedStateBackend,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures) {
+        this.mailboxExecutor = checkNotNull(mailboxExecutor);
+        this.asyncOperationsThreadPool = checkNotNull(asyncOperationsThreadPool);
+        this.subtaskName = checkNotNull(subtaskName);
+        this.asyncExceptionHandler = checkNotNull(asyncExceptionHandler);
+        this.keyedStateBackend = checkNotNull(keyedStateBackend);
+
+        this.periodicMaterializeDelay = periodicMaterializeDelay;
+        this.allowedNumberOfFailures = allowedNumberOfFailures;
+        this.numberOfConsecutiveFailures = new AtomicInteger(0);
+
+        this.periodicExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory(
+                                "periodic-materialization-scheduler-" + subtaskName));
+    }
+
+    public void start() {
+        if (!started) {
+
+            started = true;
+
+            LOG.info(
+                    "Task {} starts periodic materialization, scheduling the next one in {} seconds",
+                    subtaskName,
+                    periodicMaterializeDelay / 1000);
+
+            scheduleNextMaterialization();
+        }
+    }
+
+    private void triggerMaterialization() {
+        mailboxExecutor.execute(
+                () -> {
+                    Optional<MaterializationRunnable> materializationRunnableOptional =
+                            keyedStateBackend.initMaterialization();
+
+                    if (materializationRunnableOptional.isPresent()) {
+                        MaterializationRunnable runnable = materializationRunnableOptional.get();
+                        asyncOperationsThreadPool.execute(
+                                () ->
+                                        asyncMaterializationPhase(
+                                                runnable.getMaterializationRunnable(),
+                                                runnable.getMaterializedTo()));
+                    } else {
+                        scheduleNextMaterialization();
+
+                        LOG.info(
+                                "Task {} has no state updates since last materialization, "
+                                        + "skip this one and schedule the next one in {} seconds",
+                                subtaskName,
+                                periodicMaterializeDelay / 1000);
+                    }
+                },
+                "materialization");
+    }
+
+    private void asyncMaterializationPhase(
+            RunnableFuture<SnapshotResult<KeyedStateHandle>> materializedRunnableFuture,
+            SequenceNumber upTo) {
+
+        SnapshotResult<KeyedStateHandle> materializedSnapshot =
+                uploadSnapshot(materializedRunnableFuture);
+
+        // if succeed, update state and finish up
+        if (materializedSnapshot != null) {
+
+            numberOfConsecutiveFailures.set(0);
+
+            final SnapshotResult<KeyedStateHandle> copyMaterializedSnapshot = materializedSnapshot;
+
+            mailboxExecutor.execute(
+                    () ->
+                            keyedStateBackend.updateChangelogSnapshotState(
+                                    copyMaterializedSnapshot,
+                                    upTo),
+                    "Task {} update materializedSnapshot up to changelog sequence number: {}",
+                    subtaskName,
+                    upTo);
+        }
+
+        LOG.info(
+                "Task {} schedules the next materialization in {} seconds.",
+                subtaskName,
+                periodicMaterializeDelay / 1000);
+
+        scheduleNextMaterialization();
+    }
+
+    private SnapshotResult<KeyedStateHandle> uploadSnapshot(
+            RunnableFuture<SnapshotResult<KeyedStateHandle>> materializedRunnableFuture) {
+
+        FileSystemSafetyNet.initializeSafetyNetForThread();
+        try {
+            FutureUtils.runIfNotDoneAndGet(materializedRunnableFuture);
+
+            LOG.debug("Task {} finishes asynchronous part of materialization.", subtaskName);
+
+            return materializedRunnableFuture.get();
+
+        } catch (Exception e) {
+            int retryTime = numberOfConsecutiveFailures.incrementAndGet();
+
+            LOG.info(
+                    "Task {} asynchronous part of materialization is not completed for the {} time.",
+                    subtaskName,
+                    retryTime,
+                    e);
+
+            discardFailedUploads(materializedRunnableFuture);
+
+            if (retryTime >= allowedNumberOfFailures) {
+                // Fail the task externally, this causes task failover
+                asyncExceptionHandler.handleAsyncException(
+                        "Task "
+                                + subtaskName
+                                + " fails to complete the asynchronous part of materialization",
+                        e);
+            }
+        } finally {
+            FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+        }
+
+        return null;
+    }
+
+    private void discardFailedUploads(
+            RunnableFuture<SnapshotResult<KeyedStateHandle>> materializedRunnableFuture) {
+
+        LOG.info("Task {} cleanup asynchronous runnable for materialization.", subtaskName);
+
+        if (materializedRunnableFuture != null) {
+            // materialization has started
+            if (!materializedRunnableFuture.cancel(true)) {
+                try {
+                    StateObject stateObject = materializedRunnableFuture.get();
+                    if (stateObject != null) {
+                        stateObject.discardState();
+                    }
+                } catch (Exception ex) {
+                    LOG.debug(
+                            "Task "
+                                    + subtaskName
+                                    + " cancelled execution of snapshot future runnable. "
+                                    + "Cancellation produced the following "
+                                    + "exception, which is expected and can be ignored.",
+                            ex);
+                }
+            }
+        }
+    }
+
+    // Only be called in the task thread to simplify the threading model
+    private void scheduleNextMaterialization() {
+        periodicExecutor.schedule(
+                this::triggerMaterialization, periodicMaterializeDelay, TimeUnit.MILLISECONDS);
+    }

Review comment:
       I think there is a race condition in the code you added to address this:
   ```
           if (!shutdown) {
               periodicExecutor.schedule(
   ```
   Even with `shutdown` being `volatile`, it can be set by T1, then checked by T2, then executor closed by T1, then schedule attempt made by T2.




-- 
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-unsubscribe@flink.apache.org

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