Hi Mejri, Not exactly, you can still rely on savepoint to restart/redeploy the job from the latest offset recorded in Flink, my reply was regarding your question if you can replace that and just depend on the committed offsets in the kafka broker. For at-least-once semantic savepoints and checkpoints book-keep the offset for the Flink job after the initialization, the config I mentioned only configures the initialization of the consumers. If you start the job without savepoint and it falls back to the config (which may be using the broker committed offset) that might achieve the semantic but it doesn't guarantee that. For example assume you restore from save point, job completes a couple of checkpoints hence the offset committed is updated in kafka then for some reason you figure out a bug, if you only depend on Kafka broker committed offset you would probably break the semantic while if you use savepoints you can redeploy from the last correct version savepoint and reprocess the data that was processed by the buggy job.
Best Regards Ahmed Hamdy On Wed, 19 Mar 2025 at 00:54, mejri houssem <mejrihousse...@gmail.com> wrote: > Hello Ahmed, > > Thanks for the response. > > Does that mean checkpoints and savepoints have nothing to do with the > at-least-once guarantee, since it depends solely on the starting offset > configuration? > > Best Regards > > Le mar. 18 mars 2025 à 23:59, Ahmed Hamdy <hamdy10...@gmail.com> a écrit : > >> Hi Mejri >> >> > I’m wondering if this is strictly necessary, since the Kafka broker >> itself keeps track of offsets (i am not mistaken). In other words, if we >> redeploy the job, will it automatically resume from the last Kafka offset, >> or should we still rely on Flink’s checkpoint/savepoint mechanism to ensure >> correct offset recovery? >> >> This depends on the starting offset you set in the source config[1]. you >> can configure it to start from earliest or last committed or latest or at >> specific offset. >> >> I am not 100% sure about RabbitMQ, IIRC it uses checkpoints to ack read >> messages unlike Kafka. >> >> >> 1- >> https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/kafka/#starting-offset >> Best Regards >> Ahmed Hamdy >> >> >> On Tue, 18 Mar 2025 at 22:20, mejri houssem <mejrihousse...@gmail.com> >> wrote: >> >>> >>> Hello everyone, >>> >>> We have a stateless Flink job that uses a Kafka source with >>> at-least-once guarantees. We’ve enabled checkpoints so that, in the event >>> of a restart, Flink can restore from the last committed offset stored in a >>> successful checkpoint. Now we’re considering enabling savepoints for our >>> production deployment. >>> >>> I’m wondering if this is strictly necessary, since the Kafka broker >>> itself keeps track of offsets (i am not mistaken). In other words, if we >>> redeploy the job, will it automatically resume from the last Kafka offset, >>> or should we still rely on Flink’s checkpoint/savepoint mechanism to ensure >>> correct offset recovery? >>> >>> Additionally, we have another job that uses a RabbitMQ source with >>> checkpoints enabled to manage manual acknowledgments. Does the same logic >>> apply in that case as well? >>> >>> Thanks in advance for any guidance!point enabled in order to activate >>> manual ack. Does this apply to this job also? >>> >>> Thanks in advance. >>> >>> >>> Best Regards. >>> >>