Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-12 Thread Alexander Sorokoumov
Hi Jark, John, Thank you for the discussion! I will proceed with completing the patch that adds exactly-once to upsert-kafka connector. Best, Alexander On Wed, Apr 12, 2023 at 12:21 AM Jark Wu wrote: > Hi John, > > Thank you for your valuable input. It sounds reasonable to me. > > From this po

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-12 Thread Jark Wu
Hi John, Thank you for your valuable input. It sounds reasonable to me. >From this point of view, the exactly-once is used to guarantee transaction semantics other than avoid duplication/upserts. This is similar to the JDBC connectors that already support eventual consistency with idempotent upda

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-11 Thread John Roesler
Hi Jark, I hope you don’t mind if I chime in. You have a good point that the sequence of upserts will eventually converge to the correct value under the at-least-once delivery guarantee, but it can still be important to avoid passing on uncommitted results. Some thoughts, numbered for referenc

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-08 Thread Jark Wu
Hi Alexander, Yes, Kafka’s exactly-once semantics are used to avoid duplicated records in case of producer retries or failovers. But as I explained above, it can’t avoid intentionally duplicated records. Actually, I would like to call them "upsert records" instead of "duplicates", that's why the c

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-07 Thread Alexander Sorokoumov
Hi Jark, To my knowledge, Kafka's EXACTLY_ONCE transactions together with idempotent producers prevent duplicated records[1], at least in the cases when upstream does not produce them intentionally and across checkpoints. Could you please elaborate or point me to the docs that explain the reason

Re: [DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-07 Thread Jark Wu
Hi Alexander, I’m not sure I fully understand the reasons. I left my comments inline. > 1. There might be other non-Flink topic consumers that would rather not have duplicated records. Exactly once can’t avoid producing duplicated records. Because the upstream produces duplicated records intent

[DISCUSS] EXACTLY_ONCE delivery semantics for upsert-kafka connector

2023-04-04 Thread Alexander Sorokoumov
Hello Flink community, I would like to discuss if it is worth adding EXACTLY_ONCE delivery semantics to upsert-kafka connector. According to upsert-kafka docs[1] and ReducingUpsertSink javadoc[2], the connector is correct even with duplicate records under AT_LEAST_ONCE because the records are idem