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



##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -531,18 +539,76 @@ public void notifyCheckpointAborted(long checkpointId) 
throws Exception {
         return is;
     }
 
-    private void completeRestore(Collection<ChangelogStateBackendHandle> 
stateHandles) {
-        if (!stateHandles.isEmpty()) {
-            synchronized (materialized) { // ensure visibility
-                for (ChangelogStateBackendHandle h : stateHandles) {
-                    if (h != null) {
-                        materialized.addAll(h.getMaterializedStateHandles());
-                        
restoredNonMaterialized.addAll(h.getNonMaterializedStateHandles());
-                    }
-                }
+    private ChangelogSnapshotState completeRestore(
+            Collection<ChangelogStateBackendHandle> stateHandles) {
+
+        List<KeyedStateHandle> materialized = new ArrayList<>();
+        List<ChangelogStateHandle> restoredNonMaterialized = new ArrayList<>();
+
+        for (ChangelogStateBackendHandle h : stateHandles) {
+            if (h != null) {
+                materialized.addAll(h.getMaterializedStateHandles());
+                
restoredNonMaterialized.addAll(h.getNonMaterializedStateHandles());
             }
         }
+
         changelogStates.clear();
+        return new ChangelogSnapshotState(
+                materialized,
+                restoredNonMaterialized,
+                stateChangelogWriter.initialSequenceNumber());
+    }
+
+    /**
+     * Initialize state materialization so that materialized data can be 
persisted durably and
+     * included into the checkpoint.
+     *
+     * <p>This method is not thread safe. It should be called either under a 
lock or through task
+     * mailbox executor.
+     *
+     * @return a tuple of - future snapshot result from the underlying state 
backend - a {@link
+     *     SequenceNumber} identifying the latest change in the changelog
+     */
+    public Optional<MaterializationRunnable> initMaterialization() throws 
Exception {
+        SequenceNumber upTo = 
stateChangelogWriter.lastAppendedSequenceNumber();
+
+        if (upTo.compareTo(changelogSnapshotState.lastMaterializedTo()) > 0) {
+            return Optional.of(
+                    new MaterializationRunnable(
+                            keyedStateBackend.snapshot(
+                                    // This ID is not needed for 
materialization;
+                                    // But since we are re-using the 
streamFactory
+                                    // that is designed for state backend 
snapshot,
+                                    // which requires unique checkpoint ID.
+                                    // A faked materialized Id is provided 
here.
+                                    // TODO: implement its own streamFactory.
+                                    materializedId++,
+                                    System.currentTimeMillis(),
+                                    streamFactory,
+                                    CHECKPOINT_OPTIONS),
+                            // TODO: add metadata to log FLINK-23170
+                            upTo));
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    /**
+     * This method is not thread safe. It should be called either under a lock 
or through task
+     * mailbox executor.
+     */
+    public void updateChangelogSnapshotState(
+            SnapshotResult<KeyedStateHandle> materializedSnapshot, 
SequenceNumber upTo) {
+        changelogSnapshotState =
+                new ChangelogSnapshotState(
+                        getMaterializedResult(materializedSnapshot), new 
ArrayList<>(), upTo);

Review comment:
       nit: `Collections.emptyList`?
   nit: factory method with two args?
   

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -625,4 +691,52 @@ public T get(long timeout, TimeUnit unit)
             }
         };
     }
+
+    /**
+     * Snapshot State for ChangelogKeyedStatebackend.
+     *
+     * <p>It includes three parts: - materialized snapshot from the underlying 
delegated state
+     * backend - non-materialized part in the current changelog - 
non-materialized changelog, from
+     * previous logs (before failover or rescaling)
+     */
+    private static class ChangelogSnapshotState {
+        /**
+         * Materialized snapshot from the underlying delegated state backend. 
Set initially on
+         * restore and later upon materialization.
+         */
+        private final List<KeyedStateHandle> materializedSnapshot;
+
+        /**
+         * The {@link SequenceNumber} up to which the state is materialized, 
exclusive. This
+         * indicates the non-materialized part of the current changelog.
+         */
+        private final SequenceNumber materializedTo;
+
+        /**
+         * Non-materialized changelog, from previous logs. Set initially on 
restore and later
+         * cleared upon materialization.
+         */
+        private final List<ChangelogStateHandle> restoredNonMaterialized;
+
+        public ChangelogSnapshotState(
+                List<KeyedStateHandle> materializedSnapshot,
+                List<ChangelogStateHandle> restoredNonMaterialized,
+                SequenceNumber materializedTo) {
+            this.materializedSnapshot = 
checkNotNull(unmodifiableList((materializedSnapshot)));
+            this.restoredNonMaterialized = 
checkNotNull(unmodifiableList(restoredNonMaterialized));

Review comment:
       Isn't null check already done in `unmodifiableList`?

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/ChangelogOptions.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to changelog. */
+public class ChangelogOptions {
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Duration> 
PERIODIC_MATERIALIZATION_INTERVAL =
+            ConfigOptions.key("state.backend.periodic-materialize.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMinutes(10))
+                    .withDescription(
+                            "Defines the interval in milliseconds to perform 
periodic materialization for state backend.");
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Boolean> PERIODIC_MATERIALIZATION_ENABLED 
=
+            ConfigOptions.key("state.backend.periodic-materialize.enabled")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            "Whether to perform periodic materialization for 
state backend.");
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Integer> 
MATERIALIZATION_MAX_ALLOWED_FAILURES =
+            ConfigOptions.key("state.backend.periodic-materialize.enabled")

Review comment:
       Wrong key I guess?

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/ChangelogOptions.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to changelog. */
+public class ChangelogOptions {

Review comment:
       Could you please extract this change into a separate commit?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/AsyncExceptionHandler.java
##########
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.streaming.runtime.tasks;
+package org.apache.flink.runtime.taskmanager;

Review comment:
       Could you please extract this change (moving of `AsyncExceptionHandler`) 
into a separate commit?

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/ChangelogOptions.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to changelog. */
+public class ChangelogOptions {
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Duration> 
PERIODIC_MATERIALIZATION_INTERVAL =
+            ConfigOptions.key("state.backend.periodic-materialize.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMinutes(10))
+                    .withDescription(
+                            "Defines the interval in milliseconds to perform 
periodic materialization for state backend.");
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Boolean> PERIODIC_MATERIALIZATION_ENABLED 
=
+            ConfigOptions.key("state.backend.periodic-materialize.enabled")

Review comment:
       Remove? (as discussed above)

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+/** 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;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures,
+            ChangelogKeyedStateBackend<?> keyedStateBackend) {
+        this.mailboxExecutor = mailboxExecutor;
+        this.asyncOperationsThreadPool = asyncOperationsThreadPool;
+
+        this.subtaskName = subtaskName;
+        this.periodicMaterializeDelay = periodicMaterializeDelay;
+        this.asyncExceptionHandler = asyncExceptionHandler;
+        this.allowedNumberOfFailures = allowedNumberOfFailures;
+        this.numberOfConsecutiveFailures = new AtomicInteger(0);
+        this.keyedStateBackend = keyedStateBackend;
+
+        this.periodicExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory(
+                                "periodic-materialization-scheduler-" + 
subtaskName));
+    }
+
+    public void start() {
+        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 = null;
+        FileSystemSafetyNet.initializeSafetyNetForThread();
+        try {
+            FutureUtils.runIfNotDoneAndGet(materializedRunnableFuture);
+
+            LOG.debug("Task {} finishes asynchronous part of 
materialization.", subtaskName);
+
+            materializedSnapshot = 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);
+
+            handleExecutionException(materializedRunnableFuture);
+
+            if (retryTime >= allowedNumberOfFailures) {
+                // Fail the task externally
+                asyncExceptionHandler.handleAsyncException(
+                        "Task "
+                                + subtaskName
+                                + " fails to complete the asynchronous part of 
materialization",
+                        e);
+
+                return;
+            }
+        } finally {
+            FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+        }
+
+        // if succeed, update state and finish up
+        if (materializedSnapshot != null) {

Review comment:
       The flow isn't very clear to follow here. How about extracting the code 
above into a function that creates this `materializedSnapshot`?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+/** 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;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures,
+            ChangelogKeyedStateBackend<?> keyedStateBackend) {
+        this.mailboxExecutor = mailboxExecutor;
+        this.asyncOperationsThreadPool = asyncOperationsThreadPool;

Review comment:
       Check for null (ditto other args)?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+/** 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;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures,
+            ChangelogKeyedStateBackend<?> keyedStateBackend) {
+        this.mailboxExecutor = mailboxExecutor;
+        this.asyncOperationsThreadPool = asyncOperationsThreadPool;
+
+        this.subtaskName = subtaskName;
+        this.periodicMaterializeDelay = periodicMaterializeDelay;
+        this.asyncExceptionHandler = asyncExceptionHandler;
+        this.allowedNumberOfFailures = allowedNumberOfFailures;
+        this.numberOfConsecutiveFailures = new AtomicInteger(0);
+        this.keyedStateBackend = keyedStateBackend;
+
+        this.periodicExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory(
+                                "periodic-materialization-scheduler-" + 
subtaskName));
+    }
+
+    public void start() {
+        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 = null;
+        FileSystemSafetyNet.initializeSafetyNetForThread();
+        try {
+            FutureUtils.runIfNotDoneAndGet(materializedRunnableFuture);
+
+            LOG.debug("Task {} finishes asynchronous part of 
materialization.", subtaskName);
+
+            materializedSnapshot = 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);
+
+            handleExecutionException(materializedRunnableFuture);
+
+            if (retryTime >= allowedNumberOfFailures) {
+                // Fail the task externally
+                asyncExceptionHandler.handleAsyncException(
+                        "Task "
+                                + subtaskName
+                                + " fails to complete the asynchronous part of 
materialization",
+                        e);
+
+                return;
+            }
+        } finally {
+            FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+        }
+
+        // if succeed, update state and finish up
+        if (materializedSnapshot != null) {
+
+            final SnapshotResult<KeyedStateHandle> copyMaterializedSnapshot = 
materializedSnapshot;
+
+            mailboxExecutor.execute(
+                    () -> {
+                        keyedStateBackend.updateChangelogSnapshotState(
+                                copyMaterializedSnapshot, upTo);
+                        numberOfConsecutiveFailures.set(0);

Review comment:
       Do we have update `numberOfConsecutiveFailures` in the task thread?

##########
File path: 
flink-core/src/main/java/org/apache/flink/configuration/ChangelogOptions.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.docs.Documentation;
+import org.apache.flink.configuration.description.Description;
+
+import java.time.Duration;
+
+/** A collection of all configuration options that relate to changelog. */
+public class ChangelogOptions {
+
+    @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG)
+    public static final ConfigOption<Duration> 
PERIODIC_MATERIALIZATION_INTERVAL =
+            ConfigOptions.key("state.backend.periodic-materialize.interval")

Review comment:
       Add changelog prefix to key? (ditto other keys)
   Maybe `state.backend.changelog.periodic-materialization.interval`?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+/** 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;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures,
+            ChangelogKeyedStateBackend<?> keyedStateBackend) {
+        this.mailboxExecutor = mailboxExecutor;
+        this.asyncOperationsThreadPool = asyncOperationsThreadPool;
+
+        this.subtaskName = subtaskName;
+        this.periodicMaterializeDelay = periodicMaterializeDelay;
+        this.asyncExceptionHandler = asyncExceptionHandler;
+        this.allowedNumberOfFailures = allowedNumberOfFailures;
+        this.numberOfConsecutiveFailures = new AtomicInteger(0);
+        this.keyedStateBackend = keyedStateBackend;
+
+        this.periodicExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory(
+                                "periodic-materialization-scheduler-" + 
subtaskName));
+    }
+
+    public void start() {

Review comment:
       Can we prevent this method from being called twice?
   Otherwise, we can get very strange behavior (in case of programmatic error 
and calling it twice).

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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;
+
+/** 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;
+
+    PeriodicMaterializationManager(
+            MailboxExecutor mailboxExecutor,
+            ExecutorService asyncOperationsThreadPool,
+            String subtaskName,
+            AsyncExceptionHandler asyncExceptionHandler,
+            long periodicMaterializeDelay,
+            int allowedNumberOfFailures,
+            ChangelogKeyedStateBackend<?> keyedStateBackend) {
+        this.mailboxExecutor = mailboxExecutor;
+        this.asyncOperationsThreadPool = asyncOperationsThreadPool;
+
+        this.subtaskName = subtaskName;
+        this.periodicMaterializeDelay = periodicMaterializeDelay;
+        this.asyncExceptionHandler = asyncExceptionHandler;
+        this.allowedNumberOfFailures = allowedNumberOfFailures;
+        this.numberOfConsecutiveFailures = new AtomicInteger(0);
+        this.keyedStateBackend = keyedStateBackend;
+
+        this.periodicExecutor =
+                Executors.newSingleThreadScheduledExecutor(
+                        new ExecutorThreadFactory(
+                                "periodic-materialization-scheduler-" + 
subtaskName));
+    }
+
+    public void start() {
+        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 = null;
+        FileSystemSafetyNet.initializeSafetyNetForThread();
+        try {
+            FutureUtils.runIfNotDoneAndGet(materializedRunnableFuture);
+
+            LOG.debug("Task {} finishes asynchronous part of 
materialization.", subtaskName);
+
+            materializedSnapshot = 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);
+
+            handleExecutionException(materializedRunnableFuture);
+
+            if (retryTime >= allowedNumberOfFailures) {
+                // Fail the task externally
+                asyncExceptionHandler.handleAsyncException(
+                        "Task "
+                                + subtaskName
+                                + " fails to complete the asynchronous part of 
materialization",
+                        e);
+
+                return;
+            }
+        } finally {
+            FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+        }
+
+        // if succeed, update state and finish up
+        if (materializedSnapshot != null) {
+
+            final SnapshotResult<KeyedStateHandle> copyMaterializedSnapshot = 
materializedSnapshot;
+
+            mailboxExecutor.execute(
+                    () -> {
+                        keyedStateBackend.updateChangelogSnapshotState(
+                                copyMaterializedSnapshot, upTo);
+                        numberOfConsecutiveFailures.set(0);
+                    },
+                    "Task {} update materializedSnapshot up to changelog 
sequence number: {}",
+                    subtaskName,
+                    upTo);
+        }
+
+        LOG.info(
+                "Task {} schedules the next materialization in {} seconds.",
+                subtaskName,
+                periodicMaterializeDelay / 1000);
+
+        scheduleNextMaterialization();
+    }
+
+    private void handleExecutionException(
+            RunnableFuture<SnapshotResult<KeyedStateHandle>> 
materializedRunnableFuture) {

Review comment:
       There is already error handling logic calling this method. How about 
renaming this method to `discardFailedUpload` or something like this ?




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to