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 10:15:21 UTC

[GitHub] [flink] Myasuka commented on a change in pull request #14662: [FLINK-20675][checkpointing] Ensure asynchronous checkpoint failure could fail the job by default

Myasuka commented on a change in pull request #14662:
URL: https://github.com/apache/flink/pull/14662#discussion_r561711105



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/SerializedCheckpointException.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.messages.checkpoint;
+
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.util.SerializedThrowable;
+
+import java.io.Serializable;
+
+/**
+ * Serialized checkpoint exception which wraps the checkpoint failure reason and its serialized
+ * throwable.
+ */
+public class SerializedCheckpointException implements Serializable {

Review comment:
       This is because we need to use reflect to unwrap `CheckpointException` from `SerializedThrowable` when `checkpoint coordinator` received the decline checkpoint message, which is not easy to use. 

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/SerializedCheckpointException.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.messages.checkpoint;
+
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.util.SerializedThrowable;
+
+import java.io.Serializable;
+
+/**
+ * Serialized checkpoint exception which wraps the checkpoint failure reason and its serialized
+ * throwable.
+ */
+public class SerializedCheckpointException implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final CheckpointFailureReason checkpointFailureReason;
+    private final SerializedThrowable serializedThrowable;

Review comment:
       `serializedThrowable` is used to when calling `unwrap` to rebuild the `CheckpointException`. If we remove this field, we cannot see the actual error message when receiving the decline checkpoint.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java
##########
@@ -237,19 +241,29 @@ private void handleExecutionException(Exception e) {
                                         + '.',
                                 e);
 
-                // We only report the exception for the original cause of fail and cleanup.
-                // Otherwise this followup exception could race the original exception in failing
-                // the task.
-                try {
-                    taskEnvironment.declineCheckpoint(
+                if (isTaskRunning.get()) {
+                    // We only report the exception for the original cause of fail and cleanup.
+                    // Otherwise this followup exception could race the original exception in
+                    // failing the task.
+                    try {
+                        taskEnvironment.declineCheckpoint(
+                                checkpointMetaData.getCheckpointId(),
+                                new CheckpointException(
+                                        CheckpointFailureReason.CHECKPOINT_ASYNC_EXCEPTION,
+                                        checkpointException));
+                    } catch (Exception unhandled) {
+                        AsynchronousException asyncException = new AsynchronousException(unhandled);
+                        asyncExceptionHandler.handleAsyncException(
+                                "Failure in asynchronous checkpoint materialization",
+                                asyncException);
+                    }
+                } else {
+                    // We never decline checkpoint after task is not running to avoid unexpected job
+                    // failover, which caused by exceeding checkpoint tolerable failure threshold.
+                    LOG.warn(

Review comment:
       We changed the logic to fail job from task side to JM side with responsibility of `CheckpointFailureManager` at Flink-1.9. 
   Previously (take Flink-1.8 for example), if the asynchronous part of checkpoint failed on task side, it would call `FailingCheckpointExceptionHandler` to `tryHandleCheckpointException` which just [rethrow the exception](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java#L56), and then `AsyncCheckpointExceptionHandler` [handle the exception](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L1156) with [code below](https://github.com/apache/flink/blob/0e8e8062bcc159e9ed2a0d4a0a61db4efcb01f2f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L791):
   
   ```java
   	public void handleAsyncException(String message, Throwable exception) {
   		if (isRunning) {
   			// only fail if the task is still running
   			getEnvironment().failExternally(exception);
   		}
   	}
   ```
   As you can see, if task is not running, it would not fail externally again.
   
   Thus, after JM decided whether to fail the job, task should not send decline message to JM if it is not running to avoid unexpected failover from `CheckpointFailureManager`.
   If such task is not running, there should existed two cases:
   1. Task is finished. This cannot be stopping with savepoint as we would stop checkpoint scheduler first to avoid newer checkpoint after final savepoint. And this could only be the case that task reach to finished stage while async phase of period checkpoint is still running. In this case, the job will come to finished status thus we should not failover the job anymore by decline. Or some tasks are not running, that checkpoint would finally timeout just like previous behavior. Since FLIP-147 is accepted, I think we could think about what would happen in this case after FLIP-147. cc @gaoyunhaii 
   2. Task is canceled. This should be triggered by existing failover, and we should not decline one more time.
   
   
   If the task has already failed, the job would 




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