You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Pritam Agarwala <pr...@gmail.com> on 2023/05/09 06:23:51 UTC
OffsetCommitMode.Kafka_periodic with checkpointing enabled
Hi Team,
I need to get kafka-lag to prepare a graph and its dependent on kafka
committed offset. Flink is updating the offsets only after checkpointing to
make it consistent.
Default Behaviour as per doc :
If checkpoint is enabled, but consumer.setCommitOffsetsOnCheckpoints set to
false, then offset will not be committed at all even if the
enable.auto.commit is set to true.
So, when consumer.setCommitOffsetsOnCheckpoints set to false, *shouldn't it
fall back on the enable.auto.commit to do offset commit regularly since* *in
any case flink doesn't use consumer committed offsets for recovery.*
Jira Ticket : https://issues.apache.org/jira/browse/FLINK-32038
Thanks & Regards,
Pritam Agarwala
Senior Data Engineer
Re: OffsetCommitMode.Kafka_periodic with checkpointing enabled
Posted by Hang Ruan <ru...@gmail.com>.
Hi, Pritam,
I see Martijn has responsed the ticket.
Kafka source (FLIP-27) will commit offsets in two places: kafka consumer
auto commit and invoke `consumer.commitAsync` when checkpoint is completed.
- If the checkpoint is enabled and commit.offsets.on.checkpoint = true,
kafka connector commits offsets when a checkpoint is completed.
- If properties.enable.auto.commit = true, kafka consumers auto commit
offsets periodically (the interval depends on `
properties.auto.commit.interval.ms`).
If the checkpoint is enabled and commit.offsets.on.checkpoint = true and
properties.enable.auto.commit = true, I think the offsets will be committed
in both two places.
Best,
Hang
Pritam Agarwala <pr...@gmail.com> 于2023年5月9日周二 14:24写道:
> Hi Team,
>
> I need to get kafka-lag to prepare a graph and its dependent on kafka
> committed offset. Flink is updating the offsets only after checkpointing to
> make it consistent.
>
> Default Behaviour as per doc :
> If checkpoint is enabled, but consumer.setCommitOffsetsOnCheckpoints set
> to false, then offset will not be committed at all even if the
> enable.auto.commit is set to true.
>
> So, when consumer.setCommitOffsetsOnCheckpoints set to false, *shouldn't
> it fall back on the enable.auto.commit to do offset commit regularly since*
> *in any case flink doesn't use consumer committed offsets for recovery.*
>
>
> Jira Ticket : https://issues.apache.org/jira/browse/FLINK-32038
>
> Thanks & Regards,
> Pritam Agarwala
> Senior Data Engineer
>