You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Yu Li <ca...@gmail.com> on 2020/10/06 04:04:35 UTC

Re: flink checkpoint timeout

I'm not 100% sure but from the given information this might be related to
FLINK-14498 [1] and partially relieved by FLINK-16645 [2].

@Omkar Could you try the 1.11.0 release out and see whether the issue
disappeared?

@zhijiang <wa...@aliyun.com> @yingjie could you also take a look
here? Thanks.

Best Regards,
Yu

[1] https://issues.apache.org/jira/browse/FLINK-14498
[2] https://issues.apache.org/jira/browse/FLINK-16645


On Fri, 18 Sep 2020 at 09:28, Deshpande, Omkar <Om...@intuit.com>
wrote:

> These are the hostspot method. Any pointers on debugging this? The
> checkpoints keep timing out since migrating to 1.10 from 1.9
> ------------------------------
> *From:* Deshpande, Omkar <Om...@intuit.com>
> *Sent:* Wednesday, September 16, 2020 5:27 PM
> *To:* Congxian Qiu <qc...@gmail.com>
> *Cc:* user@flink.apache.org <us...@flink.apache.org>; Yun Tang <
> myasuka@live.com>
> *Subject:* Re: flink checkpoint timeout
>
> This email is from an external sender.
>
> This thread seems to stuck in awaiting notification state -
> 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:1693)
> at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> at
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
> at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
> at
> org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestMemorySegmentBlocking(LocalBufferPool.java:231)
>
> ------------------------------
> *From:* Congxian Qiu <qc...@gmail.com>
> *Sent:* Monday, September 14, 2020 10:57 PM
> *To:* Deshpande, Omkar <Om...@intuit.com>
> *Cc:* user@flink.apache.org <us...@flink.apache.org>
> *Subject:* Re: flink checkpoint timeout
>
> This email is from an external sender.
>
> Hi
>     You can try to find out is there is some hot method, or the snapshot
> stack is waiting for some lock. and maybe
> Best,
> Congxian
>
>
> Deshpande, Omkar <Om...@intuit.com> 于2020年9月15日周二 下午12:30写道:
>
> Few of the subtasks fail. I cannot upgrade to 1.11 yet. But I can still
> get the thread dump. What should I be looking for in the thread dump?
>
> ------------------------------
> *From:* Yun Tang <my...@live.com>
> *Sent:* Monday, September 14, 2020 8:52 PM
> *To:* Deshpande, Omkar <Om...@intuit.com>; user@flink.apache.org
> <us...@flink.apache.org>
> *Subject:* Re: flink checkpoint timeout
>
> This email is from an external sender.
>
> Hi Omkar
>
> First of all, you should check the web UI of checkpoint [1] to see whether
> many subtasks fail to complete in time or just few of them. The former one
> might be your checkpoint time out is not enough for current case. The later
> one might be some task stuck in slow machine or cannot grab checkpoint lock
> to process sync phase of checkpointing, you can use thread dump [2] (needs
> to bump to Flink-1.11) or jstack to see what happened in java process.
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/checkpoint_monitoring.html
> [2] https://issues.apache.org/jira/browse/FLINK-14816
>
> Best
> Yun Tang
> ------------------------------
> *From:* Deshpande, Omkar <Om...@intuit.com>
> *Sent:* Tuesday, September 15, 2020 10:25
> *To:* user@flink.apache.org <us...@flink.apache.org>
> *Subject:* Re: flink checkpoint timeout
>
> I have followed this
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_migration.html
> <https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_migration.html#container-cut-off-memory>
> and I am using taskmanager.memory.flink.size now instead of
> taskmanager.heap.size
> ------------------------------
> *From:* Deshpande, Omkar <Om...@intuit.com>
> *Sent:* Monday, September 14, 2020 6:23 PM
> *To:* user@flink.apache.org <us...@flink.apache.org>
> *Subject:* flink checkpoint timeout
>
> This email is from an external sender.
>
> Hello,
>
> I recently upgraded from flink 1.9 to 1.10. The checkpointing succeeds
> first couple of times and then starts failing because of timeouts. The
> checkpoint time grows with every checkpoint and starts exceeding 10
> minutes. I do not see any exceptions in the logs. I have enabled debug
> logging at "org.apache.flink" level. How do I investigate this? The garbage
> collection seems fine. There is no backpressure. This used to work as is
> with flink 1.9 without any issue.
>
> Any pointers on how to investigate long time taken to complete checkpoint?
>
> Omkar
>
>

Re: flink checkpoint timeout

Posted by Arvid Heise <ar...@ververica.com>.
Hi Omkar,

I don't see anything suspicious in regards to how Flink handles
checkpointing; it simply took longer than 10m (configured checkpointing
timeout) to checkpoint.

The usual reason for long checkpointing times is backpressure. And indeed
looking at your thread dump, I see that you have a sleep Fn in it. Can you
shed some light on this? Why do you need it? If you want to throttle
things, it's best to throttle at the source if possible. Alternatively,
have the sleep as early as possible, so that it's ideally directly chained
to the source. That would reduce the number of records in network buffers
significantly, which speeds up checkpointing tremendously. Lastly, you
might want to reduce the number of network buffers if you indeed have
backpressure (check Web UI for that).

