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

Anton Kalashnikov commented on FLINK-23466:
-------------------------------------------

Unfortunately, I failed with reproducing this problem locally, but here is some 
information from cluster log: 
One of subtasks can not by restored by some reason:

{noformat}

03:28:52,885 [ Checkpoint Timer] INFO 
org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Failed to 
trigger checkpoint for job 776bcb49e2efd18ee411248fa66ed39d since Checkpoint 
triggering task 
keyed (15/20) of job 776bcb49e2efd18ee411248fa66ed39d is not being executed at 
the moment. Aborting checkpoint. Failure reason: Not all required tasks are 
currently running.

{noformat}

{noformat}
"keyed (15/20)#4" #20483 prio=5 os_prio=0 tid=0x00007fbe200f1000 nid=0x383c 
waiting on condition [0x00007fbc416d4000] 
 java.lang.Thread.State: TIMED_WAITING (parking) 
 at sun.misc.Unsafe.park(Native Method) 
 - parking to wait for <0x0000000094200658> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) 
 at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) 
 at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2163)
 
 at 
org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl.take(TaskMailboxImpl.java:149)
 
 at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMailsWhenDefaultActionUnavailable(MailboxProcessor.java:335)
 
 at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:324)
 
 at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:201)
 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.executeRestore(StreamTask.java:669)
 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$1116/826634555.run(Unknown
 Source) 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:785)
 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:638)
 
 at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:766) 
 at org.apache.flink.runtime.taskmanager.Task.run(Task.java:572) 
 at java.lang.Thread.run(Thread.java:748)
{noformat}

In one log I see that `restore` is stuck on requestBufferBlocking which may 
indicate taht we have problem with buffers availability(the only test with 
buffersPerChannel=0 fails) but unfortunatelly, I don't see the same log in the 
second failed build which may mean that we have different problem or more than 
one problem(or threaddump was taken in wrong moment):

{noformat}
"channel-state-unspilling-thread-1" #17216 daemon prio=5 os_prio=0 
tid=0x00007f6660005000 nid=0x739d in Object.wait() [0x00007f63ff1f0000] 
 java.lang.Thread.State: WAITING (on object monitor) 
 at java.lang.Object.wait(Native Method) 
 at java.lang.Object.wait(Object.java:502) 
 at 
org.apache.flink.runtime.io.network.partition.consumer.BufferManager.requestBufferBlocking(BufferManager.java:117)
 
 - locked <0x00000000954d5fa8> (a 
org.apache.flink.runtime.io.network.partition.consumer.BufferManager$AvailableBufferQueue)
 
 at 
org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel.requestBufferBlocking(RecoveredInputChannel.java:268)
 
 at 
org.apache.flink.runtime.checkpoint.channel.InputChannelRecoveredStateHandler.getBuffer(RecoveredChannelStateHandler.java:90)
 
 at 
org.apache.flink.runtime.checkpoint.channel.InputChannelRecoveredStateHandler.getBuffer(RecoveredChannelStateHandler.java:69)
 
 at 
org.apache.flink.runtime.checkpoint.channel.ChannelStateChunkReader.readChunk(SequentialChannelStateReaderImpl.java:198)
 
 at 
org.apache.flink.runtime.checkpoint.channel.SequentialChannelStateReaderImpl.readSequentially(SequentialChannelStateReaderImpl.java:107)
 
 at 
org.apache.flink.runtime.checkpoint.channel.SequentialChannelStateReaderImpl.read(SequentialChannelStateReaderImpl.java:93)
 
 at 
org.apache.flink.runtime.checkpoint.channel.SequentialChannelStateReaderImpl.readInputData(SequentialChannelStateReaderImpl.java:64)
 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$restoreGates$0(StreamTask.java:609)
 
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$1081/1369458294.run(Unknown
 Source) 
 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)
{noformat}

Again, I see that `buffersPerChannel=0` and I see(at least in one build) the 
problem with buffer avilability. So maybe [~kevin.cyj] have any ideas in top of 
your head why it happens? In fact, I still don't sure that it is the true 
reason. 
The second reason can be the problem with compliting StateConsumedFuture after 
the recovery but there is no any clue that point to this.

> UnalignedCheckpointITCase hangs on Azure
> ----------------------------------------
>
>                 Key: FLINK-23466
>                 URL: https://issues.apache.org/jira/browse/FLINK-23466
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.14.0
>            Reporter: Dawid Wysakowicz
>            Priority: Major
>              Labels: test-stability
>             Fix For: 1.14.0
>
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=20813&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=2ef0effc-1da1-50e5-c2bd-aab434b1c5b7&l=16016



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

Reply via email to