Hi,
We have updated our Flink applications to 1.12.2, alone with the following modifications to improve its performance: - Use unaligned checkpoint - Change the following fs config - state.backend.fs.memory-threshold: 1048576 - state.backend.fs.write-buffer-size: 4194304 However, now our Flink applications will occasionally stuck when doing unaligned checkpoint or savepoint. The following are operators that stuck in our cases. - Kafka source connector. - BroadcastProcessFunction with data input, and broadcasted configuration. Also, when it is stuck, Flink also stops to consume any data. Since these operators do not have many data to be stored in checkpoint/savepoint, we wonder, how can we debug this problem? -- ChangZhuo Chen (陳昌倬) czchen@{czchen,debconf,debian}.org http://czchen.info/ Key fingerprint = BA04 346D C2E1 FE63 C790 8793 CC65 B0CD EC27 5D5B signature.asc (849 bytes) Download Attachment |
Hi ChangZhuo, Did you upgrade to Flink 1.12.2 and change the settings at the time? If so, could you maybe reset the settings to the old values on Flink 1.12.2 and check if the job still gets stuck? Especially, turning off unaligned checkpoints (UC) should clarify if it's a general issue in Flink 1.12.2 or with UC. If it's indeed an issue with UC, then it would help to get the debug logs in particular for the package org.apache.flink.streaming.runtime.io.checkpointing. You could add the following to your log4js.properties (set general log level to INFO). logger.checkpointing.name = org.apache.flink.streaming.runtime.io.checkpointing I would be especially interested to see how the broadcast is behaving. Also a final remark: UC is only improving performance of checkpointing if you are under backpressure. So maybe you don't need it at all. (I'd still be grateful for debug logs) On Wed, Mar 10, 2021 at 3:30 PM ChangZhuo Chen (陳昌倬) <[hidden email]> wrote: Hi, |
On Thu, Mar 11, 2021 at 02:14:32PM +0100, Arvid Heise wrote:
> Hi ChangZhuo, > > Did you upgrade to Flink 1.12.2 and change the settings at the time? If so, > could you maybe reset the settings to the old values on Flink 1.12.2 and > check if the job still gets stuck? Especially, turning off unaligned > checkpoints (UC) should clarify if it's a general issue in Flink 1.12.2 or > with UC. > > If it's indeed an issue with UC, then it would help to get the debug logs > in particular for the package > org.apache.flink.streaming.runtime.io.checkpointing. You could add the > following to your log4js.properties (set general log level to INFO). > > logger.checkpointing.name = org.apache.flink.streaming.runtime.io.checkpointing > logger.checkpointing.level = DEBUG when we get log. * Also, we got the stack track when checkpoint stuck, please let us know if you need full trace. * The stuck task in UI is KafkaProducer -> ProcessFunction 128 * The following is BLOCKED thread for Source: KafkaProducer -> ProcessFunction (129/140)#2 "Source: KafkaProducer -> ProcessFunction (129/140)#2" #66336 prio=5 os_prio=0 cpu=582.01ms elapsed=5079.15s tid=0x00007feb32717000 nid=0x9696 waiting for monitor entry [0x00007feb28b61000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:92) - waiting to lock <0x000000058e8c5070> (a java.lang.Object) 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) at java.lang.Thread.run(java.base@11.0.8/Thread.java:834) ps: * The original UID is redacted by their underlying type. * It looks like subtask id in UI is off-by-one in stacktrace. -- ChangZhuo Chen (陳昌倬) czchen@{czchen,debconf,debian}.org http://czchen.info/ Key fingerprint = BA04 346D C2E1 FE63 C790 8793 CC65 B0CD EC27 5D5B signature.asc (849 bytes) Download Attachment |
Yes, please send me the full stack trace. You could also send it to me personally if you don't want to share it on the ML. I'm especially interested in the legacy source thread that holds the lock 0x000000058e8c5070 if you only want to share an excerpt.On Fri, Mar 12, 2021 at 2:29 AM ChangZhuo Chen (陳昌倬) <[hidden email]> wrote: On Thu, Mar 11, 2021 at 02:14:32PM +0100, Arvid Heise wrote: |
Free forum by Nabble | Edit this page |