[ https://issues.apache.org/jira/browse/FLINK-15105?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16991179#comment-16991179 ]
Congxian Qiu(klion26) commented on FLINK-15105: ----------------------------------------------- The test complete checkpoint successfully in the first job, and resumed from the checkpoint successfully in the second job, and can complete checkpoint in the seconde job successfully, {code:java} // log for first job 2019-12-05 20:12:17,410 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph - SlidingWindowOperator (1/2) (5a5c73dd041a0145bc02dc017e46bf1f) switched from DE PLOYING to RUNNING. 2019-12-05 20:12:17,970 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1 @ 1575576737956 for job 39a292088648857cac5f7e110547c18 0. 2019-12-05 20:12:21,095 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1 for job 39a292088648857cac5f7e110547c180 (261564 bytes in 3114 ms). 2019-12-05 20:12:21,113 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 2 @ 1575576741094 for job 39a292088648857cac5f7e110547c180. 2019-12-05 20:12:22,002 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph - FailureMapper (1/1) (4d273da136346ef3ff6e1a54d197f00b) switched from RUNNING to FAILED. java.lang.RuntimeException: Error while confirming checkpoint at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1205) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.Exception: Artificial failure. at org.apache.flink.streaming.tests.FailureMapper.notifyCheckpointComplete(FailureMapper.java:70) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:130) at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:822) at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1200) ... 5 more 2019-12-05 20:12:22,014 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Discarding checkpoint 2 of job 39a292088648857cac5f7e110547c180. java.lang.RuntimeException: Error while confirming checkpoint at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1205) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.Exception: Artificial failure. at org.apache.flink.streaming.tests.FailureMapper.notifyCheckpointComplete(FailureMapper.java:70) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.notifyCheckpointComplete(AbstractUdfStreamOperator.java:130) at org.apache.flink.streaming.runtime.tasks.StreamTask.notifyCheckpointComplete(StreamTask.java:822) at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1200) ... 5 more // log for second job 2019-12-05 20:12:27,190 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Starting job 7c862506012fb04c0d565bfda7cc9595 from savepoint file:///home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-02075534631/externalized-chckpt-e2e-backend-dir/39a292088648857cac5f7e110547c180/chk-1 () 2019-12-05 20:12:27,213 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Reset the checkpoint ID of job 7c862506012fb04c0d565bfda7cc9595 to 2. 2019-12-05 20:12:27,220 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Restoring job 7c862506012fb04c0d565bfda7cc9595 from latest valid checkpoint: Checkpoint 1 @ 0 for 7c862506012fb04c0d565bfda7cc9595. 2019-12-05 20:12:27,231 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - No master state to restore 2019-12-05 20:12:27,232 INFO org.apache.flink.runtime.jobmaster.JobManagerRunner - JobManager runner for job General purpose test job (7c862506012fb04c0d565bfda7cc9595) was granted leadership with session id 00000000-0000-0000-0000-000000000000 at akka.tcp://flink@localhost:6123/user/jobmanager_1. 2019-12-05 20:12:27,233 INFO org.apache.flink.runtime.jobmaster.JobMaster - Starting execution of job General purpose test job (7c862506012fb04c0d565bfda7cc9595) under job master id 00000000000000000000000000000000. 2019-12-05 20:12:27,233 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph - Job General purpose test job (7c862506012fb04c0d565bfda7cc9595) switched from state CREATED to RUNNING. 2019-12-05 20:12:29,668 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 2 for job 7c862506012fb04c0d565bfda7cc9595 (272831 bytes in 1389 ms). 2019-12-05 20:12:29,683 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 3 @ 1575576749668 for job 7c862506012fb04c0d565bfda7cc9595. {code} it failed because the log contains some error {code:java} Found error in log files:^M {code} if we execute the command(copied from common.sh) {code:java} grep -rv "GroupCoordinatorNotAvailableException" log.txt| grep -v "RetriableCommitFailedException" | grep -v "NoAvailableBrokersException" | grep -v "Async Kafka commit failed" | grep -v "DisconnectException" | grep -v "AskTimeoutException" | grep -v "Error while loading kafka-version.properties" | grep -v "WARN akka.remote.transport.netty.NettyTransport" | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" | grep -v "jvm-exit-on-fatal-error" | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' | grep -v "RejectedExecutionException" | grep -v "An exception was thrown by an exception handler" | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/exceptions/YarnException" | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" | grep -v "org.apache.flink.fs.shaded.hadoop3.org.apache.commons.beanutils.FluentPropertyBeanIntrospector - Error when creating PropertyDescriptor for public final void org.apache.flink.fs.shaded.hadoop3.org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)! Ignoring this property." | grep -v "Error while loading kafka-version.properties :null" | grep -v "Failed Elasticsearch item request" | grep -v "[Terror] modules" | grep -i "error" {code} then we'll get the following output {code:java} log.txt:Checking for errors... log.txt:Found error in log files: log.txt:java.lang.RuntimeException: Error while confirming checkpoint log.txt:java.lang.RuntimeException: Error while confirming checkpoint log.txt:java.lang.RuntimeException: Error while confirming checkpoint log.txt:java.lang.RuntimeException: Error while confirming checkpoint log.txt:java.lang.RuntimeException: Error while confirming checkpoint log.txt:java.lang.RuntimeException: Error while confirming checkpoint {code} All the above {{java.lang.RuntimeException: Error while confirming checkpoint}} was caused by Artificial failure. The Artificial failure was throw when completing checkpoint in [FailureMapper|https://github.com/apache/flink/blob/171020749f7fccfa7781563569e2c88ea5e8b6a1/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/FailureMapper.java#L66], and seems we have an infinite source from [SequenceGeneratorSource|https://github.com/apache/flink/blob/171020749f7fccfa7781563569e2c88ea5e8b6a1/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java#L102], I'm not sure why we need to throw Artificial failure when completing checkpoint, maybe [~trohrmann] know more about this, as he introduced the {{FailureMapper}}. If we do not rely on throwing Exception when completing checkpoint, I think we can remove the logic throwing Artifical failure when completing checkpoint. and if this is the right direction, I can help to fix it. > Resuming Externalized Checkpoint after terminal failure (rocks, incremental) > end-to-end test stalls on travis > ------------------------------------------------------------------------------------------------------------- > > Key: FLINK-15105 > URL: https://issues.apache.org/jira/browse/FLINK-15105 > Project: Flink > Issue Type: Bug > Components: Runtime / Checkpointing > Affects Versions: 1.10.0 > Reporter: Yu Li > Priority: Critical > Labels: test-stability > Fix For: 1.10.0 > > > Resuming Externalized Checkpoint after terminal failure (rocks, incremental) > end-to-end test fails on release-1.9 nightly build stalls with "The job > exceeded the maximum log length, and has been terminated". > https://api.travis-ci.org/v3/job/621090394/log.txt -- This message was sent by Atlassian Jira (v8.3.4#803005)