Hi Roman,
I took thread dump:
"Source: digital-itx-eastus2 -> Filter (6/6)#0" Id=200 BLOCKED on 
java.lang.Object@5366a0e2 owned by "Legacy Source Thread - Source: 
digital-itx-eastus2 -> Filter (6/6)#0" Id=202
    at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:92)
    -  blocked on java.lang.Object@5366a0e2
    at org.apache.flink.streaming.runtime.tasks.mailbox.Mail.run(Mail.java:90)
    at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.processMail(MailboxProcessor.java:317)
    at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:189)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:617)
    at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:581)
    at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:755)
    at org.apache.flink.runtime.taskmanager.Task.run(Task.java:570)

"Legacy Source Thread - Source: digital-itx-eastus2 -> Filter (6/6)#0" Id=202 
WAITING on java.util.concurrent.CompletableFuture$Signaller@6915c7ef
    at sun.misc.Unsafe.park(Native Method)
    -  waiting on java.util.concurrent.CompletableFuture$Signaller@6915c7ef
    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.runtime.io.network.buffer.LocalBufferPool.requestMemorySegmentBlocking(LocalBufferPool.java:319)
    at 
org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBufferBuilderBlocking(LocalBufferPool.java:291)

Is it checkpoint lock? Is checkpoint lock per task or per TM? I see multiple 
threads in SynchronizedStreamTaskActionExecutor.runThrowing blocked on 
different Objects.

Thanks,
Alexey
________________________________
From: Roman Khachatryan <ro...@apache.org>
Sent: Monday, March 15, 2021 2:16 AM
To: Alexey Trenikhun <yen...@msn.com>
Cc: Flink User Mail List <user@flink.apache.org>
Subject: Re: Checkpoint fail due to timeout

Hello Alexey,

Thanks for the details.

It looks like backpressure is indeed the cause of the issue.
You can check that by looking at the (succeeded) checkpoint start
delay in the tasks following the suspected source
(digital-itx-eastus2?).
To be sure, you can take a thread dump (or profile) those sources: the
task thread should be waiting for checkpoint lock; while the legacy
source thread should be holding it and waiting to output data.

One way to deal with this is to use the new Kafka source (based on
FLIP-27) which will hopefully be released in 1.13 (it is an
experimental feature in 1.12).

Regards,
Roman

On Fri, Mar 12, 2021 at 8:43 PM Alexey Trenikhun <yen...@msn.com> wrote:
>
> Hello Roman,
>
>  history, details and summary stats are attached.
> There is backpressure on all sources except Source:gca-cfg and 
> Source:heartbeat
> Flink version 1.12.1, I also trying 1.12.2 with same results
>
> Thanks,
> Alexey
> ________________________________
> From: Roman Khachatryan <ro...@apache.org>
> Sent: Thursday, March 11, 2021 11:49 PM
> To: Alexey Trenikhun <yen...@msn.com>
> Cc: Flink User Mail List <user@flink.apache.org>
> Subject: Re: Checkpoint fail due to timeout
>
> Hello,
>
> This can be caused by several reasons such as back-pressure, large
> snapshots or bugs.
>
> Could you please share:
> - the stats of the previous (successful) checkpoints
> - back-pressure metrics for sources
> - which Flink version do you use?
>
> Regards,
> Roman
>
>
> On Thu, Mar 11, 2021 at 7:03 AM Alexey Trenikhun <yen...@msn.com> wrote:
> >
> > Hello,
> > We are experiencing the problem with checkpoints failing due to timeout 
> > (already set to 30 minute, still failing), checkpoints were not too big 
> > before they started to fail, around 1.2Gb. Looks like one of sources 
> > (Kafka) never acknowledged (see attached screenshot). What could be the 
> > reason?
> >
> > Thanks,
> > Alexey
> >
> >

Reply via email to