You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/09 07:31:00 UTC

[jira] [Commented] (FLINK-10074) Allowable number of checkpoint failures

    [ https://issues.apache.org/jira/browse/FLINK-10074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16642907#comment-16642907 ] 

ASF GitHub Bot commented on FLINK-10074:
----------------------------------------

yanghua commented on a change in pull request #6567: [FLINK-10074] Allowable number of checkpoint failures
URL: https://github.com/apache/flink/pull/6567#discussion_r223586404
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * 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.checkpoint;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * The checkpoint failure manager to manage how to process checkpoint failure.
+ */
+public class CheckpointFailureManager {
+
+	private final boolean failOnCheckpointingErrors;
+	private final int tolerableCpFailureNumber;
+	private final AtomicInteger continuousFailureCounter;
+	private final ExecutionGraph executionGraph;
+	private final Object lock = new Object();
+
+	public CheckpointFailureManager(
+		boolean failOnCheckpointingErrors,
+		int tolerableCpFailureNumber,
+		ExecutionGraph executionGraph) {
+		this.failOnCheckpointingErrors = failOnCheckpointingErrors;
+		this.tolerableCpFailureNumber = tolerableCpFailureNumber;
+		this.continuousFailureCounter = new AtomicInteger(0);
+		this.executionGraph = checkNotNull(executionGraph);
+	}
+
+	@VisibleForTesting
+	public AtomicInteger getContinuousFailureCounter() {
+		return continuousFailureCounter;
+	}
+
+	public void resetCounter() {
+		continuousFailureCounter.set(0);
+	}
+
+	public void tryHandleFailure(String reason, long checkpointId) {
+		synchronized (lock) {
+			if (failOnCheckpointingErrors ||
+				continuousFailureCounter.incrementAndGet() > tolerableCpFailureNumber) {
+				executionGraph.failGlobal(new Throwable(reason));
 
 Review comment:
   hi @azagrebin and @tillrohrmann , Can you take a general look at my PR implementation and comment on how to trigger a job failure when the failure condition is met? My current implementation seems to be a bit problematic (cyclic dependency), which caused a lot of test failures.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> Allowable number of checkpoint failures 
> ----------------------------------------
>
>                 Key: FLINK-10074
>                 URL: https://issues.apache.org/jira/browse/FLINK-10074
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>            Reporter: Thomas Weise
>            Assignee: vinoyang
>            Priority: Major
>              Labels: pull-request-available
>
> For intermittent checkpoint failures it is desirable to have a mechanism to avoid restarts. If, for example, a transient S3 error prevents checkpoint completion, the next checkpoint may very well succeed. The user may wish to not incur the expense of restart under such scenario and this could be expressed with a failure threshold (number of subsequent checkpoint failures), possibly combined with a list of exceptions to tolerate.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)