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

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 (陳昌倬) <czc...@czchen.org>
wrote:

> 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
>

Reply via email to