Hi Gordon!

Thank you for publishing this FLIP! I would like to ask several questions
and confirm my understanding of several aspects of this proposal. Even
though this discussion is focused on FLIP-319, as it is based on top of
KIP-939, my questions will also cover the KIP.



   1. Why does KafkaProducer have both commitTransaction and
   completeTransaction methods? Why can't we fuse them?
      1. Is it correct that commitTransaction and abortTransaction bring
      the transaction to its final state (committed/aborted) while
      completeTransaction is a recovery method that rolls a txn back/forward,
      depending on the prepared state?
      2. In what situations is PreparedTransactionState in the state store
      different from PreparedTransactionState in Kafka?
   2. Why does the pre-commit phase initialize the producer with
   currentProducer.initTransaction(false)?
      1. Is it because "false" means that we do not expect any prepared txn
      to be there, and if there is one, we should either roll it
forward or abort
      it? In the pre-commit phase with a new producer, there shouldn't be any
      dangling txns.
   3. Shouldn't we call completeTransaction on restore instead of
   commitTransaction? In what situations would the flink Kafka connector abort
   the transaction?
   4. Do we need to keep the current KafkaInternalProducer for a while to
   remain compatible with older Kafka versions that do not support KIP-939?
   5. How will the connector handle
   transaction.two.phase.commit.enable=false on the broker (not client) level?
   6. Does it make sense for the connector users to override
   transaction.two.phase.commit.enable? If it does not make sense, would the
   connector ignore the config or throw an exception when it is passed?


Best regards,
Alex

On Wed, Aug 23, 2023 at 6:09 AM Gyula Fóra <gyula.f...@gmail.com> wrote:

> Hi Gordon!
>
> Thank you for preparing the detailed FLIP, I think this is a huge
> improvement that enables the exactly-once Kafka sink in many environments /
> use-cases where this was previously unfeasible due to the limitations
> highlighted in the FLIP.
>
> Big +1
>
> Cheers,
> Gyula
>
> On Fri, Aug 18, 2023 at 7:54 PM Tzu-Li (Gordon) Tai <tzuli...@apache.org>
> wrote:
>
> > Hi Flink devs,
> >
> > I’d like to officially start a discussion for FLIP-319: Integrating with
> > Kafka’s proper support for 2PC participation (KIP-939) [1].
> >
> > This is the “sister” joint FLIP for KIP-939 [2] [3]. It has been a
> > long-standing issue that Flink’s Kafka connector doesn’t work fully
> > correctly under exactly-once mode due to lack of distributed transaction
> > support in the Kafka transaction protocol. This has led to subpar hacks
> in
> > the connector such as Java reflections to workaround the protocol's
> > limitations (which causes a bunch of problems on its own, e.g. long
> > recovery times for the connector), while still having corner case
> scenarios
> > that can lead to data loss.
> >
> > This joint effort with the Kafka community attempts to address this so
> that
> > the Flink Kafka connector can finally work against public Kafka APIs,
> which
> > should result in a much more robust integration between the two systems,
> > and for Flink developers, easier maintainability of the code.
> >
> > Obviously, actually implementing this FLIP relies on the joint KIP being
> > implemented and released first. Nevertheless, I'd like to start the
> > discussion for the design as early as possible so we can benefit from the
> > new Kafka changes as soon as it is available.
> >
> > Looking forward to feedback and comments on the proposal!
> >
> > Thanks,
> > Gordon
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255071710
> > [2]
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC
> > [3] https://lists.apache.org/thread/wbs9sqs3z1tdm7ptw5j4o9osmx9s41nf
> >
>

Reply via email to