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

Hangxiang Yu commented on FLINK-32523:
--------------------------------------

>From the exception stack and attached logs, I saw:
 # the failure reason is not same, timeout and assert error
 # in the timeout cases, the job failed then restored, In the assert error 
cases, the job aborted two times (these cases are all enabling unaligned 
checkpoint), the job never failed and ran until finished for every success cases

So I think there are two exceptions in this ITCase:
 # Assert error -> All operators haven't snapshotState together strictly for 
marked decline checkpoint id in the teste case (This could be reproduced by 
adding Thread.sleep after first verifyAllOperatorsNotifyAborted() )
 # Timeout exception -> restarting (due to 1 tolerable checkpoint failure 
number) and notifying aborted occur in different threads, and the order is 
uncertain, if the job restart firstly, this will cause timeout exception (This 
could be reproduced by adding Thread.sleep in NormalMap#notifyCheckpointAborted)


For the first exception, we could just make them snapshotState together 
strictly which I think the ITCase should guarantee.

For the second one, I think it's acceptable that the abort function may not be 
called if the job failover (notifyCheckpointAborted is a best effort function). 
So we could just increase the tolerable checkpoint number.

 



Why timeout exception just occured in 1.18 ?

This is because FLINK-32347 which fixes the exception that 
CompletedCheckpointStore are not registered by the CheckpointFailureManager, 
After this, the job could fail due to the tolerable checkpoint failure number.

> NotifyCheckpointAbortedITCase.testNotifyCheckpointAborted fails with timeout 
> on AZP
> -----------------------------------------------------------------------------------
>
>                 Key: FLINK-32523
>                 URL: https://issues.apache.org/jira/browse/FLINK-32523
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.16.2, 1.18.0
>            Reporter: Sergey Nuyanzin
>            Priority: Critical
>              Labels: test-stability
>         Attachments: failure.log
>
>
> This build
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=50795&view=logs&j=39d5b1d5-3b41-54dc-6458-1e2ddd1cdcf3&t=0c010d0c-3dec-5bf1-d408-7b18988b1b2b&l=8638
>  fails with timeout
> {noformat}
> Jul 03 01:26:35 org.junit.runners.model.TestTimedOutException: test timed out 
> after 100000 milliseconds
> Jul 03 01:26:35       at java.lang.Object.wait(Native Method)
> Jul 03 01:26:35       at java.lang.Object.wait(Object.java:502)
> Jul 03 01:26:35       at 
> org.apache.flink.core.testutils.OneShotLatch.await(OneShotLatch.java:61)
> Jul 03 01:26:35       at 
> org.apache.flink.test.checkpointing.NotifyCheckpointAbortedITCase.verifyAllOperatorsNotifyAborted(NotifyCheckpointAbortedITCase.java:198)
> Jul 03 01:26:35       at 
> org.apache.flink.test.checkpointing.NotifyCheckpointAbortedITCase.testNotifyCheckpointAborted(NotifyCheckpointAbortedITCase.java:189)
> Jul 03 01:26:35       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native 
> Method)
> Jul 03 01:26:35       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> Jul 03 01:26:35       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> Jul 03 01:26:35       at java.lang.reflect.Method.invoke(Method.java:498)
> Jul 03 01:26:35       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
> Jul 03 01:26:35       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
> Jul 03 01:26:35       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
> Jul 03 01:26:35       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
> Jul 03 01:26:35       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
> Jul 03 01:26:35       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
> Jul 03 01:26:35       at 
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> Jul 03 01:26:35       at java.lang.Thread.run(Thread.java:748)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to