On Tue, Oct 6, 2020 at 6:04 AM Yu Li <ca...@gmail.com> wrote:

> I'm not 100% sure but from the given information this might be related to
> FLINK-14498 [1] and partially relieved by FLINK-16645 [2].
>
> @Omkar Could you try the 1.11.0 release out and see whether the issue
> disappeared?
>
> @zhijiang <wa...@aliyun.com> @yingjie could you also take a
> look here? Thanks.
>
> Best Regards,
> Yu
>
> [1] https://issues.apache.org/jira/browse/FLINK-14498
> [2] https://issues.apache.org/jira/browse/FLINK-16645
>
>
> On Fri, 18 Sep 2020 at 09:28, Deshpande, Omkar <Om...@intuit.com>
> wrote:
>
>> These are the hostspot method. Any pointers on debugging this? The
>> checkpoints keep timing out since migrating to 1.10 from 1.9
>> ------------------------------
>> *From:* Deshpande, Omkar <Om...@intuit.com>
>> *Sent:* Wednesday, September 16, 2020 5:27 PM
>> *To:* Congxian Qiu <qc...@gmail.com>
>> *Cc:* user@flink.apache.org <us...@flink.apache.org>; Yun Tang <
>> myasuka@live.com>
>> *Subject:* Re: flink checkpoint timeout
>>
>> This email is from an external sender.
>>
>> This thread seems to stuck in awaiting notification state -
>> 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:1693)
>> at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
>> at
>> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
>> at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
>> at
>> org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestMemorySegmentBlocking(LocalBufferPool.java:231)
>>
>> ------------------------------
>> *From:* Congxian Qiu <qc...@gmail.com>
>> *Sent:* Monday, September 14, 2020 10:57 PM
>> *To:* Deshpande, Omkar <Om...@intuit.com>
>> *Cc:* user@flink.apache.org <us...@flink.apache.org>
>> *Subject:* Re: flink checkpoint timeout
>>
>> This email is from an external sender.
>>
>> Hi
>>     You can try to find out is there is some hot method, or the snapshot
>> stack is waiting for some lock. and maybe
>> Best,
>> Congxian
>>
>>
>> Deshpande, Omkar <Om...@intuit.com> 于2020年9月15日周二 下午12:30写道:
>>
>> Few of the subtasks fail. I cannot upgrade to 1.11 yet. But I can still
>> get the thread dump. What should I be looking for in the thread dump?
>>
>> ------------------------------
>> *From:* Yun Tang <my...@live.com>
>> *Sent:* Monday, September 14, 2020 8:52 PM
>> *To:* Deshpande, Omkar <Om...@intuit.com>;
>> user@flink.apache.org <us...@flink.apache.org>
>> *Subject:* Re: flink checkpoint timeout
>>
>> This email is from an external sender.
>>
>> Hi Omkar
>>
>> First of all, you should check the web UI of checkpoint [1] to see
>> whether many subtasks fail to complete in time or just few of them. The
>> former one might be your checkpoint time out is not enough for current
>> case. The later one might be some task stuck in slow machine or cannot grab
>> checkpoint lock to process sync phase of checkpointing, you can use thread
>> dump [2] (needs to bump to Flink-1.11) or jstack to see what happened in
>> java process.
>>
>> [1]
>> https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/checkpoint_monitoring.html
>> [2] https://issues.apache.org/jira/browse/FLINK-14816
>>
>> Best
>> Yun Tang
>> ------------------------------
>> *From:* Deshpande, Omkar <Om...@intuit.com>
>> *Sent:* Tuesday, September 15, 2020 10:25
>> *To:* user@flink.apache.org <us...@flink.apache.org>
>> *Subject:* Re: flink checkpoint timeout
>>
>> I have followed this
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_migration.html
>> <https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_migration.html#container-cut-off-memory>
>> and I am using taskmanager.memory.flink.size now instead of
>> taskmanager.heap.size
>> ------------------------------
>> *From:* Deshpande, Omkar <Om...@intuit.com>
>> *Sent:* Monday, September 14, 2020 6:23 PM
>> *To:* user@flink.apache.org <us...@flink.apache.org>
>> *Subject:* flink checkpoint timeout
>>
>> This email is from an external sender.
>>
>> Hello,
>>
>> I recently upgraded from flink 1.9 to 1.10. The checkpointing succeeds
>> first couple of times and then starts failing because of timeouts. The
>> checkpoint time grows with every checkpoint and starts exceeding 10
>> minutes. I do not see any exceptions in the logs. I have enabled debug
>> logging at "org.apache.flink" level. How do I investigate this? The garbage
>> collection seems fine. There is no backpressure. This used to work as is
>> with flink 1.9 without any issue.
>>
>> Any pointers on how to investigate long time taken to complete checkpoint?
>>
>> Omkar
>>
>>

-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng