Hello Gabriel,

What you're asking is a very fair question :) In fact, for Streams where
the partition-assignment to producer-consumer pairs are purely flexible, we
think the new EOS would not have hard requirement on transactional.id:
https://issues.apache.org/jira/browse/KAFKA-9453

I you implemented the transactional messaging via a DIY producer+consumer
though, it depends on how you'd expect the life-time of a producer, e.g. if
you do not have a 1-1 producer-consumer mapping then transactional.id is
not crucial, but if your have a N-1 producer-consumer mapping then you may
still need to configure that id.


Guozhang



On Fri, May 20, 2022 at 8:39 AM Gabriel Giussi <gabrielgiu...@gmail.com>
wrote:

> Before KIP-447 I understood the use of transactional.id to prevent us from
> zombies introducing duplicates, as explained in this talk
> https://youtu.be/j0l_zUhQaTc?t=822.
> So in order to get zombie fencing working correctly we should assign
> producers with a transactional.id that included the partition id,
> something
> like <application><topic>-<partition-id>, as shown in this slide
> https://youtu.be/j0l_zUhQaTc?t=1047 where processor 2 should use the same
> txnl.id A as the process 1 that crashed.
> This prevented us from having process 2 consuming the message again and
> committing, while process 1 could come back to life and also commit the
> pending transaction, hence having duplicates message being produced. In
> this case process 1 will be fenced by having an outdated epoch.
>
> With KIP-447 we no longer have that potential scenario of two pending
> transactions trying to produce and mark a message as committed, because we
> won't let process 2 even start the transaction if there is a pending one
> (basically by not returning any messages since we reject the Offset Fetch
> if a there is a pending transaction for that offset partition). This is
> explained in this post
>
> https://www.confluent.io/blog/simplified-robust-exactly-one-semantics-in-kafka-2-5/#client-api-simplification
>
> Having that, I don't see anymore the value of transactional.id or how I
> should configure it in my producers. The main benefit of KIP-447 is that we
> no longer have to start one producer per input partition, a quote from the
> post
> "The only way the static assignment requirement could be met is if each
> input partition uses a separate producer instance, which is in fact what
> Kafka Streams previously relied on. However, this made running EOS
> applications much more costly in terms of the client resources and load on
> the brokers. A large number of client connections could heavily impact the
> stability of brokers and become a waste of resources as well."
>
> I guess now I can reuse my producer between different input partitions, so
> what transactional.id should I assign to it and why should I care, isn't
> zombie fencing resolved by rejecting offset fetch already?
>
> Thanks.
>


-- 
-- Guozhang

Reply via email to