curcur commented on a change in pull request #16606:
URL: https://github.com/apache/flink/pull/16606#discussion_r717191787
##########
File path: pom.xml
##########
@@ -1518,7 +1518,7 @@ under the License.
random: enable it
randomly, unless explicitly set
unset: don't alter the
configuration
-->
-
<checkpointing.changelog>random</checkpointing.changelog>
+
<checkpointing.changelog>on</checkpointing.changelog>
Review comment:
yes, it will be changed to random before merging. It is used purely for
testing
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/test/resources/log4j2-test.properties
##########
@@ -18,11 +18,11 @@
# Set root logger level to OFF to not flood build logs
# set manually to INFO for debugging purposes
-rootLogger.level = OFF
+rootLogger.level = INFO
Review comment:
yes, we could. The hotfix is to fix the name. The previous one has the
wrong name and does not work for controlling the logging level.
I will change the logging level back.
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/test/resources/log4j2-test.properties
##########
@@ -18,11 +18,11 @@
# Set root logger level to OFF to not flood build logs
# set manually to INFO for debugging purposes
-rootLogger.level = OFF
+rootLogger.level = INFO
Review comment:
yes, we could. The hotfix is to fix the log4j2 file name. The previous
one has the wrong name and does not work for controlling the logging level.
I will change the logging level back.
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/test/resources/log4j2-test.properties
##########
@@ -18,11 +18,11 @@
# Set root logger level to OFF to not flood build logs
# set manually to INFO for debugging purposes
-rootLogger.level = OFF
+rootLogger.level = INFO
Review comment:
yes, we could. The hotfix is to fix the log4j2 file name. The previous
one has the wrong name and does not control the logging level.
I will change the logging level back.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
##########
@@ -258,4 +259,10 @@ default void setAsyncOperationsThreadPool(ExecutorService
executorService) {}
default ExecutorService getAsyncOperationsThreadPool() {
throw new UnsupportedOperationException();
}
+
+ default void setCheckpointStorageAccess(CheckpointStorageAccess
checkpointStorageAccess) {}
+
+ default CheckpointStorageAccess getCheckpointStorageAccess() {
+ throw new UnsupportedOperationException();
+ }
Review comment:
> Thanks updating the PR @curcur.
>
> I've left some comments, PTAL.
>
> Besides that, could you please name the extracted commits according to
[conventions](https://flink.apache.org/contributing/code-style-and-quality-pull-requests.html#commit-naming-conventions)
(add `[hotfix]` and component label, fix case, etc.).
which of the commits do you think not following the convention?
If you mean the [hotfix] one, It is to fix a file naming issue for log4j2
file. I am not sure whether it is belonging to a "component"
##########
File path:
flink-state-backends/flink-statebackend-changelog/src/test/resources/log4j2-test.properties
##########
@@ -18,11 +18,11 @@
# Set root logger level to OFF to not flood build logs
# set manually to INFO for debugging purposes
-rootLogger.level = OFF
+rootLogger.level = INFO
Review comment:
BTW, what's the problem for `layout.pattern`? Do you mean we should
remove it?
##########
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();
Review comment:
If the retry limit is exceeded, the task will be failed and trigger
failover
Do you mean it is possible that before failover is triggered, another
materialization is triggered?
##########
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();
Review comment:
What's the problem if `scheduleNextMaterialization` is called?
##########
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();
Review comment:
If the retry limit is exceeded, the task will be failed and trigger
failover
Do you mean it is possible that before failover is triggered, another
materialization is triggered?
And what's the problem if scheduleNextMaterialization is called?
--
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]