[jira] [Created] (FLINK-21241) E2E test exception check might fail if older checkpoint not completed when triggering the stop-with-savepoint

classic Classic list List threaded Threaded
1 message Options
Reply | Threaded
Open this post in threaded view
|

[jira] [Created] (FLINK-21241) E2E test exception check might fail if older checkpoint not completed when triggering the stop-with-savepoint

Shang Yuanchun (Jira)
Yun Tang created FLINK-21241:
--------------------------------

             Summary: E2E test exception check might fail if older checkpoint not completed when triggering the stop-with-savepoint
                 Key: FLINK-21241
                 URL: https://issues.apache.org/jira/browse/FLINK-21241
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Checkpointing, Tests
    Affects Versions: 1.12.1, 1.11.3, 1.13.0
            Reporter: Yun Tang
             Fix For: 1.11.4, 1.12.2, 1.13.0


[https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=12724&view=logs&j=c88eea3b-64a0-564d-0031-9fdcd7b8abee&t=ff888d9b-cd34-53cc-d90f-3e446d355529] E2E test fails due to find exceptions in logs.

This is because we trigger the {{stop-with-savepoint-13}} when checkpoint-12 was not completed.

{code:java}
2021-02-01 07:54:22,542 INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering checkpoint 12 (type=CHECKPOINT) @ 1612166062541 for job 603ede5f1dcc9858f8db6a787992c181.
2021-02-01 07:54:22,543 INFO  org.apache.flink.runtime.jobmaster.JobMaster                 [] - Triggering stop-with-savepoint for job 603ede5f1dcc9858f8db6a787992c181.
2021-02-01 07:54:22,599 INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering checkpoint 13 (type=SYNC_SAVEPOINT) @ 1612166062597 for job 603ede5f1dcc9858f8db6a787992c181.
{code}

As we will stop the checkpoint scheduler when triggering the {{stop-with-savepoint}}, the old checkpoint-12 would be aborted. And we could find exceptions in task managers:

{code:java}
2021-02-01 07:54:22,592 WARN  org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable [] - Could not properly clean up the async checkpoint runnable.
java.lang.IllegalStateException: null
        at org.apache.flink.util.Preconditions.checkState(Preconditions.java:177) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.util.Preconditions.checkCompletedNormally(Preconditions.java:261) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.concurrent.FutureUtils.checkStateAndGet(FutureUtils.java:1176) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.checkpoint.CheckpointMetricsBuilder.build(CheckpointMetricsBuilder.java:133) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.reportAbortedSnapshotStats(AsyncCheckpointRunnable.java:223) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.close(AsyncCheckpointRunnable.java:306) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:275) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.cancelAsyncCheckpointRunnable(SubtaskCheckpointCoordinatorImpl.java:451) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.notifyCheckpointAborted(SubtaskCheckpointCoordinatorImpl.java:340) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointAbortAsync$12(StreamTask.java:1070) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointOperation$13(StreamTask.java:1083) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsWhenDefaultActionUnavailable(MailboxProcessor.java:314) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:300) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:188) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:615) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:579) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:565) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_282]
2021-02-01 07:54:22,632 INFO  org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable [] - ArtificalKeyedStateMapper_Kryo_and_Custom_Stateful (3/4)#0 - asynchronous part of checkpoint 12 could not be completed.
java.util.concurrent.ExecutionException: java.util.concurrent.CancellationException: checkpoint aborted via notification
        at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) ~[?:1.8.0_282]
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908) ~[?:1.8.0_282]
        at org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:66) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:127) [flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_282]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_282]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_282]
Caused by: java.util.concurrent.CancellationException: checkpoint aborted via notification
        at org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.notifyCheckpointAborted(SubtaskCheckpointCoordinatorImpl.java:349) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointAbortAsync$12(StreamTask.java:1070) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$notifyCheckpointOperation$13(StreamTask.java:1083) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:50) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsNonBlocking(MailboxProcessor.java:329) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:293) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:188) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:615) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:579) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:565) ~[flink-dist_2.11-1.13-SNAPSHOT.jar:1.13-SNAPSHOT]
{code}

We should avoid such expected exceptions to break the E2E tests.




--
This message was sent by Atlassian Jira
(v8.3.4#803005)