[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16712643#comment-16712643 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann commented on issue #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#issuecomment-445194513 Manually merged. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Assignee: Andrey Zagrebin >Priority: Major > Labels: pull-request-available > Fix For: 1.5.6, 1.6.3, 1.8.0, 1.7.1 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > > > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16712644#comment-16712644 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann closed pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java index dad45eb669c..9e15aebd048 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.checkpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkArgument; @@ -26,6 +29,7 @@ * Counts of checkpoints. */ public class CheckpointStatsCounts implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(CheckpointStatsCounts.class); private static final long serialVersionUID = -5229425063269482528L; @@ -147,9 +151,8 @@ void incrementInProgressCheckpoints() { * {@link #incrementInProgressCheckpoints()}. */ void incrementCompletedCheckpoints() { - if (--numInProgressCheckpoints < 0) { - throw new IllegalStateException("Incremented the completed number of checkpoints " + - "without incrementing the in progress checkpoints before."); + if (canDecrementOfInProgressCheckpointsNumber()) { + numInProgressCheckpoints--; } numCompletedCheckpoints++; } @@ -161,9 +164,8 @@ void incrementCompletedCheckpoints() { * {@link #incrementInProgressCheckpoints()}. */ void incrementFailedCheckpoints() { - if (--numInProgressCheckpoints < 0) { - throw new IllegalStateException("Incremented the completed number of checkpoints " + - "without incrementing the in progress checkpoints before."); + if (canDecrementOfInProgressCheckpointsNumber()) { + numInProgressCheckpoints--; } numFailedCheckpoints++; } @@ -181,4 +183,14 @@ CheckpointStatsCounts createSnapshot() { numCompletedCheckpoints, numFailedCheckpoints); } + + private boolean canDecrementOfInProgressCheckpointsNumber() { + boolean decrementLeadsToNegativeNumber = numInProgressCheckpoints - 1 < 0; + if (decrementLeadsToNegativeNumber) { + String errorMessage = "Incremented the completed number of checkpoints " + + "without incrementing the in progress checkpoints before."; + LOG.warn(errorMessage); + } + return !decrementLeadsToNegativeNumber; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 5d2d363cf71..54c14af8983 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -962,19 +962,18 @@ public void heartbeatFromResourceManager(final ResourceID resourceID) { return checkpointCoordinator .triggerSavepoint(System.currentTimeMillis(), targetDirectory) .thenApply(CompletedCheckpoint::getExternalPointer) - .thenApplyAsync(path -> { - if (cancelJob) { + .handleAsync((path, throwable) -> { + if (throwable != null) { + if (cancelJob) { + startCheckpointScheduler(checkpointCoordinator); + } + throw new CompletionException(throwable); + } else if (cancelJob) { log.info("Savepoint stored in {}. Now cancelling {}.", path, jobGraph.getJobID()); cancel(timeout);
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16710357#comment-16710357 ] ASF GitHub Bot commented on FLINK-10482: azagrebin commented on issue #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#issuecomment-444564761 Thank you for the review @tillrohrmann ! I addressed the comments. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > > > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16710221#comment-16710221 ] ASF GitHub Bot commented on FLINK-10482: azagrebin commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r239108282 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ## @@ -1195,11 +1196,16 @@ public void stopCheckpointScheduler() { currentPeriodicTrigger = null; } - for (PendingCheckpoint p : pendingCheckpoints.values()) { + // take a snapshot of pendingCheckpoints to clear them and prevent aborting them twice + // in case of subsequent call of stopCheckpointScheduler() + List pendingCheckpointsSnapshot = + new ArrayList<>(pendingCheckpoints.values()); + pendingCheckpoints.clear(); + + for (PendingCheckpoint p : pendingCheckpointsSnapshot) { Review comment: Agree, this fix will solve the original problem without other changes. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16708906#comment-16708906 ] ASF GitHub Bot commented on FLINK-10482: azagrebin commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r238720755 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java ## @@ -181,4 +183,15 @@ CheckpointStatsCounts createSnapshot() { numCompletedCheckpoints, numFailedCheckpoints); } + + private boolean assertDecrementOfInProgressCheckpointsNumber() { + boolean decrementLeadsToNegativeNumber = numInProgressCheckpoints - 1 < 0; + if (decrementLeadsToNegativeNumber) { + String errorMessage = "Incremented the completed number of checkpoints " + + "without incrementing the in progress checkpoints before."; + LOG.warn(errorMessage); + LOG.debug("Inconsistent CheckpointStatsCounts", new IllegalStateException(errorMessage)); Review comment: thanks, debugging leftover 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16708893#comment-16708893 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r238705456 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java ## @@ -181,4 +183,15 @@ CheckpointStatsCounts createSnapshot() { numCompletedCheckpoints, numFailedCheckpoints); } + + private boolean assertDecrementOfInProgressCheckpointsNumber() { Review comment: maybe rename method into `canDecrementOfInProgressCheckpointsNumber`. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16708892#comment-16708892 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r238715678 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ## @@ -1195,11 +1196,16 @@ public void stopCheckpointScheduler() { currentPeriodicTrigger = null; } - for (PendingCheckpoint p : pendingCheckpoints.values()) { + // take a snapshot of pendingCheckpoints to clear them and prevent aborting them twice + // in case of subsequent call of stopCheckpointScheduler() + List pendingCheckpointsSnapshot = + new ArrayList<>(pendingCheckpoints.values()); + pendingCheckpoints.clear(); + + for (PendingCheckpoint p : pendingCheckpointsSnapshot) { Review comment: Just as a note, I guess we could have solved the problem also by changing `JobMaster.java:965-975` into a `handleAsync` part because the problem seems to be that by aborting a `PendingCheckpoint` we fail the savepoint future which is then directly triggers the execution of the `exceptionally` handler. By decoupling the `exceptionally` handler from the `stopCheckpointScheduler` call, there should not be any concurrency problem. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16708891#comment-16708891 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r238709753 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java ## @@ -970,7 +970,7 @@ public void heartbeatFromResourceManager(final ResourceID resourceID) { return path; }, getMainThreadExecutor()) .exceptionally(throwable -> { - if (cancelJob) { + if (cancelJob && executionGraph.getState() == JobStatus.RUNNING) { Review comment: What situation are we filtering out with this change? 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16708889#comment-16708889 ] ASF GitHub Bot commented on FLINK-10482: tillrohrmann commented on a change in pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118#discussion_r238705914 ## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java ## @@ -181,4 +183,15 @@ CheckpointStatsCounts createSnapshot() { numCompletedCheckpoints, numFailedCheckpoints); } + + private boolean assertDecrementOfInProgressCheckpointsNumber() { + boolean decrementLeadsToNegativeNumber = numInProgressCheckpoints - 1 < 0; + if (decrementLeadsToNegativeNumber) { + String errorMessage = "Incremented the completed number of checkpoints " + + "without incrementing the in progress checkpoints before."; + LOG.warn(errorMessage); + LOG.debug("Inconsistent CheckpointStatsCounts", new IllegalStateException(errorMessage)); Review comment: Let's remove this logging statement. 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16688527#comment-16688527 ] ASF GitHub Bot commented on FLINK-10482: azagrebin opened a new pull request #7118: [FLINK-10482] Fix double counting of checkpoint stat URL: https://github.com/apache/flink/pull/7118 ## What is the purpose of the change This PR fixes double counting of checkpoints in progress in their statistics. ## Brief change log - If savepoint fails, restart checkpoint coordinator only if job is still running (prevents from double stop of checkpoint coordinator in case of global failure) - clear pending checkpoint in checkpoint coordinator stop method before aborting them (prevents from double aborting them if stop is called inside abort) - log error if number of checkpoints in progress is negative but do not throw exception and do not fail the job (prevents stats bugs from failing the job) ## Verifying this change submit DataStreamAllroundTestProgram in a loop in aws emr, wait for failure, check there was no negative number of checkpoints logs ## Does this pull request potentially affect one of the following parts: - Dependencies (does it add or upgrade a dependency): (no) - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no) - The serializers: (no) - The runtime per-record code paths (performance sensitive): (no) - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes) - The S3 file system connector: (no) ## Documentation - Does this pull request introduce a new feature? (no) - If yes, how is the feature documented? (not applicable) 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: us...@infra.apache.org > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Labels: pull-request-available > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16681680#comment-16681680 ] Andrey Zagrebin commented on FLINK-10482: - [~JBiason] do you still have the full log? can you post it here? do you see there something like this: _Incremented the completed number of checkpoints without incrementing the in progress checkpoints before._ ? > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > > > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > > > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > > > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > > at >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16656892#comment-16656892 ] Gary Yao commented on FLINK-10482: -- While working on FLINK-10309, I discovered FLINK-10615, which can also cause the issue described here. If I recall correctly, the reason is that a pending checkpoint may be [aborted multiple times|https://github.com/apache/flink/blob/d2480af40f5145d865196a95ec58a415482d8cff/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java#L512], which messes up the statistics. > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > Fix For: 1.8.0 > > > Recently I found the following log on my JobManager log: > {noformat} > 2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress > checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > > > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > > > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) >
[jira] [Commented] (FLINK-10482) java.lang.IllegalArgumentException: Negative number of in progress checkpoints
[ https://issues.apache.org/jira/browse/FLINK-10482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16637875#comment-16637875 ] Till Rohrmann commented on FLINK-10482: --- [~GJL] could this one be related to your recent findings? > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > -- > > Key: FLINK-10482 > URL: https://issues.apache.org/jira/browse/FLINK-10482 > Project: Flink > Issue Type: Bug >Affects Versions: 1.6.1 >Reporter: Julio Biason >Priority: Major > > Recently I found the following log on my JobManager log: > ```2018-10-02 17:44:50,090 [flink-akka.actor.default-dispatcher-4117] ERROR > org.apache.flink.runtime.rest.handler.job.JobDetailsHandler - Implementation > error: Unhandled exception. > java.lang.IllegalArgumentException: Negative number of in progress checkpoints > at > org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:139) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.(CheckpointStatsCounts.java:72) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsCounts.createSnapshot(CheckpointStatsCounts.java:177) > at > org.apache.flink.runtime.checkpoint.CheckpointStatsTracker.createSnapshot(CheckpointStatsTracker.java:166) > at > org.apache.flink.runtime.executiongraph.ExecutionGraph.getCheckpointStatsSnapshot(ExecutionGraph.java:553) > at > org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph.createFrom(ArchivedExecutionGraph.java:340) > at > org.apache.flink.runtime.jobmaster.JobMaster.requestJob(JobMaster.java:923) > at sun.reflect.GeneratedMethodAccessor101.invoke(Unknown Source) > > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:247) > > > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > > > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > > > at > akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165) > > > at akka.actor.Actor$class.aroundReceive(Actor.scala:502) > > > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > > > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > > > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > > > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > > > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > > > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > > ``` > > Related: The job details don't appear, the screen shows only the skeleton, > but no information (like