Hi Alexey, You should definitely investigate why the job is stuck. 1. First of all, is it completely stuck, or is something moving? - Use Flink metrics [1] (number bytes/records processed), and go through all of the operators/tasks to check this. 2. The stack traces like the one you quoted: > 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) you can most likely ignore. Such Task ("Legacy Source Thread - Source: digital-itx-eastus2 -> Filter (6/6)#0") is backpressured and the problem lies downstream. 3. To check what tasks are backpressured, you can also use Flink metrics - check "isBackPressured" metric. Again, back pressured tasks are most likely not the source of the problem. Check downstream from the back pressured task. 4. First (the most upstream) not backpressured task, which is accepting/processing data from some backpressured tasks is the interesting one. It's causing backpressure and you need to investigate what is the problem. Take a look at it's stack traces, maybe attach a remote profiler and profile it's code (if it's making slow progress). Maybe it's stuck in your user code doing something.
Please let us know what you have found out. Piotrek [1] https://ci.apache.org/projects/flink/flink-docs-stable/ops/metrics.html pon., 22 mar 2021 o 19:18 Alexey Trenikhun <yen...@msn.com> napisał(a): > Great! I doubt that it will help in my case however, since in my case even > unaligned checkpoints “stuck”, in difference with aligned checkpoints, > after unaligned checkpoint triggered, Flink at some moment become idle, > kubernetes metrics report very little CPU usage by container, but unaligned > checkpoint still times out after 3hr. > > ------------------------------ > *From:* Arvid Heise <ar...@apache.org> > *Sent:* Monday, March 22, 2021 6:58:20 AM > *To:* ChangZhuo Chen (陳昌倬) <czc...@czchen.org> > *Cc:* Alexey Trenikhun <yen...@msn.com>; ro...@apache.org < > ro...@apache.org>; Flink User Mail List <user@flink.apache.org> > *Subject:* Re: Checkpoint fail due to timeout > > Hi Alexey, > > rescaling from unaligned checkpoints will be supported with the upcoming > 1.13 release (expected at the end of April). > > Best, > > Arvid > > On Wed, Mar 17, 2021 at 8:29 AM ChangZhuo Chen (陳昌倬) <czc...@czchen.org> > wrote: > > On Wed, Mar 17, 2021 at 05:45:38AM +0000, Alexey Trenikhun wrote: > > In my opinion looks similar. Were you able to tune-up Flink to make it > work? I'm stuck with it, I wanted to scale up hoping to reduce > backpressure, but to rescale I need to take savepoint, which never > completes (at least takes longer than 3 hours). > > You can use aligned checkpoint to scala your job. Just restarting from > checkpoint with the same jar file, and new parallelism shall do the > trick. > > > -- > ChangZhuo Chen (陳昌倬) czchen@{czchen,debian}.org > http://czchen.info/ > Key fingerprint = BA04 346D C2E1 FE63 C790 8793 CC65 B0CD EC27 5D5B > >