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

Arvid Heise commented on FLINK-22003:
-------------------------------------

This one is really strange:

Usually, when we trigger a checkpoint, we see some actions on the source
{noformat}
22:24:51,290 [    Checkpoint Timer] INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 1 (type=CHECKPOINT) @ 1616883891289 for job 
bb1c23aad807944d0a54775098106574.
22:24:51,290 [SourceCoordinator-Source: source] INFO  
org.apache.flink.test.checkpointing.UnalignedCheckpointTestBase [] - 
snapshotState EnumeratorState{unassignedSplits=[], numRestarts=0, 
numCompletedCheckpoints=0}
22:24:51,291 [Flink Netty Server (0) Thread 0] TRACE 
org.apache.flink.runtime.io.network.logger.NetworkActionsLogger [] - [Source: 
source (2/5)#0 (8f1ca6eb04b6e2341c658cc0b1ac7c6c)] 
PipelinedSubpartition#pollBuffer Buffer{size=38, hash=924008396} @ 
ResultSubpartitionInfo{partitionIdx=0, subPartitionIdx=0}
{noformat}

In this case, after checkpoint 11 is triggered nothing happens. 

{noformat}
22:24:54,694 [    Checkpoint Timer] INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 11 (type=CHECKPOINT) @ 1616883893885 for job 
bb1c23aad807944d0a54775098106574.
22:24:54,694 [ failing-map (5/5)#4] INFO  
org.apache.flink.runtime.taskmanager.Task                    [] - failing-map 
(5/5)#4 (7c0e288b2cd57831596d58e8ce31e435) switched from CREATED to DEPLOYING.
{noformat}

Actually, it should have been canceled, as obviously not all tasks are running 
similar to

{noformat}
22:24:51,044 [    Checkpoint Timer] WARN  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Failed to 
trigger checkpoint for job bb1c23aad807944d0a54775098106574.)
org.apache.flink.runtime.checkpoint.CheckpointException: Checkpoint triggering 
task Source: source (1/5) of job bb1c23aad807944d0a54775098106574 has not being 
executed at the moment. Aborting checkpoint. Failure reason: Not all required 
tasks are currently running.
{noformat}

I'm currently assuming that there is a race condition in the code of 
FLINK-21067.

> UnalignedCheckpointITCase fail
> ------------------------------
>
>                 Key: FLINK-22003
>                 URL: https://issues.apache.org/jira/browse/FLINK-22003
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.13.0
>            Reporter: Guowei Ma
>            Priority: Major
>              Labels: test-stability
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=15601&view=logs&j=119bbba7-f5e3-5e08-e72d-09f1529665de&t=7dc1f5a9-54e1-502e-8b02-c7df69073cfc&l=4142
> {code:java}
> [ERROR] execute[parallel pipeline with remote channels, p = 
> 5](org.apache.flink.test.checkpointing.UnalignedCheckpointITCase)  Time 
> elapsed: 60.018 s  <<< ERROR!
> org.junit.runners.model.TestTimedOutException: test timed out after 60000 
> milliseconds
>       at sun.misc.Unsafe.park(Native Method)
>       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>       at 
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
>       at 
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
>       at 
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
>       at 
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1859)
>       at 
> org.apache.flink.streaming.api.environment.LocalStreamEnvironment.execute(LocalStreamEnvironment.java:69)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1839)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1822)
>       at 
> org.apache.flink.test.checkpointing.UnalignedCheckpointTestBase.execute(UnalignedCheckpointTestBase.java:138)
>       at 
> org.apache.flink.test.checkpointing.UnalignedCheckpointITCase.execute(UnalignedCheckpointITCase.java:184)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
>       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at java.lang.Thread.run(Thread.java:748)
> {code}



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

Reply via email to