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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java
##########
@@ -93,9 +95,7 @@ private void handleCheckpointException(
             checkFailureCounter(exception, checkpointId);
             if (continuousFailureCounter.get() > tolerableCpFailureNumber) {
                 clearCount();
-                errorHandler.accept(
-                        new FlinkRuntimeException(
-                                "Exceeded checkpoint tolerable failure 
threshold."));
+                
errorHandler.accept(EXCEEDED_CHECKPOINT_TOLERABLE_FAILURE_EXCEPTION);

Review comment:
       I'm afraid we are losing the stacktrace here.

##########
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:
       Thanks!

##########
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:
       Got it, thanks!




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


Reply via email to