You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Guozhang Wang <wa...@gmail.com> on 2016/11/30 22:19:16 UTC

[DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hi all,

I have just created KIP-98 to enhance Kafka with exactly once delivery
semantics:

*https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*

This KIP adds a transactional messaging mechanism along with an idempotent
producer implementation to make sure that 1) duplicated messages sent from
the same identified producer can be detected on the broker side, and 2) a
group of messages sent within a transaction will atomically be either
reflected and fetchable to consumers or not as a whole.

The above wiki page provides a high-level view of the proposed changes as
well as summarized guarantees. Initial draft of the detailed implementation
design is described in this Google doc:

https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
0wSw9ra8


We would love to hear your comments and suggestions.

Thanks,

-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hello all,

I updated the KIP document with the changes discussed here. In particular:


   1. The consumer config was renamed from 'fetch.mode' to
   'isolation.level'. The values were changed appropriately as well.
   2. Clarified the transactional guarantees for the consumer.
   3. Added a subsection about the streams usecase for transactions, as
   this is a driving motivation behind the current proposal. The existing
   motivation was not strong enough, as evidenced by some of the discussions
   that took place here.
   4.  Miscellaneous minor clarifications which have been pointed out in
   the thread by multiple folks.


I have not yet updated the 'Rejected alternatives' since we have a bunch of
higher level proposals which are a bit open right now. I think the
proposals bifurcate into doing buffering client side vs complicating the
server side when handling transactions. We plan on doing a KIP call this
month where we can discuss our options in this regard, at which point we
will update the document (and rejected alternatives) to reflect the
collective decision.

Thanks for all the comments, it has been a great discussion so far! Here is
the KIP link, for convenience:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging

Apurva

On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Daniel Schierbeck <da...@zendesk.com.INVALID>.
Hi Apurva,

Thanks for taking the time to reply. You make excellent points, and it
sounds like the right tradeoff. It would be great if the coordinator code
could be shared with the consumer API, hope that's actually the case.

Daniel Schierbeck

On Thu, 1 Dec 2016 at 18.34 Apurva Mehta <ap...@confluent.io> wrote:

> Hi Daniel,
>
> That is a very good point. You are correct in saying that one does not need
> a transaction coordinator to get idempotent semantics.
>
> There are, however, three reasons why we chose this route:
>
> 1. The request to find a transaction coordinator is exactly the same as
> the request consumers use to find the group coordinator. So if clients
> already implement the new consumer, you should already have the code you
> need to find the transaction coordinator. I would even so far as to say
> that the majority coordinator discovery code can be effectively shared
> between producers and consumers. Jason should correct me on this, however,
> since he is most familiar with that bit.
> 2. With this route, the broker side changes are simpler. In particular,
> we have to implement the InitPIDRequest only in the coordinator.
> 3. By always having a transaction coordinator, we can enable
> applications to use transactions even if they don't specify the AppId. The
> only thing you lose is transaction recovery across sessions.
>
> Needless to say, we did debate this point extensively. What swung our
> decision ultimately was the following observation: if the user does not
> provide a transaction.app.id, the client can generate a UUID and use that
> as the appId for the rest of the session. This means that there are no
> branches in the client and server code, and is overall simpler to maintain.
> All the producer APIs are also available to the user and it would be more
> intuitive.
>
> It also means that clients cannot choose idempotence without transactions,
> and hence it does place a greater burden on implementors of kafka clients.
> But the cost should be minimal given point 1 above, and was deemed worth
> it.
>
> Thanks once more for your thoughtful comments. It would be great for other
> client implementors to chime in on this.
>
> Regards,
> Apurva
>
>
>
> On Thu, Dec 1, 2016 at 3:16 AM, Daniel Schierbeck
> <dasch@zendesk.com.invalid
> > wrote:
>
> > Hi there,
> >
> > I'm the author of ruby-kafka, and as such am slightly biased towards
> > simplicity of implementation :-)
> >
> > I like the proposal, and would love to use idempotent producer semantics
> in
> > our projects at Zendesk, but I'm a bit worried about the complexity that
> > would go into the clients; specifically: it sounds to me that in order to
> > get idempotent producer semantics, I'd have to implement the transaction
> > coordinator discovery. I may be wrong, but it would seem that it's not
> > strictly necessary if you're not using transactions – we could just use
> the
> > topic partition's leader as the coordinator, avoiding the extra
> discovery.
> > In my experience, most bugs are related to figuring out which broker is
> the
> > leader of which partition/group/whatever, so minimizing the number of
> > moving parts would be beneficial to me. I'd also like to point out that I
> > would be reluctant to implement the transaction API in the near future,
> but
> > would love to implement the idempotency API soon. The former seems only
> > relevant to real stream processing frameworks, which is probably not the
> > best use case for ruby-kafka.
> >
> > Cheers,
> > Daniel Schierbeck
> >
> > On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io>
> wrote:
> >
> > > Hey Neha,
> > >
> > > Thanks for the thoughtful questions. I'll try to address the first
> > question
> > > since Apurva addressed the second. Since most readers are probably
> > getting
> > > up to speed with this large proposal, let me first take a step back and
> > > explain why we need the AppID at all. As Confluent tradition demands, I
> > > present you a big wall of text:
> > >
> > > Clearly "exactly once" delivery requires resilience to client failures.
> > > When a client crashes or turns into a zombie, another client must
> > > eventually be started to resume the work. There are two problems: 1) we
> > > need to ensure that the old process is actually dead or at least that
> it
> > > cannot write any more data, and 2) we need to be able to pick up
> wherever
> > > the last process left off. To do either of these, we need some kind of
> > > identifier to tie the two instances together.
> > >
> > > There are only two choices for where this ID comes from: either the
> user
> > > gives it to us or the server generates it. In the latter case, the user
> > is
> > > responsible for fetching it from the client and persisting it somewhere
> > for
> > > use after failure. We ultimately felt that the most flexible option is
> to
> > > have the user give it to us. In many applications, there is already a
> > > natural identifier which is already used to divide the workload. For
> > > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > > applications where there is no natural ID, the user can generate a UUID
> > and
> > > persist it locally, which is as good as having the server generate it.
> > >
> > > So the AppID is used to provide continuity between the instances of a
> > > producer which are handling a certain workload. One of the early design
> > > decisions we made in this work was to make the delivery guarantees we
> > > provide agnostic of the workload that the producer is assigned. The
> > > producer is not in the business of trying to divide up the work among
> all
> > > its peers who are participating in the same duty (unlike the consumer,
> we
> > > don't know anything about where the data comes from). This has huge
> > > implications for "exactly-once" delivery because it puts the burden on
> > the
> > > user to divide the total workload among producer instances and to
> assign
> > > AppIDs accordingly.
> > >
> > > I've been using the term "workload" loosely, but we usually imagine
> > > something like Kafka Connect's notion of a "source partition." A source
> > > partition could be a topic partition if the source is Kafka, or it
> could
> > be
> > > a database table, a log file, or whatever makes sense for the source of
> > the
> > > data. The point is that it's an independent source of data which can be
> > > assigned to a producer instance.
> > >
> > > If the same source partition is always assigned to the producer with
> the
> > > the same AppID, then Kafka transactions will give you "exactly once"
> > > delivery without much additional work. On initialization, the producer
> > will
> > > ensure that 1) any previous producers using that AppID are "fenced"
> off,
> > > and 2) that any transaction which had been started by a previous
> producer
> > > with that AppID have either completed or aborted.
> > >
> > > Based on this, it should be clear that the ideal is to divide the
> > workload
> > > so that you have a one-to-one mapping from the source partition to the
> > > AppID. If the source of the data is Kafka, then the source partition is
> > > just a topic partition, and the AppID can be generated from the name of
> > the
> > > topic and the partition number.
> > >
> > > To finally get back to your auto-scaling question, let's assume for a
> > > moment the ideal mapping of source partition to AppID. The main
> question
> > is
> > > whether the scaling is "horizontal" or "vertical." By horizontal, I
> mean
> > an
> > > increase in the number of source partitions. This case is easy. Assign
> > new
> > > AppIDs based on the new source partitions and you're done.
> > >
> > > But if the scaling is vertical (i.e. an increase in the load on the
> > source
> > > partitions), there's not much this proposal can do to help. You're
> going
> > to
> > > have to break the source partition into child partitions, and assign
> each
> > > of the new partitions a new AppID. To preserve "exactly once" delivery,
> > you
> > > must make sure that the producers using the AppID assigned to the
> parent
> > > partition have been shutdown cleanly. We could provide a way to pass
> in a
> > > "parent AppID" so that the producer could check the appropriate safety
> > > conditions, but for the first version, we assume that users consider
> > > scaling requirements when dividing the workload into source partitions.
> > >
> > > Unfortunately, the real world is always falling short of the ideal, and
> > > it's not always practical to have a one-to-one mapping of source
> > partition
> > > to AppID, since that also implies a one-to-one mapping of source
> > partition
> > > to producer instance. If I were a user, I'd push this limit as far as
> is
> > > reasonable, but with enough source partitions, it eventually breaks
> down.
> > > At some point, you need a producer to handle the load of more than one
> > > source partition. This is fine in itself if the assignment is sticky:
> > that
> > > is, if we can ensure that the same source partition is assigned to the
> > > producer using a certain AppID. If not, then the user is responsible
> for
> > > ensuring a clean hand-off. The producer reading from the migrating
> source
> > > partition must stop reading, commit or abort any transaction containing
> > > data processed from that source partition, and then signal the producer
> > > which is taking over that it is safe to begin.
> > >
> > > This burden is a consequence of the decision to keep the producer out
> of
> > > the role of assigning work. We could do more if we forced users to
> > > formalize their application-specific notion of a source partition, and
> if
> > > we turned the producer into something like a consumer group, with a
> > > rebalance protocol. This would allow the broker to be the one to
> ensure a
> > > clean hand-off of work, but it would be a huge departure from the way
> the
> > > producer currently works, and not all applications have a notion of
> > source
> > > partition anyway. So the result is a bit more work for the user, though
> > of
> > > course it would be transparent to for Kafka Streams users.
> > >
> > > One final note. I've described above how to get the strongest
> guarantees
> > > that this work is capable of providing in an auto-scaling environment.
> We
> > > also provide weaker guarantees, which are still an improvement over the
> > > current state. For example, without specifying any kind of AppID, we
> > > provide idempotent production for the lifetime of a producer instance.
> > This
> > > ensures reliable delivery without duplicates even with broker failures.
> > It
> > > is also possible to use transactions without an ephemeral AppID. If the
> > > application generates a UUID for user as the AppID, and only uses it
> for
> > > the lifetime of a single producer, you can still take advantage of
> > > transactional semantics, which allows you to write to a set of messages
> > to
> > > multiple partitions atomically.
> > >
> > > Hope that answers the question and helps others understand the work a
> bit
> > > better!
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > >
> > > On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> > >
> > > > Thanks for your comment, I updated the document. Let me know if it is
> > > clear
> > > > now.
> > > >
> > > > Apurva
> > > >
> > > > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > > > onurkaraman.apache@gmail.com>
> > > > wrote:
> > > >
> > > > > @Apurva yep that's what I was trying to say.
> > > > >
> > > > > Original message:
> > > > > If there is already an entry with the AppID in the mapping,
> increment
> > > the
> > > > > epoch number and go on to the next step. If there is no entry with
> > the
> > > > > AppID in the mapping, construct a PID with initialized epoch
> number;
> > > > append
> > > > > an AppID message into the transaction topic, insert into the
> mapping
> > > and
> > > > > reply with the PID / epoch / timestamp.
> > > > >
> > > > > Just wanted to make it explicit because:
> > > > > 1. The "append an AppID message..." chunk was ambiguous on whether
> it
> > > > > applied to the "if exists" or "if not exists" condition
> > > > > 2. I think the google doc is pretty explicit on appending to the
> log
> > > > > everywhere else.
> > > > >
> > > > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <apurva@confluent.io
> >
> > > > wrote:
> > > > >
> > > > > > The first line in step 2 of that section is: "If there is already
> > an
> > > > > entry
> > > > > > with the AppID in the mapping, increment the epoch number and go
> on
> > > to
> > > > > the
> > > > > > next step."
> > > > > >
> > > > > > Are you suggesting that it be made explicit that 'increment the
> > epoch
> > > > > > number' includes persisting the updated value to the log?
> > > > > >
> > > > > > Thanks,
> > > > > > Apurva
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > > > onurkaraman.apache@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Nice google doc!
> > > > > > >
> > > > > > > Probably need to go over the google doc a few more times, but a
> > > minor
> > > > > > > comment from the first pass:
> > > > > > >
> > > > > > > In Transaction Coordinator Request Handling (
> > > > > > > https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > <https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>>
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > > > step 2 mentions that if the Transaction Coordinator doesn't
> > already
> > > > > see a
> > > > > > > producer with the same app-id, it creates a pid and appends
> > > (app-id,
> > > > > pid,
> > > > > > > epoch) into the transaction log.
> > > > > > >
> > > > > > > What about if the app-id/pid pair already exists and we
> increment
> > > the
> > > > > > > epoch? Should we append (app-id, pid, epoch++) to the
> transaction
> > > > log?
> > > > > I
> > > > > > > think we should, but step 2 doesn't mention this.
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <
> > apurva@confluent.io
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for your comments, let me deal with your second point
> > > > > regarding
> > > > > > > > merging the __consumer-offsets and transactions topic.
> > > > > > > >
> > > > > > > > Needless to say, we considered doing this, but chose to keep
> > them
> > > > > > > separate
> > > > > > > > for the following reasons:
> > > > > > > >
> > > > > > > > 1. Your assumption that group.id and transaction.app.id can
> be
> > > > > the
> > > > > > > same
> > > > > > > > does not hold for streams applications. All colocated tasks
> of
> > a
> > > > > > > streams
> > > > > > > > application will share the same consumer (and hence
> implicitly
> > > > > will
> > > > > > > have
> > > > > > > > the same group.id), but each task will have its own producer
> > > > > > > instance.
> > > > > > > > The transaction.app.id for each producer instance will still
> > > > have
> > > > > > to
> > > > > > > be
> > > > > > > > distinct. So to colocate the transaction and consumer group
> > > > > > > > coordinators,
> > > > > > > > we will have to now introduce a 'group.id' config in the
> > > > producer
> > > > > > and
> > > > > > > > require it to be the same as the consumer. This seemed like a
> > > > very
> > > > > > > > fragile
> > > > > > > > option.
> > > > > > > > 2. Following on from the above, the transaction coordinator
> and
> > > > > > group
> > > > > > > > coordinator would _have_ to be colocated inorder to be the
> > > > leader
> > > > > > for
> > > > > > > > the
> > > > > > > > same TopicPartition, unless we wanted to make even more
> > > > > fundamental
> > > > > > > > changes
> > > > > > > > to Kafka.
> > > > > > > > 3. We don't require that the consumer coordinator and the
> > > > > > transaction
> > > > > > > > coordinator have the same view of the current PID/Epoch pair.
> > > > If a
> > > > > > > > producer
> > > > > > > > instance is bounced, the epoch will be bumped. Any
> transactions
> > > > > > > > initiated
> > > > > > > > by the previous instance would either be fully committed or
> > > > fully
> > > > > > > rolled
> > > > > > > > back. Since the writes to the offset topics are just like
> > writes
> > > > > to
> > > > > > a
> > > > > > > > regular topic, these would enjoy the same guarantees, and the
> > > > > > > > inconsistency
> > > > > > > > will be eventually resolved.
> > > > > > > > 4. Finally, every application will have consumers, and hence
> > > > > record
> > > > > > > > consumer offsets. But a very small fraction of applications
> > > > would
> > > > > > use
> > > > > > > > transactions. Blending the two topics would make recovering
> > > > > > > transaction
> > > > > > > > coordinator state unnecessarily inefficient since it has to
> > read
> > > > > > from
> > > > > > > > the
> > > > > > > > beginning of the topic to reconstruct its data structures --
> it
> > > > > > would
> > > > > > > > have
> > > > > > > > to inspect and skip a majority of the messages if the offsets
> > > > were
> > > > > > in
> > > > > > > > the
> > > > > > > > same topic.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Apurva
> > > > > > > >
> > > > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> > > neha@confluent.io>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for initiating this KIP! I think it is well written
> > and
> > > > I'm
> > > > > > > > excited
> > > > > > > > > to see the first step towards adding an important feature
> in
> > > > Kafka.
> > > > > > > > >
> > > > > > > > > I had a few initial thoughts on the KIP, mostly not as
> deeply
> > > > > thought
> > > > > > > > > through than what you've done -
> > > > > > > > >
> > > > > > > > > 1. Perhaps you’ve thought about how this would work
> already —
> > > > since
> > > > > > we
> > > > > > > > now
> > > > > > > > > require a producer to specify a unique AppID across
> different
> > > > > > instances
> > > > > > > > of
> > > > > > > > > an application, how would applications that run in the
> cloud
> > > use
> > > > > this
> > > > > > > > > feature with auto scaling?
> > > > > > > > >
> > > > > > > > > 2. Making it easy for applications to get exactly-once
> > > semantics
> > > > > for
> > > > > > a
> > > > > > > > > consume-process-produce workflow is a great feature to
> have.
> > To
> > > > > > enable
> > > > > > > > > this, the proposal now includes letting a producer
> initiate a
> > > > write
> > > > > > to
> > > > > > > > the
> > > > > > > > > offset topic as well (just like consumers do). The consumer
> > > > > > coordinator
> > > > > > > > > (which could be on a different broker than the txn
> > coordinator)
> > > > > would
> > > > > > > > then
> > > > > > > > > validate if the PID and producer epoch is valid before it
> > > writes
> > > > to
> > > > > > the
> > > > > > > > > offset topic along with the associated PID. This is a great
> > > > feature
> > > > > > > > though
> > > > > > > > > I see 2 difficulties
> > > > > > > > >
> > > > > > > > > -- This needs the consumer coordinator to have a consistent
> > > view
> > > > of
> > > > > > the
> > > > > > > > > PID/epochs that is same as the view on the txn coordinator.
> > > > > However,
> > > > > > as
> > > > > > > > the
> > > > > > > > > offset and the transaction topics are different, the 2
> > > > coordinators
> > > > > > > might
> > > > > > > > > live on different brokers.
> > > > > > > > > -- We now also have 2 internal topics - a transaction topic
> > and
> > > > the
> > > > > > > > > __consumer_offsets topic.
> > > > > > > > >
> > > > > > > > > Maybe you’ve thought about this already and discarded it
> ...
> > > let
> > > > me
> > > > > > > make
> > > > > > > > a
> > > > > > > > > somewhat crazy proposal — Why don’t we upgrade the
> > transaction
> > > > > topic
> > > > > > to
> > > > > > > > be
> > > > > > > > > the new offsets topic as well? For consumers that want EoS
> > > > > guarantees
> > > > > > > for
> > > > > > > > > a consume-process-produce pattern, the group.id is the
> same
> > as
> > > > the
> > > > > > > > > transaction.app.id set for the producer. Assume that the
> > > > > transaction
> > > > > > > > topic
> > > > > > > > > also stores consumer offsets. It stores both the
> transaction
> > > > > metadata
> > > > > > > > > messages as well as offset messages, both for transactional
> > as
> > > > well
> > > > > > as
> > > > > > > > > non-transactional consumers. Since the group.id of the
> > > consumer
> > > > > and
> > > > > > > the
> > > > > > > > > app.id of the producer is the same, the offsets associated
> > > with
> > > > a
> > > > > > > > consumer
> > > > > > > > > group and topic-partition end up in the same transaction
> > topic
> > > > > > > partition
> > > > > > > > as
> > > > > > > > > the transaction metadata messages. The transaction
> > coordinator
> > > > and
> > > > > > the
> > > > > > > > > consumer coordinator always live on the same broker since
> > they
> > > > both
> > > > > > map
> > > > > > > > to
> > > > > > > > > the same partition in the transaction topic. Even if there
> > are
> > > > > > > failures,
> > > > > > > > > they end up on the same new broker. Hence, they share the
> > same
> > > > and
> > > > > > > > > consistent view of the PIDs, epochs and App IDs, whatever
> it
> > > is.
> > > > > The
> > > > > > > > > consumer coordinator will skip over the transaction
> metadata
> > > > > messages
> > > > > > > > when
> > > > > > > > > it bootstraps the offsets from this new topic for consumer
> > > groups
> > > > > > that
> > > > > > > > are
> > > > > > > > > not involved in a transaction and don’t have a txn id
> > > associated
> > > > > with
> > > > > > > the
> > > > > > > > > offset message in the transaction topic. The consumer
> > > coordinator
> > > > > > will
> > > > > > > > > expose only committed offsets in cases of consumer groups
> > that
> > > > are
> > > > > > > > involved
> > > > > > > > > in a txn. It will also be able to validate the
> > > > OffsetCommitRequests
> > > > > > > > coming
> > > > > > > > > from a transactional producer by ensuring that it is coming
> > > from
> > > > a
> > > > > > > valid
> > > > > > > > > PID, producer epoch since it uses the same view of this
> data
> > > > > created
> > > > > > by
> > > > > > > > the
> > > > > > > > > transaction coordinator (that lives on the same broker).
> And
> > we
> > > > > will
> > > > > > > end
> > > > > > > > up
> > > > > > > > > with one internal topic, not too.
> > > > > > > > >
> > > > > > > > > This proposal offers better operational simplicity and
> fewer
> > > > > internal
> > > > > > > > > topics but there are some downsides that come with it —
> there
> > > > are 2
> > > > > > > types
> > > > > > > > > of messages in one topic (txn metadata ones and offset
> ones).
> > > > Since
> > > > > > > this
> > > > > > > > > internal topic serves a dual purpose, it will be harder to
> > name
> > > > it
> > > > > > and
> > > > > > > > also
> > > > > > > > > design a message format that includes the different types
> of
> > > > > messages
> > > > > > > > that
> > > > > > > > > will live in the topic. Though the transaction topic
> already
> > > > needs
> > > > > to
> > > > > > > > write
> > > > > > > > > 5 different types of messages (the AppID->PID mapping, the
> > > > BeginTxn
> > > > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so
> > maybe
> > > > > adding
> > > > > > > the
> > > > > > > > > offset message isn't a big deal?
> > > > > > > > >
> > > > > > > > > Back when we introduced the offsets topic, we had discussed
> > > > making
> > > > > it
> > > > > > > > more
> > > > > > > > > general and allowing the producer to send offset commit
> > > messages
> > > > to
> > > > > > it
> > > > > > > > but
> > > > > > > > > ended up creating a specialized topic to allow the consumer
> > > > > > coordinator
> > > > > > > > to
> > > > > > > > > wall off and prevent unauthorized writes from consumers
> > outside
> > > > of
> > > > > a
> > > > > > > > group.
> > > > > > > > > Jason can comment on the specifics but I don't believe that
> > > goal
> > > > of
> > > > > > the
> > > > > > > > new
> > > > > > > > > consumer protocol was quite achieved.
> > > > > > > > >
> > > > > > > > > I have other comments on the message format, request names
> > etc
> > > > but
> > > > > > > wanted
> > > > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> > once
> > > > > > > delivery
> > > > > > > > > > semantics:
> > > > > > > > > >
> > > > > > > > > > *
> > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > <
> > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > >*
> > > > > > > > > >
> > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > with
> > > an
> > > > > > > > > idempotent
> > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > messages
> > > > > > sent
> > > > > > > > > from
> > > > > > > > > > the same identified producer can be detected on the
> broker
> > > > side,
> > > > > > and
> > > > > > > > 2) a
> > > > > > > > > > group of messages sent within a transaction will
> atomically
> > > be
> > > > > > either
> > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > > > > >
> > > > > > > > > > The above wiki page provides a high-level view of the
> > > proposed
> > > > > > > changes
> > > > > > > > as
> > > > > > > > > > well as summarized guarantees. Initial draft of the
> > detailed
> > > > > > > > > implementation
> > > > > > > > > > design is described in this Google doc:
> > > > > > > > > >
> > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > <https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>>
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > 0wSw9ra8
> > > > > > > > > > <https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > <https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>>
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > 0wSw9ra8>
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > -- Guozhang
> > > > > > > > > >
> > > > > > > > > --
> > > > > > > > > Thanks,
> > > > > > > > > Neha
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
@Henry

Seems like you cannot wait for the Streams EOS KIP any more :P

We do not make assumptions on the underlying state store's transaction
semantics, since it may not be necessarily RocksDB as users could customize
their own store engines.

As I mentioned, in the worst case we will invalidate the persistent store
engine and restore from changelog from "scratch", which can take long time
if we do not have any checkpointing mechanism (we have a JIRA open for it
and are debating whether to have it along with EOS at the first place). But
remember that with transactions many of the exceptions can now be handled
as simply retry sending since duplicate messages within the transaction
will be dedupped on the server side, only fatal errors will cause this
scenario.


Guozhang


On Thu, Dec 1, 2016 at 11:23 AM, Henry Cai <hc...@pinterest.com.invalid>
wrote:

> I see, having both topic data and changelog in kafka can make the kafka
> transaction atomic.  But in the case of streaming, the RocksDB is an
> external data source during applying changelog to RocksDB.  In terms of
> rollback and resume/re-apply kafka transaction, how do we make sure the
> RocksDB transaction can also rolled back and re-apply during kafka
> transaction boundary?
>
>
> On Thu, Dec 1, 2016 at 11:05 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > @Henry Cai,
> >
> > I am working on a separate KIP on Streams to leverage this KIP to have
> > exactly-once processing semantics (note the exactly-once processing is a
> > bit different from exactly-once delivery semantics), which should cover
> > your question.
> >
> > The short answer is that writing the changelog messages need to be part
> of
> > the transaction, and when a fatal error happens within a transaction,
> since
> > the store updates cannot be rolled back like the messages in the worst
> case
> > we need to restore from the changelog from scratch, or from a checkpoint
> > with a starting offset in changelog, and restoring consumer will fetch
> > committed messages only as well.
> >
> >
> > Guozhang
> >
> > On Thu, Dec 1, 2016 at 9:34 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Hi Daniel,
> > >
> > > That is a very good point. You are correct in saying that one does not
> > need
> > > a transaction coordinator to get idempotent semantics.
> > >
> > > There are, however, three reasons why we chose this route:
> > >
> > >    1. The request to find a transaction coordinator is exactly the same
> > as
> > >    the request consumers use to find the group coordinator. So if
> clients
> > >    already implement the new consumer, you should already have the code
> > you
> > >    need to find the transaction coordinator. I would even so far as to
> > say
> > >    that the majority coordinator discovery code can be effectively
> shared
> > >    between producers and consumers. Jason should correct me on this,
> > > however,
> > >    since he is most familiar with that bit.
> > >    2. With this route, the broker side changes are simpler. In
> > particular,
> > >    we have to implement the InitPIDRequest only in the coordinator.
> > >    3. By always having a transaction coordinator, we can enable
> > >    applications to use transactions even if they don't specify the
> AppId.
> > > The
> > >    only thing you lose is transaction recovery across sessions.
> > >
> > > Needless to say, we did debate this point extensively. What swung our
> > > decision ultimately was the following observation: if the user does not
> > > provide a transaction.app.id, the client can generate a UUID and use
> > that
> > > as the appId for the rest of the session. This means that there are no
> > > branches in the client and server code, and is overall simpler to
> > maintain.
> > > All the producer APIs are also available to the user and it would be
> more
> > > intuitive.
> > >
> > > It also means that clients cannot choose idempotence without
> > transactions,
> > > and hence it does place a greater burden on implementors of kafka
> > clients.
> > > But the cost should be minimal given point 1 above, and was deemed
> worth
> > > it.
> > >
> > > Thanks once more for your thoughtful comments. It would be great for
> > other
> > > client implementors to chime in on this.
> > >
> > > Regards,
> > > Apurva
> > >
> > >
> > > On Thu, Dec 1, 2016 at 3:16 AM, Daniel Schierbeck
> > > <dasch@zendesk.com.invalid
> > > > wrote:
> > >
> > > > Hi there,
> > > >
> > > > I'm the author of ruby-kafka, and as such am slightly biased towards
> > > > simplicity of implementation :-)
> > > >
> > > > I like the proposal, and would love to use idempotent producer
> > semantics
> > > in
> > > > our projects at Zendesk, but I'm a bit worried about the complexity
> > that
> > > > would go into the clients; specifically: it sounds to me that in
> order
> > to
> > > > get idempotent producer semantics, I'd have to implement the
> > transaction
> > > > coordinator discovery. I may be wrong, but it would seem that it's
> not
> > > > strictly necessary if you're not using transactions – we could just
> use
> > > the
> > > > topic partition's leader as the coordinator, avoiding the extra
> > > discovery.
> > > > In my experience, most bugs are related to figuring out which broker
> is
> > > the
> > > > leader of which partition/group/whatever, so minimizing the number of
> > > > moving parts would be beneficial to me. I'd also like to point out
> > that I
> > > > would be reluctant to implement the transaction API in the near
> future,
> > > but
> > > > would love to implement the idempotency API soon. The former seems
> only
> > > > relevant to real stream processing frameworks, which is probably not
> > the
> > > > best use case for ruby-kafka.
> > > >
> > > > Cheers,
> > > > Daniel Schierbeck
> > > >
> > > > On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > > >
> > > > > Hey Neha,
> > > > >
> > > > > Thanks for the thoughtful questions. I'll try to address the first
> > > > question
> > > > > since Apurva addressed the second. Since most readers are probably
> > > > getting
> > > > > up to speed with this large proposal, let me first take a step back
> > and
> > > > > explain why we need the AppID at all. As Confluent tradition
> > demands, I
> > > > > present you a big wall of text:
> > > > >
> > > > > Clearly "exactly once" delivery requires resilience to client
> > failures.
> > > > > When a client crashes or turns into a zombie, another client must
> > > > > eventually be started to resume the work. There are two problems:
> 1)
> > we
> > > > > need to ensure that the old process is actually dead or at least
> that
> > > it
> > > > > cannot write any more data, and 2) we need to be able to pick up
> > > wherever
> > > > > the last process left off. To do either of these, we need some kind
> > of
> > > > > identifier to tie the two instances together.
> > > > >
> > > > > There are only two choices for where this ID comes from: either the
> > > user
> > > > > gives it to us or the server generates it. In the latter case, the
> > user
> > > > is
> > > > > responsible for fetching it from the client and persisting it
> > somewhere
> > > > for
> > > > > use after failure. We ultimately felt that the most flexible option
> > is
> > > to
> > > > > have the user give it to us. In many applications, there is
> already a
> > > > > natural identifier which is already used to divide the workload.
> For
> > > > > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > > > > applications where there is no natural ID, the user can generate a
> > UUID
> > > > and
> > > > > persist it locally, which is as good as having the server generate
> > it.
> > > > >
> > > > > So the AppID is used to provide continuity between the instances
> of a
> > > > > producer which are handling a certain workload. One of the early
> > design
> > > > > decisions we made in this work was to make the delivery guarantees
> we
> > > > > provide agnostic of the workload that the producer is assigned. The
> > > > > producer is not in the business of trying to divide up the work
> among
> > > all
> > > > > its peers who are participating in the same duty (unlike the
> > consumer,
> > > we
> > > > > don't know anything about where the data comes from). This has huge
> > > > > implications for "exactly-once" delivery because it puts the burden
> > on
> > > > the
> > > > > user to divide the total workload among producer instances and to
> > > assign
> > > > > AppIDs accordingly.
> > > > >
> > > > > I've been using the term "workload" loosely, but we usually imagine
> > > > > something like Kafka Connect's notion of a "source partition." A
> > source
> > > > > partition could be a topic partition if the source is Kafka, or it
> > > could
> > > > be
> > > > > a database table, a log file, or whatever makes sense for the
> source
> > of
> > > > the
> > > > > data. The point is that it's an independent source of data which
> can
> > be
> > > > > assigned to a producer instance.
> > > > >
> > > > > If the same source partition is always assigned to the producer
> with
> > > the
> > > > > the same AppID, then Kafka transactions will give you "exactly
> once"
> > > > > delivery without much additional work. On initialization, the
> > producer
> > > > will
> > > > > ensure that 1) any previous producers using that AppID are "fenced"
> > > off,
> > > > > and 2) that any transaction which had been started by a previous
> > > producer
> > > > > with that AppID have either completed or aborted.
> > > > >
> > > > > Based on this, it should be clear that the ideal is to divide the
> > > > workload
> > > > > so that you have a one-to-one mapping from the source partition to
> > the
> > > > > AppID. If the source of the data is Kafka, then the source
> partition
> > is
> > > > > just a topic partition, and the AppID can be generated from the
> name
> > of
> > > > the
> > > > > topic and the partition number.
> > > > >
> > > > > To finally get back to your auto-scaling question, let's assume
> for a
> > > > > moment the ideal mapping of source partition to AppID. The main
> > > question
> > > > is
> > > > > whether the scaling is "horizontal" or "vertical." By horizontal, I
> > > mean
> > > > an
> > > > > increase in the number of source partitions. This case is easy.
> > Assign
> > > > new
> > > > > AppIDs based on the new source partitions and you're done.
> > > > >
> > > > > But if the scaling is vertical (i.e. an increase in the load on the
> > > > source
> > > > > partitions), there's not much this proposal can do to help. You're
> > > going
> > > > to
> > > > > have to break the source partition into child partitions, and
> assign
> > > each
> > > > > of the new partitions a new AppID. To preserve "exactly once"
> > delivery,
> > > > you
> > > > > must make sure that the producers using the AppID assigned to the
> > > parent
> > > > > partition have been shutdown cleanly. We could provide a way to
> pass
> > > in a
> > > > > "parent AppID" so that the producer could check the appropriate
> > safety
> > > > > conditions, but for the first version, we assume that users
> consider
> > > > > scaling requirements when dividing the workload into source
> > partitions.
> > > > >
> > > > > Unfortunately, the real world is always falling short of the ideal,
> > and
> > > > > it's not always practical to have a one-to-one mapping of source
> > > > partition
> > > > > to AppID, since that also implies a one-to-one mapping of source
> > > > partition
> > > > > to producer instance. If I were a user, I'd push this limit as far
> as
> > > is
> > > > > reasonable, but with enough source partitions, it eventually breaks
> > > down.
> > > > > At some point, you need a producer to handle the load of more than
> > one
> > > > > source partition. This is fine in itself if the assignment is
> sticky:
> > > > that
> > > > > is, if we can ensure that the same source partition is assigned to
> > the
> > > > > producer using a certain AppID. If not, then the user is
> responsible
> > > for
> > > > > ensuring a clean hand-off. The producer reading from the migrating
> > > source
> > > > > partition must stop reading, commit or abort any transaction
> > containing
> > > > > data processed from that source partition, and then signal the
> > producer
> > > > > which is taking over that it is safe to begin.
> > > > >
> > > > > This burden is a consequence of the decision to keep the producer
> out
> > > of
> > > > > the role of assigning work. We could do more if we forced users to
> > > > > formalize their application-specific notion of a source partition,
> > and
> > > if
> > > > > we turned the producer into something like a consumer group, with a
> > > > > rebalance protocol. This would allow the broker to be the one to
> > > ensure a
> > > > > clean hand-off of work, but it would be a huge departure from the
> way
> > > the
> > > > > producer currently works, and not all applications have a notion of
> > > > source
> > > > > partition anyway. So the result is a bit more work for the user,
> > though
> > > > of
> > > > > course it would be transparent to for Kafka Streams users.
> > > > >
> > > > > One final note. I've described above how to get the strongest
> > > guarantees
> > > > > that this work is capable of providing in an auto-scaling
> > environment.
> > > We
> > > > > also provide weaker guarantees, which are still an improvement over
> > the
> > > > > current state. For example, without specifying any kind of AppID,
> we
> > > > > provide idempotent production for the lifetime of a producer
> > instance.
> > > > This
> > > > > ensures reliable delivery without duplicates even with broker
> > failures.
> > > > It
> > > > > is also possible to use transactions without an ephemeral AppID. If
> > the
> > > > > application generates a UUID for user as the AppID, and only uses
> it
> > > for
> > > > > the lifetime of a single producer, you can still take advantage of
> > > > > transactional semantics, which allows you to write to a set of
> > messages
> > > > to
> > > > > multiple partitions atomically.
> > > > >
> > > > > Hope that answers the question and helps others understand the
> work a
> > > bit
> > > > > better!
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <apurva@confluent.io
> >
> > > > wrote:
> > > > >
> > > > > > Thanks for your comment, I updated the document. Let me know if
> it
> > is
> > > > > clear
> > > > > > now.
> > > > > >
> > > > > > Apurva
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > > > > > onurkaraman.apache@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > @Apurva yep that's what I was trying to say.
> > > > > > >
> > > > > > > Original message:
> > > > > > > If there is already an entry with the AppID in the mapping,
> > > increment
> > > > > the
> > > > > > > epoch number and go on to the next step. If there is no entry
> > with
> > > > the
> > > > > > > AppID in the mapping, construct a PID with initialized epoch
> > > number;
> > > > > > append
> > > > > > > an AppID message into the transaction topic, insert into the
> > > mapping
> > > > > and
> > > > > > > reply with the PID / epoch / timestamp.
> > > > > > >
> > > > > > > Just wanted to make it explicit because:
> > > > > > > 1. The "append an AppID message..." chunk was ambiguous on
> > whether
> > > it
> > > > > > > applied to the "if exists" or "if not exists" condition
> > > > > > > 2. I think the google doc is pretty explicit on appending to
> the
> > > log
> > > > > > > everywhere else.
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <
> > apurva@confluent.io
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > The first line in step 2 of that section is: "If there is
> > already
> > > > an
> > > > > > > entry
> > > > > > > > with the AppID in the mapping, increment the epoch number and
> > go
> > > on
> > > > > to
> > > > > > > the
> > > > > > > > next step."
> > > > > > > >
> > > > > > > > Are you suggesting that it be made explicit that 'increment
> the
> > > > epoch
> > > > > > > > number' includes persisting the updated value to the log?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Apurva
> > > > > > > >
> > > > > > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > > > > > onurkaraman.apache@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Nice google doc!
> > > > > > > > >
> > > > > > > > > Probably need to go over the google doc a few more times,
> > but a
> > > > > minor
> > > > > > > > > comment from the first pass:
> > > > > > > > >
> > > > > > > > > In Transaction Coordinator Request Handling (
> > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > > > > > step 2 mentions that if the Transaction Coordinator doesn't
> > > > already
> > > > > > > see a
> > > > > > > > > producer with the same app-id, it creates a pid and appends
> > > > > (app-id,
> > > > > > > pid,
> > > > > > > > > epoch) into the transaction log.
> > > > > > > > >
> > > > > > > > > What about if the app-id/pid pair already exists and we
> > > increment
> > > > > the
> > > > > > > > > epoch? Should we append (app-id, pid, epoch++) to the
> > > transaction
> > > > > > log?
> > > > > > > I
> > > > > > > > > think we should, but step 2 doesn't mention this.
> > > > > > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <
> > > > apurva@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for your comments, let me deal with your second
> > point
> > > > > > > regarding
> > > > > > > > > > merging the __consumer-offsets and transactions topic.
> > > > > > > > > >
> > > > > > > > > > Needless to say, we considered doing this, but chose to
> > keep
> > > > them
> > > > > > > > > separate
> > > > > > > > > > for the following reasons:
> > > > > > > > > >
> > > > > > > > > > 1. Your assumption that group.id and transaction.app.id
> > can
> > > be
> > > > > > > the
> > > > > > > > > same
> > > > > > > > > > does not hold for streams applications. All colocated
> tasks
> > > of
> > > > a
> > > > > > > > > streams
> > > > > > > > > > application will share the same consumer (and hence
> > > implicitly
> > > > > > > will
> > > > > > > > > have
> > > > > > > > > > the same group.id), but each task will have its own
> > producer
> > > > > > > > > instance.
> > > > > > > > > > The transaction.app.id for each producer instance will
> > still
> > > > > > have
> > > > > > > > to
> > > > > > > > > be
> > > > > > > > > > distinct. So to colocate the transaction and consumer
> group
> > > > > > > > > > coordinators,
> > > > > > > > > > we will have to now introduce a 'group.id' config in the
> > > > > > producer
> > > > > > > > and
> > > > > > > > > > require it to be the same as the consumer. This seemed
> > like a
> > > > > > very
> > > > > > > > > > fragile
> > > > > > > > > > option.
> > > > > > > > > > 2. Following on from the above, the transaction
> coordinator
> > > and
> > > > > > > > group
> > > > > > > > > > coordinator would _have_ to be colocated inorder to be
> the
> > > > > > leader
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > same TopicPartition, unless we wanted to make even more
> > > > > > > fundamental
> > > > > > > > > > changes
> > > > > > > > > > to Kafka.
> > > > > > > > > > 3. We don't require that the consumer coordinator and the
> > > > > > > > transaction
> > > > > > > > > > coordinator have the same view of the current PID/Epoch
> > pair.
> > > > > > If a
> > > > > > > > > > producer
> > > > > > > > > > instance is bounced, the epoch will be bumped. Any
> > > transactions
> > > > > > > > > > initiated
> > > > > > > > > > by the previous instance would either be fully committed
> or
> > > > > > fully
> > > > > > > > > rolled
> > > > > > > > > > back. Since the writes to the offset topics are just like
> > > > writes
> > > > > > > to
> > > > > > > > a
> > > > > > > > > > regular topic, these would enjoy the same guarantees, and
> > the
> > > > > > > > > > inconsistency
> > > > > > > > > > will be eventually resolved.
> > > > > > > > > > 4. Finally, every application will have consumers, and
> > hence
> > > > > > > record
> > > > > > > > > > consumer offsets. But a very small fraction of
> applications
> > > > > > would
> > > > > > > > use
> > > > > > > > > > transactions. Blending the two topics would make
> recovering
> > > > > > > > > transaction
> > > > > > > > > > coordinator state unnecessarily inefficient since it has
> to
> > > > read
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > beginning of the topic to reconstruct its data structures
> > --
> > > it
> > > > > > > > would
> > > > > > > > > > have
> > > > > > > > > > to inspect and skip a majority of the messages if the
> > offsets
> > > > > > were
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > same topic.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Apurva
> > > > > > > > > >
> > > > > > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> > > > > neha@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks for initiating this KIP! I think it is well
> > written
> > > > and
> > > > > > I'm
> > > > > > > > > > excited
> > > > > > > > > > > to see the first step towards adding an important
> feature
> > > in
> > > > > > Kafka.
> > > > > > > > > > >
> > > > > > > > > > > I had a few initial thoughts on the KIP, mostly not as
> > > deeply
> > > > > > > thought
> > > > > > > > > > > through than what you've done -
> > > > > > > > > > >
> > > > > > > > > > > 1. Perhaps you’ve thought about how this would work
> > > already —
> > > > > > since
> > > > > > > > we
> > > > > > > > > > now
> > > > > > > > > > > require a producer to specify a unique AppID across
> > > different
> > > > > > > > instances
> > > > > > > > > > of
> > > > > > > > > > > an application, how would applications that run in the
> > > cloud
> > > > > use
> > > > > > > this
> > > > > > > > > > > feature with auto scaling?
> > > > > > > > > > >
> > > > > > > > > > > 2. Making it easy for applications to get exactly-once
> > > > > semantics
> > > > > > > for
> > > > > > > > a
> > > > > > > > > > > consume-process-produce workflow is a great feature to
> > > have.
> > > > To
> > > > > > > > enable
> > > > > > > > > > > this, the proposal now includes letting a producer
> > > initiate a
> > > > > > write
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > offset topic as well (just like consumers do). The
> > consumer
> > > > > > > > coordinator
> > > > > > > > > > > (which could be on a different broker than the txn
> > > > coordinator)
> > > > > > > would
> > > > > > > > > > then
> > > > > > > > > > > validate if the PID and producer epoch is valid before
> it
> > > > > writes
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > offset topic along with the associated PID. This is a
> > great
> > > > > > feature
> > > > > > > > > > though
> > > > > > > > > > > I see 2 difficulties
> > > > > > > > > > >
> > > > > > > > > > > -- This needs the consumer coordinator to have a
> > consistent
> > > > > view
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > PID/epochs that is same as the view on the txn
> > coordinator.
> > > > > > > However,
> > > > > > > > as
> > > > > > > > > > the
> > > > > > > > > > > offset and the transaction topics are different, the 2
> > > > > > coordinators
> > > > > > > > > might
> > > > > > > > > > > live on different brokers.
> > > > > > > > > > > -- We now also have 2 internal topics - a transaction
> > topic
> > > > and
> > > > > > the
> > > > > > > > > > > __consumer_offsets topic.
> > > > > > > > > > >
> > > > > > > > > > > Maybe you’ve thought about this already and discarded
> it
> > > ...
> > > > > let
> > > > > > me
> > > > > > > > > make
> > > > > > > > > > a
> > > > > > > > > > > somewhat crazy proposal — Why don’t we upgrade the
> > > > transaction
> > > > > > > topic
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > > the new offsets topic as well? For consumers that want
> > EoS
> > > > > > > guarantees
> > > > > > > > > for
> > > > > > > > > > > a consume-process-produce pattern, the group.id is the
> > > same
> > > > as
> > > > > > the
> > > > > > > > > > > transaction.app.id set for the producer. Assume that
> the
> > > > > > > transaction
> > > > > > > > > > topic
> > > > > > > > > > > also stores consumer offsets. It stores both the
> > > transaction
> > > > > > > metadata
> > > > > > > > > > > messages as well as offset messages, both for
> > transactional
> > > > as
> > > > > > well
> > > > > > > > as
> > > > > > > > > > > non-transactional consumers. Since the group.id of the
> > > > > consumer
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > app.id of the producer is the same, the offsets
> > associated
> > > > > with
> > > > > > a
> > > > > > > > > > consumer
> > > > > > > > > > > group and topic-partition end up in the same
> transaction
> > > > topic
> > > > > > > > > partition
> > > > > > > > > > as
> > > > > > > > > > > the transaction metadata messages. The transaction
> > > > coordinator
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > consumer coordinator always live on the same broker
> since
> > > > they
> > > > > > both
> > > > > > > > map
> > > > > > > > > > to
> > > > > > > > > > > the same partition in the transaction topic. Even if
> > there
> > > > are
> > > > > > > > > failures,
> > > > > > > > > > > they end up on the same new broker. Hence, they share
> the
> > > > same
> > > > > > and
> > > > > > > > > > > consistent view of the PIDs, epochs and App IDs,
> whatever
> > > it
> > > > > is.
> > > > > > > The
> > > > > > > > > > > consumer coordinator will skip over the transaction
> > > metadata
> > > > > > > messages
> > > > > > > > > > when
> > > > > > > > > > > it bootstraps the offsets from this new topic for
> > consumer
> > > > > groups
> > > > > > > > that
> > > > > > > > > > are
> > > > > > > > > > > not involved in a transaction and don’t have a txn id
> > > > > associated
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > offset message in the transaction topic. The consumer
> > > > > coordinator
> > > > > > > > will
> > > > > > > > > > > expose only committed offsets in cases of consumer
> groups
> > > > that
> > > > > > are
> > > > > > > > > > involved
> > > > > > > > > > > in a txn. It will also be able to validate the
> > > > > > OffsetCommitRequests
> > > > > > > > > > coming
> > > > > > > > > > > from a transactional producer by ensuring that it is
> > coming
> > > > > from
> > > > > > a
> > > > > > > > > valid
> > > > > > > > > > > PID, producer epoch since it uses the same view of this
> > > data
> > > > > > > created
> > > > > > > > by
> > > > > > > > > > the
> > > > > > > > > > > transaction coordinator (that lives on the same
> broker).
> > > And
> > > > we
> > > > > > > will
> > > > > > > > > end
> > > > > > > > > > up
> > > > > > > > > > > with one internal topic, not too.
> > > > > > > > > > >
> > > > > > > > > > > This proposal offers better operational simplicity and
> > > fewer
> > > > > > > internal
> > > > > > > > > > > topics but there are some downsides that come with it —
> > > there
> > > > > > are 2
> > > > > > > > > types
> > > > > > > > > > > of messages in one topic (txn metadata ones and offset
> > > ones).
> > > > > > Since
> > > > > > > > > this
> > > > > > > > > > > internal topic serves a dual purpose, it will be harder
> > to
> > > > name
> > > > > > it
> > > > > > > > and
> > > > > > > > > > also
> > > > > > > > > > > design a message format that includes the different
> types
> > > of
> > > > > > > messages
> > > > > > > > > > that
> > > > > > > > > > > will live in the topic. Though the transaction topic
> > > already
> > > > > > needs
> > > > > > > to
> > > > > > > > > > write
> > > > > > > > > > > 5 different types of messages (the AppID->PID mapping,
> > the
> > > > > > BeginTxn
> > > > > > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted)
> so
> > > > maybe
> > > > > > > adding
> > > > > > > > > the
> > > > > > > > > > > offset message isn't a big deal?
> > > > > > > > > > >
> > > > > > > > > > > Back when we introduced the offsets topic, we had
> > discussed
> > > > > > making
> > > > > > > it
> > > > > > > > > > more
> > > > > > > > > > > general and allowing the producer to send offset commit
> > > > > messages
> > > > > > to
> > > > > > > > it
> > > > > > > > > > but
> > > > > > > > > > > ended up creating a specialized topic to allow the
> > consumer
> > > > > > > > coordinator
> > > > > > > > > > to
> > > > > > > > > > > wall off and prevent unauthorized writes from consumers
> > > > outside
> > > > > > of
> > > > > > > a
> > > > > > > > > > group.
> > > > > > > > > > > Jason can comment on the specifics but I don't believe
> > that
> > > > > goal
> > > > > > of
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > consumer protocol was quite achieved.
> > > > > > > > > > >
> > > > > > > > > > > I have other comments on the message format, request
> > names
> > > > etc
> > > > > > but
> > > > > > > > > wanted
> > > > > > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi all,
> > > > > > > > > > > >
> > > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> > exactly
> > > > once
> > > > > > > > > delivery
> > > > > > > > > > > > semantics:
> > > > > > > > > > > >
> > > > > > > > > > > > *
> > > > > > > > > > > > https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > > <
> > > > > > > > > > > > https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > > >*
> > > > > > > > > > > >
> > > > > > > > > > > > This KIP adds a transactional messaging mechanism
> along
> > > > with
> > > > > an
> > > > > > > > > > > idempotent
> > > > > > > > > > > > producer implementation to make sure that 1)
> duplicated
> > > > > > messages
> > > > > > > > sent
> > > > > > > > > > > from
> > > > > > > > > > > > the same identified producer can be detected on the
> > > broker
> > > > > > side,
> > > > > > > > and
> > > > > > > > > > 2) a
> > > > > > > > > > > > group of messages sent within a transaction will
> > > atomically
> > > > > be
> > > > > > > > either
> > > > > > > > > > > > reflected and fetchable to consumers or not as a
> whole.
> > > > > > > > > > > >
> > > > > > > > > > > > The above wiki page provides a high-level view of the
> > > > > proposed
> > > > > > > > > changes
> > > > > > > > > > as
> > > > > > > > > > > > well as summarized guarantees. Initial draft of the
> > > > detailed
> > > > > > > > > > > implementation
> > > > > > > > > > > > design is described in this Google doc:
> > > > > > > > > > > >
> > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > > > 0wSw9ra8
> > > > > > > > > > > > <https://docs.google.com/document/d/11Jqy_
> > > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > > 0wSw9ra8>
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > -- Guozhang
> > > > > > > > > > > >
> > > > > > > > > > > --
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Neha
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Henry Cai <hc...@pinterest.com.INVALID>.
I see, having both topic data and changelog in kafka can make the kafka
transaction atomic.  But in the case of streaming, the RocksDB is an
external data source during applying changelog to RocksDB.  In terms of
rollback and resume/re-apply kafka transaction, how do we make sure the
RocksDB transaction can also rolled back and re-apply during kafka
transaction boundary?


On Thu, Dec 1, 2016 at 11:05 AM, Guozhang Wang <wa...@gmail.com> wrote:

> @Henry Cai,
>
> I am working on a separate KIP on Streams to leverage this KIP to have
> exactly-once processing semantics (note the exactly-once processing is a
> bit different from exactly-once delivery semantics), which should cover
> your question.
>
> The short answer is that writing the changelog messages need to be part of
> the transaction, and when a fatal error happens within a transaction, since
> the store updates cannot be rolled back like the messages in the worst case
> we need to restore from the changelog from scratch, or from a checkpoint
> with a starting offset in changelog, and restoring consumer will fetch
> committed messages only as well.
>
>
> Guozhang
>
> On Thu, Dec 1, 2016 at 9:34 AM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Daniel,
> >
> > That is a very good point. You are correct in saying that one does not
> need
> > a transaction coordinator to get idempotent semantics.
> >
> > There are, however, three reasons why we chose this route:
> >
> >    1. The request to find a transaction coordinator is exactly the same
> as
> >    the request consumers use to find the group coordinator. So if clients
> >    already implement the new consumer, you should already have the code
> you
> >    need to find the transaction coordinator. I would even so far as to
> say
> >    that the majority coordinator discovery code can be effectively shared
> >    between producers and consumers. Jason should correct me on this,
> > however,
> >    since he is most familiar with that bit.
> >    2. With this route, the broker side changes are simpler. In
> particular,
> >    we have to implement the InitPIDRequest only in the coordinator.
> >    3. By always having a transaction coordinator, we can enable
> >    applications to use transactions even if they don't specify the AppId.
> > The
> >    only thing you lose is transaction recovery across sessions.
> >
> > Needless to say, we did debate this point extensively. What swung our
> > decision ultimately was the following observation: if the user does not
> > provide a transaction.app.id, the client can generate a UUID and use
> that
> > as the appId for the rest of the session. This means that there are no
> > branches in the client and server code, and is overall simpler to
> maintain.
> > All the producer APIs are also available to the user and it would be more
> > intuitive.
> >
> > It also means that clients cannot choose idempotence without
> transactions,
> > and hence it does place a greater burden on implementors of kafka
> clients.
> > But the cost should be minimal given point 1 above, and was deemed worth
> > it.
> >
> > Thanks once more for your thoughtful comments. It would be great for
> other
> > client implementors to chime in on this.
> >
> > Regards,
> > Apurva
> >
> >
> > On Thu, Dec 1, 2016 at 3:16 AM, Daniel Schierbeck
> > <dasch@zendesk.com.invalid
> > > wrote:
> >
> > > Hi there,
> > >
> > > I'm the author of ruby-kafka, and as such am slightly biased towards
> > > simplicity of implementation :-)
> > >
> > > I like the proposal, and would love to use idempotent producer
> semantics
> > in
> > > our projects at Zendesk, but I'm a bit worried about the complexity
> that
> > > would go into the clients; specifically: it sounds to me that in order
> to
> > > get idempotent producer semantics, I'd have to implement the
> transaction
> > > coordinator discovery. I may be wrong, but it would seem that it's not
> > > strictly necessary if you're not using transactions – we could just use
> > the
> > > topic partition's leader as the coordinator, avoiding the extra
> > discovery.
> > > In my experience, most bugs are related to figuring out which broker is
> > the
> > > leader of which partition/group/whatever, so minimizing the number of
> > > moving parts would be beneficial to me. I'd also like to point out
> that I
> > > would be reluctant to implement the transaction API in the near future,
> > but
> > > would love to implement the idempotency API soon. The former seems only
> > > relevant to real stream processing frameworks, which is probably not
> the
> > > best use case for ruby-kafka.
> > >
> > > Cheers,
> > > Daniel Schierbeck
> > >
> > > On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io>
> > wrote:
> > >
> > > > Hey Neha,
> > > >
> > > > Thanks for the thoughtful questions. I'll try to address the first
> > > question
> > > > since Apurva addressed the second. Since most readers are probably
> > > getting
> > > > up to speed with this large proposal, let me first take a step back
> and
> > > > explain why we need the AppID at all. As Confluent tradition
> demands, I
> > > > present you a big wall of text:
> > > >
> > > > Clearly "exactly once" delivery requires resilience to client
> failures.
> > > > When a client crashes or turns into a zombie, another client must
> > > > eventually be started to resume the work. There are two problems: 1)
> we
> > > > need to ensure that the old process is actually dead or at least that
> > it
> > > > cannot write any more data, and 2) we need to be able to pick up
> > wherever
> > > > the last process left off. To do either of these, we need some kind
> of
> > > > identifier to tie the two instances together.
> > > >
> > > > There are only two choices for where this ID comes from: either the
> > user
> > > > gives it to us or the server generates it. In the latter case, the
> user
> > > is
> > > > responsible for fetching it from the client and persisting it
> somewhere
> > > for
> > > > use after failure. We ultimately felt that the most flexible option
> is
> > to
> > > > have the user give it to us. In many applications, there is already a
> > > > natural identifier which is already used to divide the workload. For
> > > > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > > > applications where there is no natural ID, the user can generate a
> UUID
> > > and
> > > > persist it locally, which is as good as having the server generate
> it.
> > > >
> > > > So the AppID is used to provide continuity between the instances of a
> > > > producer which are handling a certain workload. One of the early
> design
> > > > decisions we made in this work was to make the delivery guarantees we
> > > > provide agnostic of the workload that the producer is assigned. The
> > > > producer is not in the business of trying to divide up the work among
> > all
> > > > its peers who are participating in the same duty (unlike the
> consumer,
> > we
> > > > don't know anything about where the data comes from). This has huge
> > > > implications for "exactly-once" delivery because it puts the burden
> on
> > > the
> > > > user to divide the total workload among producer instances and to
> > assign
> > > > AppIDs accordingly.
> > > >
> > > > I've been using the term "workload" loosely, but we usually imagine
> > > > something like Kafka Connect's notion of a "source partition." A
> source
> > > > partition could be a topic partition if the source is Kafka, or it
> > could
> > > be
> > > > a database table, a log file, or whatever makes sense for the source
> of
> > > the
> > > > data. The point is that it's an independent source of data which can
> be
> > > > assigned to a producer instance.
> > > >
> > > > If the same source partition is always assigned to the producer with
> > the
> > > > the same AppID, then Kafka transactions will give you "exactly once"
> > > > delivery without much additional work. On initialization, the
> producer
> > > will
> > > > ensure that 1) any previous producers using that AppID are "fenced"
> > off,
> > > > and 2) that any transaction which had been started by a previous
> > producer
> > > > with that AppID have either completed or aborted.
> > > >
> > > > Based on this, it should be clear that the ideal is to divide the
> > > workload
> > > > so that you have a one-to-one mapping from the source partition to
> the
> > > > AppID. If the source of the data is Kafka, then the source partition
> is
> > > > just a topic partition, and the AppID can be generated from the name
> of
> > > the
> > > > topic and the partition number.
> > > >
> > > > To finally get back to your auto-scaling question, let's assume for a
> > > > moment the ideal mapping of source partition to AppID. The main
> > question
> > > is
> > > > whether the scaling is "horizontal" or "vertical." By horizontal, I
> > mean
> > > an
> > > > increase in the number of source partitions. This case is easy.
> Assign
> > > new
> > > > AppIDs based on the new source partitions and you're done.
> > > >
> > > > But if the scaling is vertical (i.e. an increase in the load on the
> > > source
> > > > partitions), there's not much this proposal can do to help. You're
> > going
> > > to
> > > > have to break the source partition into child partitions, and assign
> > each
> > > > of the new partitions a new AppID. To preserve "exactly once"
> delivery,
> > > you
> > > > must make sure that the producers using the AppID assigned to the
> > parent
> > > > partition have been shutdown cleanly. We could provide a way to pass
> > in a
> > > > "parent AppID" so that the producer could check the appropriate
> safety
> > > > conditions, but for the first version, we assume that users consider
> > > > scaling requirements when dividing the workload into source
> partitions.
> > > >
> > > > Unfortunately, the real world is always falling short of the ideal,
> and
> > > > it's not always practical to have a one-to-one mapping of source
> > > partition
> > > > to AppID, since that also implies a one-to-one mapping of source
> > > partition
> > > > to producer instance. If I were a user, I'd push this limit as far as
> > is
> > > > reasonable, but with enough source partitions, it eventually breaks
> > down.
> > > > At some point, you need a producer to handle the load of more than
> one
> > > > source partition. This is fine in itself if the assignment is sticky:
> > > that
> > > > is, if we can ensure that the same source partition is assigned to
> the
> > > > producer using a certain AppID. If not, then the user is responsible
> > for
> > > > ensuring a clean hand-off. The producer reading from the migrating
> > source
> > > > partition must stop reading, commit or abort any transaction
> containing
> > > > data processed from that source partition, and then signal the
> producer
> > > > which is taking over that it is safe to begin.
> > > >
> > > > This burden is a consequence of the decision to keep the producer out
> > of
> > > > the role of assigning work. We could do more if we forced users to
> > > > formalize their application-specific notion of a source partition,
> and
> > if
> > > > we turned the producer into something like a consumer group, with a
> > > > rebalance protocol. This would allow the broker to be the one to
> > ensure a
> > > > clean hand-off of work, but it would be a huge departure from the way
> > the
> > > > producer currently works, and not all applications have a notion of
> > > source
> > > > partition anyway. So the result is a bit more work for the user,
> though
> > > of
> > > > course it would be transparent to for Kafka Streams users.
> > > >
> > > > One final note. I've described above how to get the strongest
> > guarantees
> > > > that this work is capable of providing in an auto-scaling
> environment.
> > We
> > > > also provide weaker guarantees, which are still an improvement over
> the
> > > > current state. For example, without specifying any kind of AppID, we
> > > > provide idempotent production for the lifetime of a producer
> instance.
> > > This
> > > > ensures reliable delivery without duplicates even with broker
> failures.
> > > It
> > > > is also possible to use transactions without an ephemeral AppID. If
> the
> > > > application generates a UUID for user as the AppID, and only uses it
> > for
> > > > the lifetime of a single producer, you can still take advantage of
> > > > transactional semantics, which allows you to write to a set of
> messages
> > > to
> > > > multiple partitions atomically.
> > > >
> > > > Hope that answers the question and helps others understand the work a
> > bit
> > > > better!
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > >
> > > >
> > > >
> > > > On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > > >
> > > > > Thanks for your comment, I updated the document. Let me know if it
> is
> > > > clear
> > > > > now.
> > > > >
> > > > > Apurva
> > > > >
> > > > > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > > > > onurkaraman.apache@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > @Apurva yep that's what I was trying to say.
> > > > > >
> > > > > > Original message:
> > > > > > If there is already an entry with the AppID in the mapping,
> > increment
> > > > the
> > > > > > epoch number and go on to the next step. If there is no entry
> with
> > > the
> > > > > > AppID in the mapping, construct a PID with initialized epoch
> > number;
> > > > > append
> > > > > > an AppID message into the transaction topic, insert into the
> > mapping
> > > > and
> > > > > > reply with the PID / epoch / timestamp.
> > > > > >
> > > > > > Just wanted to make it explicit because:
> > > > > > 1. The "append an AppID message..." chunk was ambiguous on
> whether
> > it
> > > > > > applied to the "if exists" or "if not exists" condition
> > > > > > 2. I think the google doc is pretty explicit on appending to the
> > log
> > > > > > everywhere else.
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <
> apurva@confluent.io
> > >
> > > > > wrote:
> > > > > >
> > > > > > > The first line in step 2 of that section is: "If there is
> already
> > > an
> > > > > > entry
> > > > > > > with the AppID in the mapping, increment the epoch number and
> go
> > on
> > > > to
> > > > > > the
> > > > > > > next step."
> > > > > > >
> > > > > > > Are you suggesting that it be made explicit that 'increment the
> > > epoch
> > > > > > > number' includes persisting the updated value to the log?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Apurva
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > > > > onurkaraman.apache@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Nice google doc!
> > > > > > > >
> > > > > > > > Probably need to go over the google doc a few more times,
> but a
> > > > minor
> > > > > > > > comment from the first pass:
> > > > > > > >
> > > > > > > > In Transaction Coordinator Request Handling (
> > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > > > > step 2 mentions that if the Transaction Coordinator doesn't
> > > already
> > > > > > see a
> > > > > > > > producer with the same app-id, it creates a pid and appends
> > > > (app-id,
> > > > > > pid,
> > > > > > > > epoch) into the transaction log.
> > > > > > > >
> > > > > > > > What about if the app-id/pid pair already exists and we
> > increment
> > > > the
> > > > > > > > epoch? Should we append (app-id, pid, epoch++) to the
> > transaction
> > > > > log?
> > > > > > I
> > > > > > > > think we should, but step 2 doesn't mention this.
> > > > > > > >
> > > > > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <
> > > apurva@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for your comments, let me deal with your second
> point
> > > > > > regarding
> > > > > > > > > merging the __consumer-offsets and transactions topic.
> > > > > > > > >
> > > > > > > > > Needless to say, we considered doing this, but chose to
> keep
> > > them
> > > > > > > > separate
> > > > > > > > > for the following reasons:
> > > > > > > > >
> > > > > > > > > 1. Your assumption that group.id and transaction.app.id
> can
> > be
> > > > > > the
> > > > > > > > same
> > > > > > > > > does not hold for streams applications. All colocated tasks
> > of
> > > a
> > > > > > > > streams
> > > > > > > > > application will share the same consumer (and hence
> > implicitly
> > > > > > will
> > > > > > > > have
> > > > > > > > > the same group.id), but each task will have its own
> producer
> > > > > > > > instance.
> > > > > > > > > The transaction.app.id for each producer instance will
> still
> > > > > have
> > > > > > > to
> > > > > > > > be
> > > > > > > > > distinct. So to colocate the transaction and consumer group
> > > > > > > > > coordinators,
> > > > > > > > > we will have to now introduce a 'group.id' config in the
> > > > > producer
> > > > > > > and
> > > > > > > > > require it to be the same as the consumer. This seemed
> like a
> > > > > very
> > > > > > > > > fragile
> > > > > > > > > option.
> > > > > > > > > 2. Following on from the above, the transaction coordinator
> > and
> > > > > > > group
> > > > > > > > > coordinator would _have_ to be colocated inorder to be the
> > > > > leader
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > same TopicPartition, unless we wanted to make even more
> > > > > > fundamental
> > > > > > > > > changes
> > > > > > > > > to Kafka.
> > > > > > > > > 3. We don't require that the consumer coordinator and the
> > > > > > > transaction
> > > > > > > > > coordinator have the same view of the current PID/Epoch
> pair.
> > > > > If a
> > > > > > > > > producer
> > > > > > > > > instance is bounced, the epoch will be bumped. Any
> > transactions
> > > > > > > > > initiated
> > > > > > > > > by the previous instance would either be fully committed or
> > > > > fully
> > > > > > > > rolled
> > > > > > > > > back. Since the writes to the offset topics are just like
> > > writes
> > > > > > to
> > > > > > > a
> > > > > > > > > regular topic, these would enjoy the same guarantees, and
> the
> > > > > > > > > inconsistency
> > > > > > > > > will be eventually resolved.
> > > > > > > > > 4. Finally, every application will have consumers, and
> hence
> > > > > > record
> > > > > > > > > consumer offsets. But a very small fraction of applications
> > > > > would
> > > > > > > use
> > > > > > > > > transactions. Blending the two topics would make recovering
> > > > > > > > transaction
> > > > > > > > > coordinator state unnecessarily inefficient since it has to
> > > read
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > beginning of the topic to reconstruct its data structures
> --
> > it
> > > > > > > would
> > > > > > > > > have
> > > > > > > > > to inspect and skip a majority of the messages if the
> offsets
> > > > > were
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > same topic.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Apurva
> > > > > > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> > > > neha@confluent.io>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for initiating this KIP! I think it is well
> written
> > > and
> > > > > I'm
> > > > > > > > > excited
> > > > > > > > > > to see the first step towards adding an important feature
> > in
> > > > > Kafka.
> > > > > > > > > >
> > > > > > > > > > I had a few initial thoughts on the KIP, mostly not as
> > deeply
> > > > > > thought
> > > > > > > > > > through than what you've done -
> > > > > > > > > >
> > > > > > > > > > 1. Perhaps you’ve thought about how this would work
> > already —
> > > > > since
> > > > > > > we
> > > > > > > > > now
> > > > > > > > > > require a producer to specify a unique AppID across
> > different
> > > > > > > instances
> > > > > > > > > of
> > > > > > > > > > an application, how would applications that run in the
> > cloud
> > > > use
> > > > > > this
> > > > > > > > > > feature with auto scaling?
> > > > > > > > > >
> > > > > > > > > > 2. Making it easy for applications to get exactly-once
> > > > semantics
> > > > > > for
> > > > > > > a
> > > > > > > > > > consume-process-produce workflow is a great feature to
> > have.
> > > To
> > > > > > > enable
> > > > > > > > > > this, the proposal now includes letting a producer
> > initiate a
> > > > > write
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > offset topic as well (just like consumers do). The
> consumer
> > > > > > > coordinator
> > > > > > > > > > (which could be on a different broker than the txn
> > > coordinator)
> > > > > > would
> > > > > > > > > then
> > > > > > > > > > validate if the PID and producer epoch is valid before it
> > > > writes
> > > > > to
> > > > > > > the
> > > > > > > > > > offset topic along with the associated PID. This is a
> great
> > > > > feature
> > > > > > > > > though
> > > > > > > > > > I see 2 difficulties
> > > > > > > > > >
> > > > > > > > > > -- This needs the consumer coordinator to have a
> consistent
> > > > view
> > > > > of
> > > > > > > the
> > > > > > > > > > PID/epochs that is same as the view on the txn
> coordinator.
> > > > > > However,
> > > > > > > as
> > > > > > > > > the
> > > > > > > > > > offset and the transaction topics are different, the 2
> > > > > coordinators
> > > > > > > > might
> > > > > > > > > > live on different brokers.
> > > > > > > > > > -- We now also have 2 internal topics - a transaction
> topic
> > > and
> > > > > the
> > > > > > > > > > __consumer_offsets topic.
> > > > > > > > > >
> > > > > > > > > > Maybe you’ve thought about this already and discarded it
> > ...
> > > > let
> > > > > me
> > > > > > > > make
> > > > > > > > > a
> > > > > > > > > > somewhat crazy proposal — Why don’t we upgrade the
> > > transaction
> > > > > > topic
> > > > > > > to
> > > > > > > > > be
> > > > > > > > > > the new offsets topic as well? For consumers that want
> EoS
> > > > > > guarantees
> > > > > > > > for
> > > > > > > > > > a consume-process-produce pattern, the group.id is the
> > same
> > > as
> > > > > the
> > > > > > > > > > transaction.app.id set for the producer. Assume that the
> > > > > > transaction
> > > > > > > > > topic
> > > > > > > > > > also stores consumer offsets. It stores both the
> > transaction
> > > > > > metadata
> > > > > > > > > > messages as well as offset messages, both for
> transactional
> > > as
> > > > > well
> > > > > > > as
> > > > > > > > > > non-transactional consumers. Since the group.id of the
> > > > consumer
> > > > > > and
> > > > > > > > the
> > > > > > > > > > app.id of the producer is the same, the offsets
> associated
> > > > with
> > > > > a
> > > > > > > > > consumer
> > > > > > > > > > group and topic-partition end up in the same transaction
> > > topic
> > > > > > > > partition
> > > > > > > > > as
> > > > > > > > > > the transaction metadata messages. The transaction
> > > coordinator
> > > > > and
> > > > > > > the
> > > > > > > > > > consumer coordinator always live on the same broker since
> > > they
> > > > > both
> > > > > > > map
> > > > > > > > > to
> > > > > > > > > > the same partition in the transaction topic. Even if
> there
> > > are
> > > > > > > > failures,
> > > > > > > > > > they end up on the same new broker. Hence, they share the
> > > same
> > > > > and
> > > > > > > > > > consistent view of the PIDs, epochs and App IDs, whatever
> > it
> > > > is.
> > > > > > The
> > > > > > > > > > consumer coordinator will skip over the transaction
> > metadata
> > > > > > messages
> > > > > > > > > when
> > > > > > > > > > it bootstraps the offsets from this new topic for
> consumer
> > > > groups
> > > > > > > that
> > > > > > > > > are
> > > > > > > > > > not involved in a transaction and don’t have a txn id
> > > > associated
> > > > > > with
> > > > > > > > the
> > > > > > > > > > offset message in the transaction topic. The consumer
> > > > coordinator
> > > > > > > will
> > > > > > > > > > expose only committed offsets in cases of consumer groups
> > > that
> > > > > are
> > > > > > > > > involved
> > > > > > > > > > in a txn. It will also be able to validate the
> > > > > OffsetCommitRequests
> > > > > > > > > coming
> > > > > > > > > > from a transactional producer by ensuring that it is
> coming
> > > > from
> > > > > a
> > > > > > > > valid
> > > > > > > > > > PID, producer epoch since it uses the same view of this
> > data
> > > > > > created
> > > > > > > by
> > > > > > > > > the
> > > > > > > > > > transaction coordinator (that lives on the same broker).
> > And
> > > we
> > > > > > will
> > > > > > > > end
> > > > > > > > > up
> > > > > > > > > > with one internal topic, not too.
> > > > > > > > > >
> > > > > > > > > > This proposal offers better operational simplicity and
> > fewer
> > > > > > internal
> > > > > > > > > > topics but there are some downsides that come with it —
> > there
> > > > > are 2
> > > > > > > > types
> > > > > > > > > > of messages in one topic (txn metadata ones and offset
> > ones).
> > > > > Since
> > > > > > > > this
> > > > > > > > > > internal topic serves a dual purpose, it will be harder
> to
> > > name
> > > > > it
> > > > > > > and
> > > > > > > > > also
> > > > > > > > > > design a message format that includes the different types
> > of
> > > > > > messages
> > > > > > > > > that
> > > > > > > > > > will live in the topic. Though the transaction topic
> > already
> > > > > needs
> > > > > > to
> > > > > > > > > write
> > > > > > > > > > 5 different types of messages (the AppID->PID mapping,
> the
> > > > > BeginTxn
> > > > > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so
> > > maybe
> > > > > > adding
> > > > > > > > the
> > > > > > > > > > offset message isn't a big deal?
> > > > > > > > > >
> > > > > > > > > > Back when we introduced the offsets topic, we had
> discussed
> > > > > making
> > > > > > it
> > > > > > > > > more
> > > > > > > > > > general and allowing the producer to send offset commit
> > > > messages
> > > > > to
> > > > > > > it
> > > > > > > > > but
> > > > > > > > > > ended up creating a specialized topic to allow the
> consumer
> > > > > > > coordinator
> > > > > > > > > to
> > > > > > > > > > wall off and prevent unauthorized writes from consumers
> > > outside
> > > > > of
> > > > > > a
> > > > > > > > > group.
> > > > > > > > > > Jason can comment on the specifics but I don't believe
> that
> > > > goal
> > > > > of
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > consumer protocol was quite achieved.
> > > > > > > > > >
> > > > > > > > > > I have other comments on the message format, request
> names
> > > etc
> > > > > but
> > > > > > > > wanted
> > > > > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > > > > >
> > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi all,
> > > > > > > > > > >
> > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> exactly
> > > once
> > > > > > > > delivery
> > > > > > > > > > > semantics:
> > > > > > > > > > >
> > > > > > > > > > > *
> > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > <
> > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > > >*
> > > > > > > > > > >
> > > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > > with
> > > > an
> > > > > > > > > > idempotent
> > > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > > messages
> > > > > > > sent
> > > > > > > > > > from
> > > > > > > > > > > the same identified producer can be detected on the
> > broker
> > > > > side,
> > > > > > > and
> > > > > > > > > 2) a
> > > > > > > > > > > group of messages sent within a transaction will
> > atomically
> > > > be
> > > > > > > either
> > > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > > > > > >
> > > > > > > > > > > The above wiki page provides a high-level view of the
> > > > proposed
> > > > > > > > changes
> > > > > > > > > as
> > > > > > > > > > > well as summarized guarantees. Initial draft of the
> > > detailed
> > > > > > > > > > implementation
> > > > > > > > > > > design is described in this Google doc:
> > > > > > > > > > >
> > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > > 0wSw9ra8
> > > > > > > > > > > <https://docs.google.com/document/d/11Jqy_
> > > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > 0wSw9ra8>
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > -- Guozhang
> > > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > Thanks,
> > > > > > > > > > Neha
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
@Henry Cai,

I am working on a separate KIP on Streams to leverage this KIP to have
exactly-once processing semantics (note the exactly-once processing is a
bit different from exactly-once delivery semantics), which should cover
your question.

The short answer is that writing the changelog messages need to be part of
the transaction, and when a fatal error happens within a transaction, since
the store updates cannot be rolled back like the messages in the worst case
we need to restore from the changelog from scratch, or from a checkpoint
with a starting offset in changelog, and restoring consumer will fetch
committed messages only as well.


Guozhang

On Thu, Dec 1, 2016 at 9:34 AM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Daniel,
>
> That is a very good point. You are correct in saying that one does not need
> a transaction coordinator to get idempotent semantics.
>
> There are, however, three reasons why we chose this route:
>
>    1. The request to find a transaction coordinator is exactly the same as
>    the request consumers use to find the group coordinator. So if clients
>    already implement the new consumer, you should already have the code you
>    need to find the transaction coordinator. I would even so far as to say
>    that the majority coordinator discovery code can be effectively shared
>    between producers and consumers. Jason should correct me on this,
> however,
>    since he is most familiar with that bit.
>    2. With this route, the broker side changes are simpler. In particular,
>    we have to implement the InitPIDRequest only in the coordinator.
>    3. By always having a transaction coordinator, we can enable
>    applications to use transactions even if they don't specify the AppId.
> The
>    only thing you lose is transaction recovery across sessions.
>
> Needless to say, we did debate this point extensively. What swung our
> decision ultimately was the following observation: if the user does not
> provide a transaction.app.id, the client can generate a UUID and use that
> as the appId for the rest of the session. This means that there are no
> branches in the client and server code, and is overall simpler to maintain.
> All the producer APIs are also available to the user and it would be more
> intuitive.
>
> It also means that clients cannot choose idempotence without transactions,
> and hence it does place a greater burden on implementors of kafka clients.
> But the cost should be minimal given point 1 above, and was deemed worth
> it.
>
> Thanks once more for your thoughtful comments. It would be great for other
> client implementors to chime in on this.
>
> Regards,
> Apurva
>
>
> On Thu, Dec 1, 2016 at 3:16 AM, Daniel Schierbeck
> <dasch@zendesk.com.invalid
> > wrote:
>
> > Hi there,
> >
> > I'm the author of ruby-kafka, and as such am slightly biased towards
> > simplicity of implementation :-)
> >
> > I like the proposal, and would love to use idempotent producer semantics
> in
> > our projects at Zendesk, but I'm a bit worried about the complexity that
> > would go into the clients; specifically: it sounds to me that in order to
> > get idempotent producer semantics, I'd have to implement the transaction
> > coordinator discovery. I may be wrong, but it would seem that it's not
> > strictly necessary if you're not using transactions – we could just use
> the
> > topic partition's leader as the coordinator, avoiding the extra
> discovery.
> > In my experience, most bugs are related to figuring out which broker is
> the
> > leader of which partition/group/whatever, so minimizing the number of
> > moving parts would be beneficial to me. I'd also like to point out that I
> > would be reluctant to implement the transaction API in the near future,
> but
> > would love to implement the idempotency API soon. The former seems only
> > relevant to real stream processing frameworks, which is probably not the
> > best use case for ruby-kafka.
> >
> > Cheers,
> > Daniel Schierbeck
> >
> > On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io>
> wrote:
> >
> > > Hey Neha,
> > >
> > > Thanks for the thoughtful questions. I'll try to address the first
> > question
> > > since Apurva addressed the second. Since most readers are probably
> > getting
> > > up to speed with this large proposal, let me first take a step back and
> > > explain why we need the AppID at all. As Confluent tradition demands, I
> > > present you a big wall of text:
> > >
> > > Clearly "exactly once" delivery requires resilience to client failures.
> > > When a client crashes or turns into a zombie, another client must
> > > eventually be started to resume the work. There are two problems: 1) we
> > > need to ensure that the old process is actually dead or at least that
> it
> > > cannot write any more data, and 2) we need to be able to pick up
> wherever
> > > the last process left off. To do either of these, we need some kind of
> > > identifier to tie the two instances together.
> > >
> > > There are only two choices for where this ID comes from: either the
> user
> > > gives it to us or the server generates it. In the latter case, the user
> > is
> > > responsible for fetching it from the client and persisting it somewhere
> > for
> > > use after failure. We ultimately felt that the most flexible option is
> to
> > > have the user give it to us. In many applications, there is already a
> > > natural identifier which is already used to divide the workload. For
> > > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > > applications where there is no natural ID, the user can generate a UUID
> > and
> > > persist it locally, which is as good as having the server generate it.
> > >
> > > So the AppID is used to provide continuity between the instances of a
> > > producer which are handling a certain workload. One of the early design
> > > decisions we made in this work was to make the delivery guarantees we
> > > provide agnostic of the workload that the producer is assigned. The
> > > producer is not in the business of trying to divide up the work among
> all
> > > its peers who are participating in the same duty (unlike the consumer,
> we
> > > don't know anything about where the data comes from). This has huge
> > > implications for "exactly-once" delivery because it puts the burden on
> > the
> > > user to divide the total workload among producer instances and to
> assign
> > > AppIDs accordingly.
> > >
> > > I've been using the term "workload" loosely, but we usually imagine
> > > something like Kafka Connect's notion of a "source partition." A source
> > > partition could be a topic partition if the source is Kafka, or it
> could
> > be
> > > a database table, a log file, or whatever makes sense for the source of
> > the
> > > data. The point is that it's an independent source of data which can be
> > > assigned to a producer instance.
> > >
> > > If the same source partition is always assigned to the producer with
> the
> > > the same AppID, then Kafka transactions will give you "exactly once"
> > > delivery without much additional work. On initialization, the producer
> > will
> > > ensure that 1) any previous producers using that AppID are "fenced"
> off,
> > > and 2) that any transaction which had been started by a previous
> producer
> > > with that AppID have either completed or aborted.
> > >
> > > Based on this, it should be clear that the ideal is to divide the
> > workload
> > > so that you have a one-to-one mapping from the source partition to the
> > > AppID. If the source of the data is Kafka, then the source partition is
> > > just a topic partition, and the AppID can be generated from the name of
> > the
> > > topic and the partition number.
> > >
> > > To finally get back to your auto-scaling question, let's assume for a
> > > moment the ideal mapping of source partition to AppID. The main
> question
> > is
> > > whether the scaling is "horizontal" or "vertical." By horizontal, I
> mean
> > an
> > > increase in the number of source partitions. This case is easy. Assign
> > new
> > > AppIDs based on the new source partitions and you're done.
> > >
> > > But if the scaling is vertical (i.e. an increase in the load on the
> > source
> > > partitions), there's not much this proposal can do to help. You're
> going
> > to
> > > have to break the source partition into child partitions, and assign
> each
> > > of the new partitions a new AppID. To preserve "exactly once" delivery,
> > you
> > > must make sure that the producers using the AppID assigned to the
> parent
> > > partition have been shutdown cleanly. We could provide a way to pass
> in a
> > > "parent AppID" so that the producer could check the appropriate safety
> > > conditions, but for the first version, we assume that users consider
> > > scaling requirements when dividing the workload into source partitions.
> > >
> > > Unfortunately, the real world is always falling short of the ideal, and
> > > it's not always practical to have a one-to-one mapping of source
> > partition
> > > to AppID, since that also implies a one-to-one mapping of source
> > partition
> > > to producer instance. If I were a user, I'd push this limit as far as
> is
> > > reasonable, but with enough source partitions, it eventually breaks
> down.
> > > At some point, you need a producer to handle the load of more than one
> > > source partition. This is fine in itself if the assignment is sticky:
> > that
> > > is, if we can ensure that the same source partition is assigned to the
> > > producer using a certain AppID. If not, then the user is responsible
> for
> > > ensuring a clean hand-off. The producer reading from the migrating
> source
> > > partition must stop reading, commit or abort any transaction containing
> > > data processed from that source partition, and then signal the producer
> > > which is taking over that it is safe to begin.
> > >
> > > This burden is a consequence of the decision to keep the producer out
> of
> > > the role of assigning work. We could do more if we forced users to
> > > formalize their application-specific notion of a source partition, and
> if
> > > we turned the producer into something like a consumer group, with a
> > > rebalance protocol. This would allow the broker to be the one to
> ensure a
> > > clean hand-off of work, but it would be a huge departure from the way
> the
> > > producer currently works, and not all applications have a notion of
> > source
> > > partition anyway. So the result is a bit more work for the user, though
> > of
> > > course it would be transparent to for Kafka Streams users.
> > >
> > > One final note. I've described above how to get the strongest
> guarantees
> > > that this work is capable of providing in an auto-scaling environment.
> We
> > > also provide weaker guarantees, which are still an improvement over the
> > > current state. For example, without specifying any kind of AppID, we
> > > provide idempotent production for the lifetime of a producer instance.
> > This
> > > ensures reliable delivery without duplicates even with broker failures.
> > It
> > > is also possible to use transactions without an ephemeral AppID. If the
> > > application generates a UUID for user as the AppID, and only uses it
> for
> > > the lifetime of a single producer, you can still take advantage of
> > > transactional semantics, which allows you to write to a set of messages
> > to
> > > multiple partitions atomically.
> > >
> > > Hope that answers the question and helps others understand the work a
> bit
> > > better!
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > >
> > > On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> > >
> > > > Thanks for your comment, I updated the document. Let me know if it is
> > > clear
> > > > now.
> > > >
> > > > Apurva
> > > >
> > > > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > > > onurkaraman.apache@gmail.com>
> > > > wrote:
> > > >
> > > > > @Apurva yep that's what I was trying to say.
> > > > >
> > > > > Original message:
> > > > > If there is already an entry with the AppID in the mapping,
> increment
> > > the
> > > > > epoch number and go on to the next step. If there is no entry with
> > the
> > > > > AppID in the mapping, construct a PID with initialized epoch
> number;
> > > > append
> > > > > an AppID message into the transaction topic, insert into the
> mapping
> > > and
> > > > > reply with the PID / epoch / timestamp.
> > > > >
> > > > > Just wanted to make it explicit because:
> > > > > 1. The "append an AppID message..." chunk was ambiguous on whether
> it
> > > > > applied to the "if exists" or "if not exists" condition
> > > > > 2. I think the google doc is pretty explicit on appending to the
> log
> > > > > everywhere else.
> > > > >
> > > > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <apurva@confluent.io
> >
> > > > wrote:
> > > > >
> > > > > > The first line in step 2 of that section is: "If there is already
> > an
> > > > > entry
> > > > > > with the AppID in the mapping, increment the epoch number and go
> on
> > > to
> > > > > the
> > > > > > next step."
> > > > > >
> > > > > > Are you suggesting that it be made explicit that 'increment the
> > epoch
> > > > > > number' includes persisting the updated value to the log?
> > > > > >
> > > > > > Thanks,
> > > > > > Apurva
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > > > onurkaraman.apache@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Nice google doc!
> > > > > > >
> > > > > > > Probably need to go over the google doc a few more times, but a
> > > minor
> > > > > > > comment from the first pass:
> > > > > > >
> > > > > > > In Transaction Coordinator Request Handling (
> > > > > > > https://docs.google.com/document/d/11Jqy_
> > > <https://docs.google.com/document/d/11Jqy_>
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > > > step 2 mentions that if the Transaction Coordinator doesn't
> > already
> > > > > see a
> > > > > > > producer with the same app-id, it creates a pid and appends
> > > (app-id,
> > > > > pid,
> > > > > > > epoch) into the transaction log.
> > > > > > >
> > > > > > > What about if the app-id/pid pair already exists and we
> increment
> > > the
> > > > > > > epoch? Should we append (app-id, pid, epoch++) to the
> transaction
> > > > log?
> > > > > I
> > > > > > > think we should, but step 2 doesn't mention this.
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <
> > apurva@confluent.io
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for your comments, let me deal with your second point
> > > > > regarding
> > > > > > > > merging the __consumer-offsets and transactions topic.
> > > > > > > >
> > > > > > > > Needless to say, we considered doing this, but chose to keep
> > them
> > > > > > > separate
> > > > > > > > for the following reasons:
> > > > > > > >
> > > > > > > > 1. Your assumption that group.id and transaction.app.id can
> be
> > > > > the
> > > > > > > same
> > > > > > > > does not hold for streams applications. All colocated tasks
> of
> > a
> > > > > > > streams
> > > > > > > > application will share the same consumer (and hence
> implicitly
> > > > > will
> > > > > > > have
> > > > > > > > the same group.id), but each task will have its own producer
> > > > > > > instance.
> > > > > > > > The transaction.app.id for each producer instance will still
> > > > have
> > > > > > to
> > > > > > > be
> > > > > > > > distinct. So to colocate the transaction and consumer group
> > > > > > > > coordinators,
> > > > > > > > we will have to now introduce a 'group.id' config in the
> > > > producer
> > > > > > and
> > > > > > > > require it to be the same as the consumer. This seemed like a
> > > > very
> > > > > > > > fragile
> > > > > > > > option.
> > > > > > > > 2. Following on from the above, the transaction coordinator
> and
> > > > > > group
> > > > > > > > coordinator would _have_ to be colocated inorder to be the
> > > > leader
> > > > > > for
> > > > > > > > the
> > > > > > > > same TopicPartition, unless we wanted to make even more
> > > > > fundamental
> > > > > > > > changes
> > > > > > > > to Kafka.
> > > > > > > > 3. We don't require that the consumer coordinator and the
> > > > > > transaction
> > > > > > > > coordinator have the same view of the current PID/Epoch pair.
> > > > If a
> > > > > > > > producer
> > > > > > > > instance is bounced, the epoch will be bumped. Any
> transactions
> > > > > > > > initiated
> > > > > > > > by the previous instance would either be fully committed or
> > > > fully
> > > > > > > rolled
> > > > > > > > back. Since the writes to the offset topics are just like
> > writes
> > > > > to
> > > > > > a
> > > > > > > > regular topic, these would enjoy the same guarantees, and the
> > > > > > > > inconsistency
> > > > > > > > will be eventually resolved.
> > > > > > > > 4. Finally, every application will have consumers, and hence
> > > > > record
> > > > > > > > consumer offsets. But a very small fraction of applications
> > > > would
> > > > > > use
> > > > > > > > transactions. Blending the two topics would make recovering
> > > > > > > transaction
> > > > > > > > coordinator state unnecessarily inefficient since it has to
> > read
> > > > > > from
> > > > > > > > the
> > > > > > > > beginning of the topic to reconstruct its data structures --
> it
> > > > > > would
> > > > > > > > have
> > > > > > > > to inspect and skip a majority of the messages if the offsets
> > > > were
> > > > > > in
> > > > > > > > the
> > > > > > > > same topic.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Apurva
> > > > > > > >
> > > > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> > > neha@confluent.io>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for initiating this KIP! I think it is well written
> > and
> > > > I'm
> > > > > > > > excited
> > > > > > > > > to see the first step towards adding an important feature
> in
> > > > Kafka.
> > > > > > > > >
> > > > > > > > > I had a few initial thoughts on the KIP, mostly not as
> deeply
> > > > > thought
> > > > > > > > > through than what you've done -
> > > > > > > > >
> > > > > > > > > 1. Perhaps you’ve thought about how this would work
> already —
> > > > since
> > > > > > we
> > > > > > > > now
> > > > > > > > > require a producer to specify a unique AppID across
> different
> > > > > > instances
> > > > > > > > of
> > > > > > > > > an application, how would applications that run in the
> cloud
> > > use
> > > > > this
> > > > > > > > > feature with auto scaling?
> > > > > > > > >
> > > > > > > > > 2. Making it easy for applications to get exactly-once
> > > semantics
> > > > > for
> > > > > > a
> > > > > > > > > consume-process-produce workflow is a great feature to
> have.
> > To
> > > > > > enable
> > > > > > > > > this, the proposal now includes letting a producer
> initiate a
> > > > write
> > > > > > to
> > > > > > > > the
> > > > > > > > > offset topic as well (just like consumers do). The consumer
> > > > > > coordinator
> > > > > > > > > (which could be on a different broker than the txn
> > coordinator)
> > > > > would
> > > > > > > > then
> > > > > > > > > validate if the PID and producer epoch is valid before it
> > > writes
> > > > to
> > > > > > the
> > > > > > > > > offset topic along with the associated PID. This is a great
> > > > feature
> > > > > > > > though
> > > > > > > > > I see 2 difficulties
> > > > > > > > >
> > > > > > > > > -- This needs the consumer coordinator to have a consistent
> > > view
> > > > of
> > > > > > the
> > > > > > > > > PID/epochs that is same as the view on the txn coordinator.
> > > > > However,
> > > > > > as
> > > > > > > > the
> > > > > > > > > offset and the transaction topics are different, the 2
> > > > coordinators
> > > > > > > might
> > > > > > > > > live on different brokers.
> > > > > > > > > -- We now also have 2 internal topics - a transaction topic
> > and
> > > > the
> > > > > > > > > __consumer_offsets topic.
> > > > > > > > >
> > > > > > > > > Maybe you’ve thought about this already and discarded it
> ...
> > > let
> > > > me
> > > > > > > make
> > > > > > > > a
> > > > > > > > > somewhat crazy proposal — Why don’t we upgrade the
> > transaction
> > > > > topic
> > > > > > to
> > > > > > > > be
> > > > > > > > > the new offsets topic as well? For consumers that want EoS
> > > > > guarantees
> > > > > > > for
> > > > > > > > > a consume-process-produce pattern, the group.id is the
> same
> > as
> > > > the
> > > > > > > > > transaction.app.id set for the producer. Assume that the
> > > > > transaction
> > > > > > > > topic
> > > > > > > > > also stores consumer offsets. It stores both the
> transaction
> > > > > metadata
> > > > > > > > > messages as well as offset messages, both for transactional
> > as
> > > > well
> > > > > > as
> > > > > > > > > non-transactional consumers. Since the group.id of the
> > > consumer
> > > > > and
> > > > > > > the
> > > > > > > > > app.id of the producer is the same, the offsets associated
> > > with
> > > > a
> > > > > > > > consumer
> > > > > > > > > group and topic-partition end up in the same transaction
> > topic
> > > > > > > partition
> > > > > > > > as
> > > > > > > > > the transaction metadata messages. The transaction
> > coordinator
> > > > and
> > > > > > the
> > > > > > > > > consumer coordinator always live on the same broker since
> > they
> > > > both
> > > > > > map
> > > > > > > > to
> > > > > > > > > the same partition in the transaction topic. Even if there
> > are
> > > > > > > failures,
> > > > > > > > > they end up on the same new broker. Hence, they share the
> > same
> > > > and
> > > > > > > > > consistent view of the PIDs, epochs and App IDs, whatever
> it
> > > is.
> > > > > The
> > > > > > > > > consumer coordinator will skip over the transaction
> metadata
> > > > > messages
> > > > > > > > when
> > > > > > > > > it bootstraps the offsets from this new topic for consumer
> > > groups
> > > > > > that
> > > > > > > > are
> > > > > > > > > not involved in a transaction and don’t have a txn id
> > > associated
> > > > > with
> > > > > > > the
> > > > > > > > > offset message in the transaction topic. The consumer
> > > coordinator
> > > > > > will
> > > > > > > > > expose only committed offsets in cases of consumer groups
> > that
> > > > are
> > > > > > > > involved
> > > > > > > > > in a txn. It will also be able to validate the
> > > > OffsetCommitRequests
> > > > > > > > coming
> > > > > > > > > from a transactional producer by ensuring that it is coming
> > > from
> > > > a
> > > > > > > valid
> > > > > > > > > PID, producer epoch since it uses the same view of this
> data
> > > > > created
> > > > > > by
> > > > > > > > the
> > > > > > > > > transaction coordinator (that lives on the same broker).
> And
> > we
> > > > > will
> > > > > > > end
> > > > > > > > up
> > > > > > > > > with one internal topic, not too.
> > > > > > > > >
> > > > > > > > > This proposal offers better operational simplicity and
> fewer
> > > > > internal
> > > > > > > > > topics but there are some downsides that come with it —
> there
> > > > are 2
> > > > > > > types
> > > > > > > > > of messages in one topic (txn metadata ones and offset
> ones).
> > > > Since
> > > > > > > this
> > > > > > > > > internal topic serves a dual purpose, it will be harder to
> > name
> > > > it
> > > > > > and
> > > > > > > > also
> > > > > > > > > design a message format that includes the different types
> of
> > > > > messages
> > > > > > > > that
> > > > > > > > > will live in the topic. Though the transaction topic
> already
> > > > needs
> > > > > to
> > > > > > > > write
> > > > > > > > > 5 different types of messages (the AppID->PID mapping, the
> > > > BeginTxn
> > > > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so
> > maybe
> > > > > adding
> > > > > > > the
> > > > > > > > > offset message isn't a big deal?
> > > > > > > > >
> > > > > > > > > Back when we introduced the offsets topic, we had discussed
> > > > making
> > > > > it
> > > > > > > > more
> > > > > > > > > general and allowing the producer to send offset commit
> > > messages
> > > > to
> > > > > > it
> > > > > > > > but
> > > > > > > > > ended up creating a specialized topic to allow the consumer
> > > > > > coordinator
> > > > > > > > to
> > > > > > > > > wall off and prevent unauthorized writes from consumers
> > outside
> > > > of
> > > > > a
> > > > > > > > group.
> > > > > > > > > Jason can comment on the specifics but I don't believe that
> > > goal
> > > > of
> > > > > > the
> > > > > > > > new
> > > > > > > > > consumer protocol was quite achieved.
> > > > > > > > >
> > > > > > > > > I have other comments on the message format, request names
> > etc
> > > > but
> > > > > > > wanted
> > > > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> > once
> > > > > > > delivery
> > > > > > > > > > semantics:
> > > > > > > > > >
> > > > > > > > > > *
> > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > <
> > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > > >*
> > > > > > > > > >
> > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > with
> > > an
> > > > > > > > > idempotent
> > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > messages
> > > > > > sent
> > > > > > > > > from
> > > > > > > > > > the same identified producer can be detected on the
> broker
> > > > side,
> > > > > > and
> > > > > > > > 2) a
> > > > > > > > > > group of messages sent within a transaction will
> atomically
> > > be
> > > > > > either
> > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > > > > >
> > > > > > > > > > The above wiki page provides a high-level view of the
> > > proposed
> > > > > > > changes
> > > > > > > > as
> > > > > > > > > > well as summarized guarantees. Initial draft of the
> > detailed
> > > > > > > > > implementation
> > > > > > > > > > design is described in this Google doc:
> > > > > > > > > >
> > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > 0wSw9ra8
> > > > > > > > > > <https://docs.google.com/document/d/11Jqy_
> > > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > 0wSw9ra8>
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > -- Guozhang
> > > > > > > > > >
> > > > > > > > > --
> > > > > > > > > Thanks,
> > > > > > > > > Neha
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Daniel,

That is a very good point. You are correct in saying that one does not need
a transaction coordinator to get idempotent semantics.

There are, however, three reasons why we chose this route:

   1. The request to find a transaction coordinator is exactly the same as
   the request consumers use to find the group coordinator. So if clients
   already implement the new consumer, you should already have the code you
   need to find the transaction coordinator. I would even so far as to say
   that the majority coordinator discovery code can be effectively shared
   between producers and consumers. Jason should correct me on this, however,
   since he is most familiar with that bit.
   2. With this route, the broker side changes are simpler. In particular,
   we have to implement the InitPIDRequest only in the coordinator.
   3. By always having a transaction coordinator, we can enable
   applications to use transactions even if they don't specify the AppId. The
   only thing you lose is transaction recovery across sessions.

Needless to say, we did debate this point extensively. What swung our
decision ultimately was the following observation: if the user does not
provide a transaction.app.id, the client can generate a UUID and use that
as the appId for the rest of the session. This means that there are no
branches in the client and server code, and is overall simpler to maintain.
All the producer APIs are also available to the user and it would be more
intuitive.

It also means that clients cannot choose idempotence without transactions,
and hence it does place a greater burden on implementors of kafka clients.
But the cost should be minimal given point 1 above, and was deemed worth it.

Thanks once more for your thoughtful comments. It would be great for other
client implementors to chime in on this.

Regards,
Apurva


On Thu, Dec 1, 2016 at 3:16 AM, Daniel Schierbeck <dasch@zendesk.com.invalid
> wrote:

> Hi there,
>
> I'm the author of ruby-kafka, and as such am slightly biased towards
> simplicity of implementation :-)
>
> I like the proposal, and would love to use idempotent producer semantics in
> our projects at Zendesk, but I'm a bit worried about the complexity that
> would go into the clients; specifically: it sounds to me that in order to
> get idempotent producer semantics, I'd have to implement the transaction
> coordinator discovery. I may be wrong, but it would seem that it's not
> strictly necessary if you're not using transactions – we could just use the
> topic partition's leader as the coordinator, avoiding the extra discovery.
> In my experience, most bugs are related to figuring out which broker is the
> leader of which partition/group/whatever, so minimizing the number of
> moving parts would be beneficial to me. I'd also like to point out that I
> would be reluctant to implement the transaction API in the near future, but
> would love to implement the idempotency API soon. The former seems only
> relevant to real stream processing frameworks, which is probably not the
> best use case for ruby-kafka.
>
> Cheers,
> Daniel Schierbeck
>
> On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io> wrote:
>
> > Hey Neha,
> >
> > Thanks for the thoughtful questions. I'll try to address the first
> question
> > since Apurva addressed the second. Since most readers are probably
> getting
> > up to speed with this large proposal, let me first take a step back and
> > explain why we need the AppID at all. As Confluent tradition demands, I
> > present you a big wall of text:
> >
> > Clearly "exactly once" delivery requires resilience to client failures.
> > When a client crashes or turns into a zombie, another client must
> > eventually be started to resume the work. There are two problems: 1) we
> > need to ensure that the old process is actually dead or at least that it
> > cannot write any more data, and 2) we need to be able to pick up wherever
> > the last process left off. To do either of these, we need some kind of
> > identifier to tie the two instances together.
> >
> > There are only two choices for where this ID comes from: either the user
> > gives it to us or the server generates it. In the latter case, the user
> is
> > responsible for fetching it from the client and persisting it somewhere
> for
> > use after failure. We ultimately felt that the most flexible option is to
> > have the user give it to us. In many applications, there is already a
> > natural identifier which is already used to divide the workload. For
> > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > applications where there is no natural ID, the user can generate a UUID
> and
> > persist it locally, which is as good as having the server generate it.
> >
> > So the AppID is used to provide continuity between the instances of a
> > producer which are handling a certain workload. One of the early design
> > decisions we made in this work was to make the delivery guarantees we
> > provide agnostic of the workload that the producer is assigned. The
> > producer is not in the business of trying to divide up the work among all
> > its peers who are participating in the same duty (unlike the consumer, we
> > don't know anything about where the data comes from). This has huge
> > implications for "exactly-once" delivery because it puts the burden on
> the
> > user to divide the total workload among producer instances and to assign
> > AppIDs accordingly.
> >
> > I've been using the term "workload" loosely, but we usually imagine
> > something like Kafka Connect's notion of a "source partition." A source
> > partition could be a topic partition if the source is Kafka, or it could
> be
> > a database table, a log file, or whatever makes sense for the source of
> the
> > data. The point is that it's an independent source of data which can be
> > assigned to a producer instance.
> >
> > If the same source partition is always assigned to the producer with the
> > the same AppID, then Kafka transactions will give you "exactly once"
> > delivery without much additional work. On initialization, the producer
> will
> > ensure that 1) any previous producers using that AppID are "fenced" off,
> > and 2) that any transaction which had been started by a previous producer
> > with that AppID have either completed or aborted.
> >
> > Based on this, it should be clear that the ideal is to divide the
> workload
> > so that you have a one-to-one mapping from the source partition to the
> > AppID. If the source of the data is Kafka, then the source partition is
> > just a topic partition, and the AppID can be generated from the name of
> the
> > topic and the partition number.
> >
> > To finally get back to your auto-scaling question, let's assume for a
> > moment the ideal mapping of source partition to AppID. The main question
> is
> > whether the scaling is "horizontal" or "vertical." By horizontal, I mean
> an
> > increase in the number of source partitions. This case is easy. Assign
> new
> > AppIDs based on the new source partitions and you're done.
> >
> > But if the scaling is vertical (i.e. an increase in the load on the
> source
> > partitions), there's not much this proposal can do to help. You're going
> to
> > have to break the source partition into child partitions, and assign each
> > of the new partitions a new AppID. To preserve "exactly once" delivery,
> you
> > must make sure that the producers using the AppID assigned to the parent
> > partition have been shutdown cleanly. We could provide a way to pass in a
> > "parent AppID" so that the producer could check the appropriate safety
> > conditions, but for the first version, we assume that users consider
> > scaling requirements when dividing the workload into source partitions.
> >
> > Unfortunately, the real world is always falling short of the ideal, and
> > it's not always practical to have a one-to-one mapping of source
> partition
> > to AppID, since that also implies a one-to-one mapping of source
> partition
> > to producer instance. If I were a user, I'd push this limit as far as is
> > reasonable, but with enough source partitions, it eventually breaks down.
> > At some point, you need a producer to handle the load of more than one
> > source partition. This is fine in itself if the assignment is sticky:
> that
> > is, if we can ensure that the same source partition is assigned to the
> > producer using a certain AppID. If not, then the user is responsible for
> > ensuring a clean hand-off. The producer reading from the migrating source
> > partition must stop reading, commit or abort any transaction containing
> > data processed from that source partition, and then signal the producer
> > which is taking over that it is safe to begin.
> >
> > This burden is a consequence of the decision to keep the producer out of
> > the role of assigning work. We could do more if we forced users to
> > formalize their application-specific notion of a source partition, and if
> > we turned the producer into something like a consumer group, with a
> > rebalance protocol. This would allow the broker to be the one to ensure a
> > clean hand-off of work, but it would be a huge departure from the way the
> > producer currently works, and not all applications have a notion of
> source
> > partition anyway. So the result is a bit more work for the user, though
> of
> > course it would be transparent to for Kafka Streams users.
> >
> > One final note. I've described above how to get the strongest guarantees
> > that this work is capable of providing in an auto-scaling environment. We
> > also provide weaker guarantees, which are still an improvement over the
> > current state. For example, without specifying any kind of AppID, we
> > provide idempotent production for the lifetime of a producer instance.
> This
> > ensures reliable delivery without duplicates even with broker failures.
> It
> > is also possible to use transactions without an ephemeral AppID. If the
> > application generates a UUID for user as the AppID, and only uses it for
> > the lifetime of a single producer, you can still take advantage of
> > transactional semantics, which allows you to write to a set of messages
> to
> > multiple partitions atomically.
> >
> > Hope that answers the question and helps others understand the work a bit
> > better!
> >
> > Thanks,
> > Jason
> >
> >
> >
> >
> > On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Thanks for your comment, I updated the document. Let me know if it is
> > clear
> > > now.
> > >
> > > Apurva
> > >
> > > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > > onurkaraman.apache@gmail.com>
> > > wrote:
> > >
> > > > @Apurva yep that's what I was trying to say.
> > > >
> > > > Original message:
> > > > If there is already an entry with the AppID in the mapping, increment
> > the
> > > > epoch number and go on to the next step. If there is no entry with
> the
> > > > AppID in the mapping, construct a PID with initialized epoch number;
> > > append
> > > > an AppID message into the transaction topic, insert into the mapping
> > and
> > > > reply with the PID / epoch / timestamp.
> > > >
> > > > Just wanted to make it explicit because:
> > > > 1. The "append an AppID message..." chunk was ambiguous on whether it
> > > > applied to the "if exists" or "if not exists" condition
> > > > 2. I think the google doc is pretty explicit on appending to the log
> > > > everywhere else.
> > > >
> > > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > > >
> > > > > The first line in step 2 of that section is: "If there is already
> an
> > > > entry
> > > > > with the AppID in the mapping, increment the epoch number and go on
> > to
> > > > the
> > > > > next step."
> > > > >
> > > > > Are you suggesting that it be made explicit that 'increment the
> epoch
> > > > > number' includes persisting the updated value to the log?
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > > onurkaraman.apache@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Nice google doc!
> > > > > >
> > > > > > Probably need to go over the google doc a few more times, but a
> > minor
> > > > > > comment from the first pass:
> > > > > >
> > > > > > In Transaction Coordinator Request Handling (
> > > > > > https://docs.google.com/document/d/11Jqy_
> > <https://docs.google.com/document/d/11Jqy_>
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > > step 2 mentions that if the Transaction Coordinator doesn't
> already
> > > > see a
> > > > > > producer with the same app-id, it creates a pid and appends
> > (app-id,
> > > > pid,
> > > > > > epoch) into the transaction log.
> > > > > >
> > > > > > What about if the app-id/pid pair already exists and we increment
> > the
> > > > > > epoch? Should we append (app-id, pid, epoch++) to the transaction
> > > log?
> > > > I
> > > > > > think we should, but step 2 doesn't mention this.
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <
> apurva@confluent.io
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for your comments, let me deal with your second point
> > > > regarding
> > > > > > > merging the __consumer-offsets and transactions topic.
> > > > > > >
> > > > > > > Needless to say, we considered doing this, but chose to keep
> them
> > > > > > separate
> > > > > > > for the following reasons:
> > > > > > >
> > > > > > > 1. Your assumption that group.id and transaction.app.id can be
> > > > the
> > > > > > same
> > > > > > > does not hold for streams applications. All colocated tasks of
> a
> > > > > > streams
> > > > > > > application will share the same consumer (and hence implicitly
> > > > will
> > > > > > have
> > > > > > > the same group.id), but each task will have its own producer
> > > > > > instance.
> > > > > > > The transaction.app.id for each producer instance will still
> > > have
> > > > > to
> > > > > > be
> > > > > > > distinct. So to colocate the transaction and consumer group
> > > > > > > coordinators,
> > > > > > > we will have to now introduce a 'group.id' config in the
> > > producer
> > > > > and
> > > > > > > require it to be the same as the consumer. This seemed like a
> > > very
> > > > > > > fragile
> > > > > > > option.
> > > > > > > 2. Following on from the above, the transaction coordinator and
> > > > > group
> > > > > > > coordinator would _have_ to be colocated inorder to be the
> > > leader
> > > > > for
> > > > > > > the
> > > > > > > same TopicPartition, unless we wanted to make even more
> > > > fundamental
> > > > > > > changes
> > > > > > > to Kafka.
> > > > > > > 3. We don't require that the consumer coordinator and the
> > > > > transaction
> > > > > > > coordinator have the same view of the current PID/Epoch pair.
> > > If a
> > > > > > > producer
> > > > > > > instance is bounced, the epoch will be bumped. Any transactions
> > > > > > > initiated
> > > > > > > by the previous instance would either be fully committed or
> > > fully
> > > > > > rolled
> > > > > > > back. Since the writes to the offset topics are just like
> writes
> > > > to
> > > > > a
> > > > > > > regular topic, these would enjoy the same guarantees, and the
> > > > > > > inconsistency
> > > > > > > will be eventually resolved.
> > > > > > > 4. Finally, every application will have consumers, and hence
> > > > record
> > > > > > > consumer offsets. But a very small fraction of applications
> > > would
> > > > > use
> > > > > > > transactions. Blending the two topics would make recovering
> > > > > > transaction
> > > > > > > coordinator state unnecessarily inefficient since it has to
> read
> > > > > from
> > > > > > > the
> > > > > > > beginning of the topic to reconstruct its data structures -- it
> > > > > would
> > > > > > > have
> > > > > > > to inspect and skip a majority of the messages if the offsets
> > > were
> > > > > in
> > > > > > > the
> > > > > > > same topic.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Apurva
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> > neha@confluent.io>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for initiating this KIP! I think it is well written
> and
> > > I'm
> > > > > > > excited
> > > > > > > > to see the first step towards adding an important feature in
> > > Kafka.
> > > > > > > >
> > > > > > > > I had a few initial thoughts on the KIP, mostly not as deeply
> > > > thought
> > > > > > > > through than what you've done -
> > > > > > > >
> > > > > > > > 1. Perhaps you’ve thought about how this would work already —
> > > since
> > > > > we
> > > > > > > now
> > > > > > > > require a producer to specify a unique AppID across different
> > > > > instances
> > > > > > > of
> > > > > > > > an application, how would applications that run in the cloud
> > use
> > > > this
> > > > > > > > feature with auto scaling?
> > > > > > > >
> > > > > > > > 2. Making it easy for applications to get exactly-once
> > semantics
> > > > for
> > > > > a
> > > > > > > > consume-process-produce workflow is a great feature to have.
> To
> > > > > enable
> > > > > > > > this, the proposal now includes letting a producer initiate a
> > > write
> > > > > to
> > > > > > > the
> > > > > > > > offset topic as well (just like consumers do). The consumer
> > > > > coordinator
> > > > > > > > (which could be on a different broker than the txn
> coordinator)
> > > > would
> > > > > > > then
> > > > > > > > validate if the PID and producer epoch is valid before it
> > writes
> > > to
> > > > > the
> > > > > > > > offset topic along with the associated PID. This is a great
> > > feature
> > > > > > > though
> > > > > > > > I see 2 difficulties
> > > > > > > >
> > > > > > > > -- This needs the consumer coordinator to have a consistent
> > view
> > > of
> > > > > the
> > > > > > > > PID/epochs that is same as the view on the txn coordinator.
> > > > However,
> > > > > as
> > > > > > > the
> > > > > > > > offset and the transaction topics are different, the 2
> > > coordinators
> > > > > > might
> > > > > > > > live on different brokers.
> > > > > > > > -- We now also have 2 internal topics - a transaction topic
> and
> > > the
> > > > > > > > __consumer_offsets topic.
> > > > > > > >
> > > > > > > > Maybe you’ve thought about this already and discarded it ...
> > let
> > > me
> > > > > > make
> > > > > > > a
> > > > > > > > somewhat crazy proposal — Why don’t we upgrade the
> transaction
> > > > topic
> > > > > to
> > > > > > > be
> > > > > > > > the new offsets topic as well? For consumers that want EoS
> > > > guarantees
> > > > > > for
> > > > > > > > a consume-process-produce pattern, the group.id is the same
> as
> > > the
> > > > > > > > transaction.app.id set for the producer. Assume that the
> > > > transaction
> > > > > > > topic
> > > > > > > > also stores consumer offsets. It stores both the transaction
> > > > metadata
> > > > > > > > messages as well as offset messages, both for transactional
> as
> > > well
> > > > > as
> > > > > > > > non-transactional consumers. Since the group.id of the
> > consumer
> > > > and
> > > > > > the
> > > > > > > > app.id of the producer is the same, the offsets associated
> > with
> > > a
> > > > > > > consumer
> > > > > > > > group and topic-partition end up in the same transaction
> topic
> > > > > > partition
> > > > > > > as
> > > > > > > > the transaction metadata messages. The transaction
> coordinator
> > > and
> > > > > the
> > > > > > > > consumer coordinator always live on the same broker since
> they
> > > both
> > > > > map
> > > > > > > to
> > > > > > > > the same partition in the transaction topic. Even if there
> are
> > > > > > failures,
> > > > > > > > they end up on the same new broker. Hence, they share the
> same
> > > and
> > > > > > > > consistent view of the PIDs, epochs and App IDs, whatever it
> > is.
> > > > The
> > > > > > > > consumer coordinator will skip over the transaction metadata
> > > > messages
> > > > > > > when
> > > > > > > > it bootstraps the offsets from this new topic for consumer
> > groups
> > > > > that
> > > > > > > are
> > > > > > > > not involved in a transaction and don’t have a txn id
> > associated
> > > > with
> > > > > > the
> > > > > > > > offset message in the transaction topic. The consumer
> > coordinator
> > > > > will
> > > > > > > > expose only committed offsets in cases of consumer groups
> that
> > > are
> > > > > > > involved
> > > > > > > > in a txn. It will also be able to validate the
> > > OffsetCommitRequests
> > > > > > > coming
> > > > > > > > from a transactional producer by ensuring that it is coming
> > from
> > > a
> > > > > > valid
> > > > > > > > PID, producer epoch since it uses the same view of this data
> > > > created
> > > > > by
> > > > > > > the
> > > > > > > > transaction coordinator (that lives on the same broker). And
> we
> > > > will
> > > > > > end
> > > > > > > up
> > > > > > > > with one internal topic, not too.
> > > > > > > >
> > > > > > > > This proposal offers better operational simplicity and fewer
> > > > internal
> > > > > > > > topics but there are some downsides that come with it — there
> > > are 2
> > > > > > types
> > > > > > > > of messages in one topic (txn metadata ones and offset ones).
> > > Since
> > > > > > this
> > > > > > > > internal topic serves a dual purpose, it will be harder to
> name
> > > it
> > > > > and
> > > > > > > also
> > > > > > > > design a message format that includes the different types of
> > > > messages
> > > > > > > that
> > > > > > > > will live in the topic. Though the transaction topic already
> > > needs
> > > > to
> > > > > > > write
> > > > > > > > 5 different types of messages (the AppID->PID mapping, the
> > > BeginTxn
> > > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so
> maybe
> > > > adding
> > > > > > the
> > > > > > > > offset message isn't a big deal?
> > > > > > > >
> > > > > > > > Back when we introduced the offsets topic, we had discussed
> > > making
> > > > it
> > > > > > > more
> > > > > > > > general and allowing the producer to send offset commit
> > messages
> > > to
> > > > > it
> > > > > > > but
> > > > > > > > ended up creating a specialized topic to allow the consumer
> > > > > coordinator
> > > > > > > to
> > > > > > > > wall off and prevent unauthorized writes from consumers
> outside
> > > of
> > > > a
> > > > > > > group.
> > > > > > > > Jason can comment on the specifics but I don't believe that
> > goal
> > > of
> > > > > the
> > > > > > > new
> > > > > > > > consumer protocol was quite achieved.
> > > > > > > >
> > > > > > > > I have other comments on the message format, request names
> etc
> > > but
> > > > > > wanted
> > > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > > >
> > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
> > > > > > delivery
> > > > > > > > > semantics:
> > > > > > > > >
> > > > > > > > > *
> > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > <
> > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > > >*
> > > > > > > > >
> > > > > > > > > This KIP adds a transactional messaging mechanism along
> with
> > an
> > > > > > > > idempotent
> > > > > > > > > producer implementation to make sure that 1) duplicated
> > > messages
> > > > > sent
> > > > > > > > from
> > > > > > > > > the same identified producer can be detected on the broker
> > > side,
> > > > > and
> > > > > > > 2) a
> > > > > > > > > group of messages sent within a transaction will atomically
> > be
> > > > > either
> > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > > > >
> > > > > > > > > The above wiki page provides a high-level view of the
> > proposed
> > > > > > changes
> > > > > > > as
> > > > > > > > > well as summarized guarantees. Initial draft of the
> detailed
> > > > > > > > implementation
> > > > > > > > > design is described in this Google doc:
> > > > > > > > >
> > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > 0wSw9ra8
> > > > > > > > > <https://docs.google.com/document/d/11Jqy_
> > <https://docs.google.com/document/d/11Jqy_>
> > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > 0wSw9ra8>
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > -- Guozhang
> > > > > > > > >
> > > > > > > > --
> > > > > > > > Thanks,
> > > > > > > > Neha
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Daniel Schierbeck <da...@zendesk.com.INVALID>.
Hi there,

I'm the author of ruby-kafka, and as such am slightly biased towards
simplicity of implementation :-)

I like the proposal, and would love to use idempotent producer semantics in
our projects at Zendesk, but I'm a bit worried about the complexity that
would go into the clients; specifically: it sounds to me that in order to
get idempotent producer semantics, I'd have to implement the transaction
coordinator discovery. I may be wrong, but it would seem that it's not
strictly necessary if you're not using transactions – we could just use the
topic partition's leader as the coordinator, avoiding the extra discovery.
In my experience, most bugs are related to figuring out which broker is the
leader of which partition/group/whatever, so minimizing the number of
moving parts would be beneficial to me. I'd also like to point out that I
would be reluctant to implement the transaction API in the near future, but
would love to implement the idempotency API soon. The former seems only
relevant to real stream processing frameworks, which is probably not the
best use case for ruby-kafka.

Cheers,
Daniel Schierbeck

On Thu, Dec 1, 2016 at 9:54 AM Jason Gustafson <ja...@confluent.io> wrote:

> Hey Neha,
>
> Thanks for the thoughtful questions. I'll try to address the first question
> since Apurva addressed the second. Since most readers are probably getting
> up to speed with this large proposal, let me first take a step back and
> explain why we need the AppID at all. As Confluent tradition demands, I
> present you a big wall of text:
>
> Clearly "exactly once" delivery requires resilience to client failures.
> When a client crashes or turns into a zombie, another client must
> eventually be started to resume the work. There are two problems: 1) we
> need to ensure that the old process is actually dead or at least that it
> cannot write any more data, and 2) we need to be able to pick up wherever
> the last process left off. To do either of these, we need some kind of
> identifier to tie the two instances together.
>
> There are only two choices for where this ID comes from: either the user
> gives it to us or the server generates it. In the latter case, the user is
> responsible for fetching it from the client and persisting it somewhere for
> use after failure. We ultimately felt that the most flexible option is to
> have the user give it to us. In many applications, there is already a
> natural identifier which is already used to divide the workload. For
> example, in Kafka Streams and Kafka Connect, we have a taskId. For
> applications where there is no natural ID, the user can generate a UUID and
> persist it locally, which is as good as having the server generate it.
>
> So the AppID is used to provide continuity between the instances of a
> producer which are handling a certain workload. One of the early design
> decisions we made in this work was to make the delivery guarantees we
> provide agnostic of the workload that the producer is assigned. The
> producer is not in the business of trying to divide up the work among all
> its peers who are participating in the same duty (unlike the consumer, we
> don't know anything about where the data comes from). This has huge
> implications for "exactly-once" delivery because it puts the burden on the
> user to divide the total workload among producer instances and to assign
> AppIDs accordingly.
>
> I've been using the term "workload" loosely, but we usually imagine
> something like Kafka Connect's notion of a "source partition." A source
> partition could be a topic partition if the source is Kafka, or it could be
> a database table, a log file, or whatever makes sense for the source of the
> data. The point is that it's an independent source of data which can be
> assigned to a producer instance.
>
> If the same source partition is always assigned to the producer with the
> the same AppID, then Kafka transactions will give you "exactly once"
> delivery without much additional work. On initialization, the producer will
> ensure that 1) any previous producers using that AppID are "fenced" off,
> and 2) that any transaction which had been started by a previous producer
> with that AppID have either completed or aborted.
>
> Based on this, it should be clear that the ideal is to divide the workload
> so that you have a one-to-one mapping from the source partition to the
> AppID. If the source of the data is Kafka, then the source partition is
> just a topic partition, and the AppID can be generated from the name of the
> topic and the partition number.
>
> To finally get back to your auto-scaling question, let's assume for a
> moment the ideal mapping of source partition to AppID. The main question is
> whether the scaling is "horizontal" or "vertical." By horizontal, I mean an
> increase in the number of source partitions. This case is easy. Assign new
> AppIDs based on the new source partitions and you're done.
>
> But if the scaling is vertical (i.e. an increase in the load on the source
> partitions), there's not much this proposal can do to help. You're going to
> have to break the source partition into child partitions, and assign each
> of the new partitions a new AppID. To preserve "exactly once" delivery, you
> must make sure that the producers using the AppID assigned to the parent
> partition have been shutdown cleanly. We could provide a way to pass in a
> "parent AppID" so that the producer could check the appropriate safety
> conditions, but for the first version, we assume that users consider
> scaling requirements when dividing the workload into source partitions.
>
> Unfortunately, the real world is always falling short of the ideal, and
> it's not always practical to have a one-to-one mapping of source partition
> to AppID, since that also implies a one-to-one mapping of source partition
> to producer instance. If I were a user, I'd push this limit as far as is
> reasonable, but with enough source partitions, it eventually breaks down.
> At some point, you need a producer to handle the load of more than one
> source partition. This is fine in itself if the assignment is sticky: that
> is, if we can ensure that the same source partition is assigned to the
> producer using a certain AppID. If not, then the user is responsible for
> ensuring a clean hand-off. The producer reading from the migrating source
> partition must stop reading, commit or abort any transaction containing
> data processed from that source partition, and then signal the producer
> which is taking over that it is safe to begin.
>
> This burden is a consequence of the decision to keep the producer out of
> the role of assigning work. We could do more if we forced users to
> formalize their application-specific notion of a source partition, and if
> we turned the producer into something like a consumer group, with a
> rebalance protocol. This would allow the broker to be the one to ensure a
> clean hand-off of work, but it would be a huge departure from the way the
> producer currently works, and not all applications have a notion of source
> partition anyway. So the result is a bit more work for the user, though of
> course it would be transparent to for Kafka Streams users.
>
> One final note. I've described above how to get the strongest guarantees
> that this work is capable of providing in an auto-scaling environment. We
> also provide weaker guarantees, which are still an improvement over the
> current state. For example, without specifying any kind of AppID, we
> provide idempotent production for the lifetime of a producer instance. This
> ensures reliable delivery without duplicates even with broker failures. It
> is also possible to use transactions without an ephemeral AppID. If the
> application generates a UUID for user as the AppID, and only uses it for
> the lifetime of a single producer, you can still take advantage of
> transactional semantics, which allows you to write to a set of messages to
> multiple partitions atomically.
>
> Hope that answers the question and helps others understand the work a bit
> better!
>
> Thanks,
> Jason
>
>
>
>
> On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Thanks for your comment, I updated the document. Let me know if it is
> clear
> > now.
> >
> > Apurva
> >
> > On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> > onurkaraman.apache@gmail.com>
> > wrote:
> >
> > > @Apurva yep that's what I was trying to say.
> > >
> > > Original message:
> > > If there is already an entry with the AppID in the mapping, increment
> the
> > > epoch number and go on to the next step. If there is no entry with the
> > > AppID in the mapping, construct a PID with initialized epoch number;
> > append
> > > an AppID message into the transaction topic, insert into the mapping
> and
> > > reply with the PID / epoch / timestamp.
> > >
> > > Just wanted to make it explicit because:
> > > 1. The "append an AppID message..." chunk was ambiguous on whether it
> > > applied to the "if exists" or "if not exists" condition
> > > 2. I think the google doc is pretty explicit on appending to the log
> > > everywhere else.
> > >
> > > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> > >
> > > > The first line in step 2 of that section is: "If there is already an
> > > entry
> > > > with the AppID in the mapping, increment the epoch number and go on
> to
> > > the
> > > > next step."
> > > >
> > > > Are you suggesting that it be made explicit that 'increment the epoch
> > > > number' includes persisting the updated value to the log?
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > > onurkaraman.apache@gmail.com>
> > > > wrote:
> > > >
> > > > > Nice google doc!
> > > > >
> > > > > Probably need to go over the google doc a few more times, but a
> minor
> > > > > comment from the first pass:
> > > > >
> > > > > In Transaction Coordinator Request Handling (
> > > > > https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > > step 2 mentions that if the Transaction Coordinator doesn't already
> > > see a
> > > > > producer with the same app-id, it creates a pid and appends
> (app-id,
> > > pid,
> > > > > epoch) into the transaction log.
> > > > >
> > > > > What about if the app-id/pid pair already exists and we increment
> the
> > > > > epoch? Should we append (app-id, pid, epoch++) to the transaction
> > log?
> > > I
> > > > > think we should, but step 2 doesn't mention this.
> > > > >
> > > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <apurva@confluent.io
> >
> > > > wrote:
> > > > >
> > > > > > Thanks for your comments, let me deal with your second point
> > > regarding
> > > > > > merging the __consumer-offsets and transactions topic.
> > > > > >
> > > > > > Needless to say, we considered doing this, but chose to keep them
> > > > > separate
> > > > > > for the following reasons:
> > > > > >
> > > > > > 1. Your assumption that group.id and transaction.app.id can be
> > > the
> > > > > same
> > > > > > does not hold for streams applications. All colocated tasks of a
> > > > > streams
> > > > > > application will share the same consumer (and hence implicitly
> > > will
> > > > > have
> > > > > > the same group.id), but each task will have its own producer
> > > > > instance.
> > > > > > The transaction.app.id for each producer instance will still
> > have
> > > > to
> > > > > be
> > > > > > distinct. So to colocate the transaction and consumer group
> > > > > > coordinators,
> > > > > > we will have to now introduce a 'group.id' config in the
> > producer
> > > > and
> > > > > > require it to be the same as the consumer. This seemed like a
> > very
> > > > > > fragile
> > > > > > option.
> > > > > > 2. Following on from the above, the transaction coordinator and
> > > > group
> > > > > > coordinator would _have_ to be colocated inorder to be the
> > leader
> > > > for
> > > > > > the
> > > > > > same TopicPartition, unless we wanted to make even more
> > > fundamental
> > > > > > changes
> > > > > > to Kafka.
> > > > > > 3. We don't require that the consumer coordinator and the
> > > > transaction
> > > > > > coordinator have the same view of the current PID/Epoch pair.
> > If a
> > > > > > producer
> > > > > > instance is bounced, the epoch will be bumped. Any transactions
> > > > > > initiated
> > > > > > by the previous instance would either be fully committed or
> > fully
> > > > > rolled
> > > > > > back. Since the writes to the offset topics are just like writes
> > > to
> > > > a
> > > > > > regular topic, these would enjoy the same guarantees, and the
> > > > > > inconsistency
> > > > > > will be eventually resolved.
> > > > > > 4. Finally, every application will have consumers, and hence
> > > record
> > > > > > consumer offsets. But a very small fraction of applications
> > would
> > > > use
> > > > > > transactions. Blending the two topics would make recovering
> > > > > transaction
> > > > > > coordinator state unnecessarily inefficient since it has to read
> > > > from
> > > > > > the
> > > > > > beginning of the topic to reconstruct its data structures -- it
> > > > would
> > > > > > have
> > > > > > to inspect and skip a majority of the messages if the offsets
> > were
> > > > in
> > > > > > the
> > > > > > same topic.
> > > > > >
> > > > > > Thanks,
> > > > > > Apurva
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <
> neha@confluent.io>
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for initiating this KIP! I think it is well written and
> > I'm
> > > > > > excited
> > > > > > > to see the first step towards adding an important feature in
> > Kafka.
> > > > > > >
> > > > > > > I had a few initial thoughts on the KIP, mostly not as deeply
> > > thought
> > > > > > > through than what you've done -
> > > > > > >
> > > > > > > 1. Perhaps you’ve thought about how this would work already —
> > since
> > > > we
> > > > > > now
> > > > > > > require a producer to specify a unique AppID across different
> > > > instances
> > > > > > of
> > > > > > > an application, how would applications that run in the cloud
> use
> > > this
> > > > > > > feature with auto scaling?
> > > > > > >
> > > > > > > 2. Making it easy for applications to get exactly-once
> semantics
> > > for
> > > > a
> > > > > > > consume-process-produce workflow is a great feature to have. To
> > > > enable
> > > > > > > this, the proposal now includes letting a producer initiate a
> > write
> > > > to
> > > > > > the
> > > > > > > offset topic as well (just like consumers do). The consumer
> > > > coordinator
> > > > > > > (which could be on a different broker than the txn coordinator)
> > > would
> > > > > > then
> > > > > > > validate if the PID and producer epoch is valid before it
> writes
> > to
> > > > the
> > > > > > > offset topic along with the associated PID. This is a great
> > feature
> > > > > > though
> > > > > > > I see 2 difficulties
> > > > > > >
> > > > > > > -- This needs the consumer coordinator to have a consistent
> view
> > of
> > > > the
> > > > > > > PID/epochs that is same as the view on the txn coordinator.
> > > However,
> > > > as
> > > > > > the
> > > > > > > offset and the transaction topics are different, the 2
> > coordinators
> > > > > might
> > > > > > > live on different brokers.
> > > > > > > -- We now also have 2 internal topics - a transaction topic and
> > the
> > > > > > > __consumer_offsets topic.
> > > > > > >
> > > > > > > Maybe you’ve thought about this already and discarded it ...
> let
> > me
> > > > > make
> > > > > > a
> > > > > > > somewhat crazy proposal — Why don’t we upgrade the transaction
> > > topic
> > > > to
> > > > > > be
> > > > > > > the new offsets topic as well? For consumers that want EoS
> > > guarantees
> > > > > for
> > > > > > > a consume-process-produce pattern, the group.id is the same as
> > the
> > > > > > > transaction.app.id set for the producer. Assume that the
> > > transaction
> > > > > > topic
> > > > > > > also stores consumer offsets. It stores both the transaction
> > > metadata
> > > > > > > messages as well as offset messages, both for transactional as
> > well
> > > > as
> > > > > > > non-transactional consumers. Since the group.id of the
> consumer
> > > and
> > > > > the
> > > > > > > app.id of the producer is the same, the offsets associated
> with
> > a
> > > > > > consumer
> > > > > > > group and topic-partition end up in the same transaction topic
> > > > > partition
> > > > > > as
> > > > > > > the transaction metadata messages. The transaction coordinator
> > and
> > > > the
> > > > > > > consumer coordinator always live on the same broker since they
> > both
> > > > map
> > > > > > to
> > > > > > > the same partition in the transaction topic. Even if there are
> > > > > failures,
> > > > > > > they end up on the same new broker. Hence, they share the same
> > and
> > > > > > > consistent view of the PIDs, epochs and App IDs, whatever it
> is.
> > > The
> > > > > > > consumer coordinator will skip over the transaction metadata
> > > messages
> > > > > > when
> > > > > > > it bootstraps the offsets from this new topic for consumer
> groups
> > > > that
> > > > > > are
> > > > > > > not involved in a transaction and don’t have a txn id
> associated
> > > with
> > > > > the
> > > > > > > offset message in the transaction topic. The consumer
> coordinator
> > > > will
> > > > > > > expose only committed offsets in cases of consumer groups that
> > are
> > > > > > involved
> > > > > > > in a txn. It will also be able to validate the
> > OffsetCommitRequests
> > > > > > coming
> > > > > > > from a transactional producer by ensuring that it is coming
> from
> > a
> > > > > valid
> > > > > > > PID, producer epoch since it uses the same view of this data
> > > created
> > > > by
> > > > > > the
> > > > > > > transaction coordinator (that lives on the same broker). And we
> > > will
> > > > > end
> > > > > > up
> > > > > > > with one internal topic, not too.
> > > > > > >
> > > > > > > This proposal offers better operational simplicity and fewer
> > > internal
> > > > > > > topics but there are some downsides that come with it — there
> > are 2
> > > > > types
> > > > > > > of messages in one topic (txn metadata ones and offset ones).
> > Since
> > > > > this
> > > > > > > internal topic serves a dual purpose, it will be harder to name
> > it
> > > > and
> > > > > > also
> > > > > > > design a message format that includes the different types of
> > > messages
> > > > > > that
> > > > > > > will live in the topic. Though the transaction topic already
> > needs
> > > to
> > > > > > write
> > > > > > > 5 different types of messages (the AppID->PID mapping, the
> > BeginTxn
> > > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe
> > > adding
> > > > > the
> > > > > > > offset message isn't a big deal?
> > > > > > >
> > > > > > > Back when we introduced the offsets topic, we had discussed
> > making
> > > it
> > > > > > more
> > > > > > > general and allowing the producer to send offset commit
> messages
> > to
> > > > it
> > > > > > but
> > > > > > > ended up creating a specialized topic to allow the consumer
> > > > coordinator
> > > > > > to
> > > > > > > wall off and prevent unauthorized writes from consumers outside
> > of
> > > a
> > > > > > group.
> > > > > > > Jason can comment on the specifics but I don't believe that
> goal
> > of
> > > > the
> > > > > > new
> > > > > > > consumer protocol was quite achieved.
> > > > > > >
> > > > > > > I have other comments on the message format, request names etc
> > but
> > > > > wanted
> > > > > > > to get your thoughts on these 2 issues first :-)
> > > > > > >
> > > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > > > delivery
> > > > > > > > semantics:
> > > > > > > >
> > > > > > > > *
> > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > <
> > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > > >*
> > > > > > > >
> > > > > > > > This KIP adds a transactional messaging mechanism along with
> an
> > > > > > > idempotent
> > > > > > > > producer implementation to make sure that 1) duplicated
> > messages
> > > > sent
> > > > > > > from
> > > > > > > > the same identified producer can be detected on the broker
> > side,
> > > > and
> > > > > > 2) a
> > > > > > > > group of messages sent within a transaction will atomically
> be
> > > > either
> > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > > >
> > > > > > > > The above wiki page provides a high-level view of the
> proposed
> > > > > changes
> > > > > > as
> > > > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > > > implementation
> > > > > > > > design is described in this Google doc:
> > > > > > > >
> > > > > > > > https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > 0wSw9ra8
> > > > > > > > <https://docs.google.com/document/d/11Jqy_
> <https://docs.google.com/document/d/11Jqy_>
> > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8>
> > > > > > > >
> > > > > > > >
> > > > > > > > We would love to hear your comments and suggestions.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > -- Guozhang
> > > > > > > >
> > > > > > > --
> > > > > > > Thanks,
> > > > > > > Neha
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Neha,

Thanks for the thoughtful questions. I'll try to address the first question
since Apurva addressed the second. Since most readers are probably getting
up to speed with this large proposal, let me first take a step back and
explain why we need the AppID at all. As Confluent tradition demands, I
present you a big wall of text:

Clearly "exactly once" delivery requires resilience to client failures.
When a client crashes or turns into a zombie, another client must
eventually be started to resume the work. There are two problems: 1) we
need to ensure that the old process is actually dead or at least that it
cannot write any more data, and 2) we need to be able to pick up wherever
the last process left off. To do either of these, we need some kind of
identifier to tie the two instances together.

There are only two choices for where this ID comes from: either the user
gives it to us or the server generates it. In the latter case, the user is
responsible for fetching it from the client and persisting it somewhere for
use after failure. We ultimately felt that the most flexible option is to
have the user give it to us. In many applications, there is already a
natural identifier which is already used to divide the workload. For
example, in Kafka Streams and Kafka Connect, we have a taskId. For
applications where there is no natural ID, the user can generate a UUID and
persist it locally, which is as good as having the server generate it.

So the AppID is used to provide continuity between the instances of a
producer which are handling a certain workload. One of the early design
decisions we made in this work was to make the delivery guarantees we
provide agnostic of the workload that the producer is assigned. The
producer is not in the business of trying to divide up the work among all
its peers who are participating in the same duty (unlike the consumer, we
don't know anything about where the data comes from). This has huge
implications for "exactly-once" delivery because it puts the burden on the
user to divide the total workload among producer instances and to assign
AppIDs accordingly.

I've been using the term "workload" loosely, but we usually imagine
something like Kafka Connect's notion of a "source partition." A source
partition could be a topic partition if the source is Kafka, or it could be
a database table, a log file, or whatever makes sense for the source of the
data. The point is that it's an independent source of data which can be
assigned to a producer instance.

If the same source partition is always assigned to the producer with the
the same AppID, then Kafka transactions will give you "exactly once"
delivery without much additional work. On initialization, the producer will
ensure that 1) any previous producers using that AppID are "fenced" off,
and 2) that any transaction which had been started by a previous producer
with that AppID have either completed or aborted.

Based on this, it should be clear that the ideal is to divide the workload
so that you have a one-to-one mapping from the source partition to the
AppID. If the source of the data is Kafka, then the source partition is
just a topic partition, and the AppID can be generated from the name of the
topic and the partition number.

To finally get back to your auto-scaling question, let's assume for a
moment the ideal mapping of source partition to AppID. The main question is
whether the scaling is "horizontal" or "vertical." By horizontal, I mean an
increase in the number of source partitions. This case is easy. Assign new
AppIDs based on the new source partitions and you're done.

But if the scaling is vertical (i.e. an increase in the load on the source
partitions), there's not much this proposal can do to help. You're going to
have to break the source partition into child partitions, and assign each
of the new partitions a new AppID. To preserve "exactly once" delivery, you
must make sure that the producers using the AppID assigned to the parent
partition have been shutdown cleanly. We could provide a way to pass in a
"parent AppID" so that the producer could check the appropriate safety
conditions, but for the first version, we assume that users consider
scaling requirements when dividing the workload into source partitions.

Unfortunately, the real world is always falling short of the ideal, and
it's not always practical to have a one-to-one mapping of source partition
to AppID, since that also implies a one-to-one mapping of source partition
to producer instance. If I were a user, I'd push this limit as far as is
reasonable, but with enough source partitions, it eventually breaks down.
At some point, you need a producer to handle the load of more than one
source partition. This is fine in itself if the assignment is sticky: that
is, if we can ensure that the same source partition is assigned to the
producer using a certain AppID. If not, then the user is responsible for
ensuring a clean hand-off. The producer reading from the migrating source
partition must stop reading, commit or abort any transaction containing
data processed from that source partition, and then signal the producer
which is taking over that it is safe to begin.

This burden is a consequence of the decision to keep the producer out of
the role of assigning work. We could do more if we forced users to
formalize their application-specific notion of a source partition, and if
we turned the producer into something like a consumer group, with a
rebalance protocol. This would allow the broker to be the one to ensure a
clean hand-off of work, but it would be a huge departure from the way the
producer currently works, and not all applications have a notion of source
partition anyway. So the result is a bit more work for the user, though of
course it would be transparent to for Kafka Streams users.

One final note. I've described above how to get the strongest guarantees
that this work is capable of providing in an auto-scaling environment. We
also provide weaker guarantees, which are still an improvement over the
current state. For example, without specifying any kind of AppID, we
provide idempotent production for the lifetime of a producer instance. This
ensures reliable delivery without duplicates even with broker failures. It
is also possible to use transactions without an ephemeral AppID. If the
application generates a UUID for user as the AppID, and only uses it for
the lifetime of a single producer, you can still take advantage of
transactional semantics, which allows you to write to a set of messages to
multiple partitions atomically.

Hope that answers the question and helps others understand the work a bit
better!

Thanks,
Jason



On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Thanks for your comment, I updated the document. Let me know if it is clear
> now.
>
> Apurva
>
> On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > @Apurva yep that's what I was trying to say.
> >
> > Original message:
> > If there is already an entry with the AppID in the mapping, increment the
> > epoch number and go on to the next step. If there is no entry with the
> > AppID in the mapping, construct a PID with initialized epoch number;
> append
> > an AppID message into the transaction topic, insert into the mapping and
> > reply with the PID / epoch / timestamp.
> >
> > Just wanted to make it explicit because:
> > 1. The "append an AppID message..." chunk was ambiguous on whether it
> > applied to the "if exists" or "if not exists" condition
> > 2. I think the google doc is pretty explicit on appending to the log
> > everywhere else.
> >
> > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > The first line in step 2 of that section is: "If there is already an
> > entry
> > > with the AppID in the mapping, increment the epoch number and go on to
> > the
> > > next step."
> > >
> > > Are you suggesting that it be made explicit that 'increment the epoch
> > > number' includes persisting the updated value to the log?
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > onurkaraman.apache@gmail.com>
> > > wrote:
> > >
> > > > Nice google doc!
> > > >
> > > > Probably need to go over the google doc a few more times, but a minor
> > > > comment from the first pass:
> > > >
> > > > In Transaction Coordinator Request Handling (
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > step 2 mentions that if the Transaction Coordinator doesn't already
> > see a
> > > > producer with the same app-id, it creates a pid and appends (app-id,
> > pid,
> > > > epoch) into the transaction log.
> > > >
> > > > What about if the app-id/pid pair already exists and we increment the
> > > > epoch? Should we append (app-id, pid, epoch++) to the transaction
> log?
> > I
> > > > think we should, but step 2 doesn't mention this.
> > > >
> > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > > >
> > > > > Thanks for your comments, let me deal with your second point
> > regarding
> > > > > merging the __consumer-offsets and transactions topic.
> > > > >
> > > > > Needless to say, we considered doing this, but chose to keep them
> > > > separate
> > > > > for the following reasons:
> > > > >
> > > > >    1. Your assumption that group.id and transaction.app.id can be
> > the
> > > > same
> > > > >    does not hold for streams applications. All colocated tasks of a
> > > > streams
> > > > >    application will share the same consumer (and hence implicitly
> > will
> > > > have
> > > > >    the same group.id), but each task will have its own producer
> > > > instance.
> > > > >    The transaction.app.id for each producer instance will still
> have
> > > to
> > > > be
> > > > >    distinct. So to colocate the transaction and consumer group
> > > > > coordinators,
> > > > >    we will have to now introduce a 'group.id' config in the
> producer
> > > and
> > > > >    require it to be the same as the consumer. This seemed like a
> very
> > > > > fragile
> > > > >    option.
> > > > >    2. Following on from the above, the transaction coordinator and
> > > group
> > > > >    coordinator would _have_ to be colocated inorder to be the
> leader
> > > for
> > > > > the
> > > > >    same TopicPartition, unless we wanted to make even more
> > fundamental
> > > > > changes
> > > > >    to Kafka.
> > > > >    3. We don't require that the consumer coordinator and the
> > > transaction
> > > > >    coordinator have the same view of the current PID/Epoch pair.
> If a
> > > > > producer
> > > > >    instance is bounced, the epoch will be bumped. Any transactions
> > > > > initiated
> > > > >    by the previous instance would either be fully committed or
> fully
> > > > rolled
> > > > >    back. Since the writes to the offset topics are just like writes
> > to
> > > a
> > > > >    regular topic, these would enjoy the same guarantees, and the
> > > > > inconsistency
> > > > >    will be eventually resolved.
> > > > >    4. Finally, every application will have consumers, and hence
> > record
> > > > >    consumer offsets. But a very small fraction of applications
> would
> > > use
> > > > >    transactions. Blending the two topics would make recovering
> > > > transaction
> > > > >    coordinator state unnecessarily inefficient since it has to read
> > > from
> > > > > the
> > > > >    beginning of the topic to reconstruct its data structures -- it
> > > would
> > > > > have
> > > > >    to inspect and skip a majority of the messages if the offsets
> were
> > > in
> > > > > the
> > > > >    same topic.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io>
> > > > wrote:
> > > > >
> > > > > > Thanks for initiating this KIP! I think it is well written and
> I'm
> > > > > excited
> > > > > > to see the first step towards adding an important feature in
> Kafka.
> > > > > >
> > > > > > I had a few initial thoughts on the KIP, mostly not as deeply
> > thought
> > > > > > through than what you've done -
> > > > > >
> > > > > > 1. Perhaps you’ve thought about how this would work already —
> since
> > > we
> > > > > now
> > > > > > require a producer to specify a unique AppID across different
> > > instances
> > > > > of
> > > > > > an application, how would applications that run in the cloud use
> > this
> > > > > > feature with auto scaling?
> > > > > >
> > > > > > 2. Making it easy for applications to get exactly-once semantics
> > for
> > > a
> > > > > > consume-process-produce workflow is a great feature to have. To
> > > enable
> > > > > > this, the proposal now includes letting a producer initiate a
> write
> > > to
> > > > > the
> > > > > > offset topic as well (just like consumers do). The consumer
> > > coordinator
> > > > > > (which could be on a different broker than the txn coordinator)
> > would
> > > > > then
> > > > > > validate if the PID and producer epoch is valid before it writes
> to
> > > the
> > > > > > offset topic along with the associated PID. This is a great
> feature
> > > > > though
> > > > > > I see 2 difficulties
> > > > > >
> > > > > > -- This needs the consumer coordinator to have a consistent view
> of
> > > the
> > > > > > PID/epochs that is same as the view on the txn coordinator.
> > However,
> > > as
> > > > > the
> > > > > > offset and the transaction topics are different, the 2
> coordinators
> > > > might
> > > > > > live on different brokers.
> > > > > > -- We now also have 2 internal topics - a transaction topic and
> the
> > > > > > __consumer_offsets topic.
> > > > > >
> > > > > > Maybe you’ve thought about this already and discarded it ... let
> me
> > > > make
> > > > > a
> > > > > > somewhat crazy proposal — Why don’t we upgrade the transaction
> > topic
> > > to
> > > > > be
> > > > > > the new offsets topic as well? For consumers that want EoS
> > guarantees
> > > > for
> > > > > > a consume-process-produce pattern, the group.id is the same as
> the
> > > > > > transaction.app.id set for the producer. Assume that the
> > transaction
> > > > > topic
> > > > > > also stores consumer offsets. It stores both the transaction
> > metadata
> > > > > > messages as well as offset messages, both for transactional as
> well
> > > as
> > > > > > non-transactional consumers. Since the group.id of the consumer
> > and
> > > > the
> > > > > > app.id of the producer is the same, the offsets associated with
> a
> > > > > consumer
> > > > > > group and topic-partition end up in the same transaction topic
> > > > partition
> > > > > as
> > > > > > the transaction metadata messages. The transaction coordinator
> and
> > > the
> > > > > > consumer coordinator always live on the same broker since they
> both
> > > map
> > > > > to
> > > > > > the same partition in the transaction topic. Even if there are
> > > > failures,
> > > > > > they end up on the same new broker. Hence, they share the same
> and
> > > > > > consistent view of the PIDs, epochs and App IDs, whatever it is.
> > The
> > > > > > consumer coordinator will skip over the transaction metadata
> > messages
> > > > > when
> > > > > > it bootstraps the offsets from this new topic for consumer groups
> > > that
> > > > > are
> > > > > > not involved in a transaction and don’t have a txn id associated
> > with
> > > > the
> > > > > > offset message in the transaction topic. The consumer coordinator
> > > will
> > > > > > expose only committed offsets in cases of consumer groups that
> are
> > > > > involved
> > > > > > in a txn. It will also be able to validate the
> OffsetCommitRequests
> > > > > coming
> > > > > > from a transactional producer by ensuring that it is coming from
> a
> > > > valid
> > > > > > PID, producer epoch since it uses the same view of this data
> > created
> > > by
> > > > > the
> > > > > > transaction coordinator (that lives on the same broker). And we
> > will
> > > > end
> > > > > up
> > > > > > with one internal topic, not too.
> > > > > >
> > > > > > This proposal offers better operational simplicity and fewer
> > internal
> > > > > > topics but there are some downsides that come with it — there
> are 2
> > > > types
> > > > > > of messages in one topic (txn metadata ones and offset ones).
> Since
> > > > this
> > > > > > internal topic serves a dual purpose, it will be harder to name
> it
> > > and
> > > > > also
> > > > > > design a message format that includes the different types of
> > messages
> > > > > that
> > > > > > will live in the topic. Though the transaction topic already
> needs
> > to
> > > > > write
> > > > > > 5 different types of messages (the AppID->PID mapping, the
> BeginTxn
> > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe
> > adding
> > > > the
> > > > > > offset message isn't a big deal?
> > > > > >
> > > > > > Back when we introduced the offsets topic, we had discussed
> making
> > it
> > > > > more
> > > > > > general and allowing the producer to send offset commit messages
> to
> > > it
> > > > > but
> > > > > > ended up creating a specialized topic to allow the consumer
> > > coordinator
> > > > > to
> > > > > > wall off and prevent unauthorized writes from consumers outside
> of
> > a
> > > > > group.
> > > > > > Jason can comment on the specifics but I don't believe that goal
> of
> > > the
> > > > > new
> > > > > > consumer protocol was quite achieved.
> > > > > >
> > > > > > I have other comments on the message format, request names etc
> but
> > > > wanted
> > > > > > to get your thoughts on these 2 issues first :-)
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > > delivery
> > > > > > > semantics:
> > > > > > >
> > > > > > > *
> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > <
> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > >*
> > > > > > >
> > > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > > idempotent
> > > > > > > producer implementation to make sure that 1) duplicated
> messages
> > > sent
> > > > > > from
> > > > > > > the same identified producer can be detected on the broker
> side,
> > > and
> > > > > 2) a
> > > > > > > group of messages sent within a transaction will atomically be
> > > either
> > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > >
> > > > > > > The above wiki page provides a high-level view of the proposed
> > > > changes
> > > > > as
> > > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > > implementation
> > > > > > > design is described in this Google doc:
> > > > > > >
> > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8
> > > > > > > <https://docs.google.com/document/d/11Jqy_
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8>
> > > > > > >
> > > > > > >
> > > > > > > We would love to hear your comments and suggestions.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > -- Guozhang
> > > > > > >
> > > > > > --
> > > > > > Thanks,
> > > > > > Neha
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Henry Cai <hc...@pinterest.com.INVALID>.
During the 'consumer-transform-produce' cycle, if the consumer app needs to
update an external data store (e.g. update RocksDB in Kafka streams), and
that update is not idempotent (e.g. increment a counter in RocksDB), how do
you make that update part of Kafka transaction?


On Wed, Nov 30, 2016 at 9:51 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Thanks for your comment, I updated the document. Let me know if it is clear
> now.
>
> Apurva
>
> On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > @Apurva yep that's what I was trying to say.
> >
> > Original message:
> > If there is already an entry with the AppID in the mapping, increment the
> > epoch number and go on to the next step. If there is no entry with the
> > AppID in the mapping, construct a PID with initialized epoch number;
> append
> > an AppID message into the transaction topic, insert into the mapping and
> > reply with the PID / epoch / timestamp.
> >
> > Just wanted to make it explicit because:
> > 1. The "append an AppID message..." chunk was ambiguous on whether it
> > applied to the "if exists" or "if not exists" condition
> > 2. I think the google doc is pretty explicit on appending to the log
> > everywhere else.
> >
> > On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > The first line in step 2 of that section is: "If there is already an
> > entry
> > > with the AppID in the mapping, increment the epoch number and go on to
> > the
> > > next step."
> > >
> > > Are you suggesting that it be made explicit that 'increment the epoch
> > > number' includes persisting the updated value to the log?
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > > onurkaraman.apache@gmail.com>
> > > wrote:
> > >
> > > > Nice google doc!
> > > >
> > > > Probably need to go over the google doc a few more times, but a minor
> > > > comment from the first pass:
> > > >
> > > > In Transaction Coordinator Request Handling (
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > > step 2 mentions that if the Transaction Coordinator doesn't already
> > see a
> > > > producer with the same app-id, it creates a pid and appends (app-id,
> > pid,
> > > > epoch) into the transaction log.
> > > >
> > > > What about if the app-id/pid pair already exists and we increment the
> > > > epoch? Should we append (app-id, pid, epoch++) to the transaction
> log?
> > I
> > > > think we should, but step 2 doesn't mention this.
> > > >
> > > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > > >
> > > > > Thanks for your comments, let me deal with your second point
> > regarding
> > > > > merging the __consumer-offsets and transactions topic.
> > > > >
> > > > > Needless to say, we considered doing this, but chose to keep them
> > > > separate
> > > > > for the following reasons:
> > > > >
> > > > >    1. Your assumption that group.id and transaction.app.id can be
> > the
> > > > same
> > > > >    does not hold for streams applications. All colocated tasks of a
> > > > streams
> > > > >    application will share the same consumer (and hence implicitly
> > will
> > > > have
> > > > >    the same group.id), but each task will have its own producer
> > > > instance.
> > > > >    The transaction.app.id for each producer instance will still
> have
> > > to
> > > > be
> > > > >    distinct. So to colocate the transaction and consumer group
> > > > > coordinators,
> > > > >    we will have to now introduce a 'group.id' config in the
> producer
> > > and
> > > > >    require it to be the same as the consumer. This seemed like a
> very
> > > > > fragile
> > > > >    option.
> > > > >    2. Following on from the above, the transaction coordinator and
> > > group
> > > > >    coordinator would _have_ to be colocated inorder to be the
> leader
> > > for
> > > > > the
> > > > >    same TopicPartition, unless we wanted to make even more
> > fundamental
> > > > > changes
> > > > >    to Kafka.
> > > > >    3. We don't require that the consumer coordinator and the
> > > transaction
> > > > >    coordinator have the same view of the current PID/Epoch pair.
> If a
> > > > > producer
> > > > >    instance is bounced, the epoch will be bumped. Any transactions
> > > > > initiated
> > > > >    by the previous instance would either be fully committed or
> fully
> > > > rolled
> > > > >    back. Since the writes to the offset topics are just like writes
> > to
> > > a
> > > > >    regular topic, these would enjoy the same guarantees, and the
> > > > > inconsistency
> > > > >    will be eventually resolved.
> > > > >    4. Finally, every application will have consumers, and hence
> > record
> > > > >    consumer offsets. But a very small fraction of applications
> would
> > > use
> > > > >    transactions. Blending the two topics would make recovering
> > > > transaction
> > > > >    coordinator state unnecessarily inefficient since it has to read
> > > from
> > > > > the
> > > > >    beginning of the topic to reconstruct its data structures -- it
> > > would
> > > > > have
> > > > >    to inspect and skip a majority of the messages if the offsets
> were
> > > in
> > > > > the
> > > > >    same topic.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io>
> > > > wrote:
> > > > >
> > > > > > Thanks for initiating this KIP! I think it is well written and
> I'm
> > > > > excited
> > > > > > to see the first step towards adding an important feature in
> Kafka.
> > > > > >
> > > > > > I had a few initial thoughts on the KIP, mostly not as deeply
> > thought
> > > > > > through than what you've done -
> > > > > >
> > > > > > 1. Perhaps you’ve thought about how this would work already —
> since
> > > we
> > > > > now
> > > > > > require a producer to specify a unique AppID across different
> > > instances
> > > > > of
> > > > > > an application, how would applications that run in the cloud use
> > this
> > > > > > feature with auto scaling?
> > > > > >
> > > > > > 2. Making it easy for applications to get exactly-once semantics
> > for
> > > a
> > > > > > consume-process-produce workflow is a great feature to have. To
> > > enable
> > > > > > this, the proposal now includes letting a producer initiate a
> write
> > > to
> > > > > the
> > > > > > offset topic as well (just like consumers do). The consumer
> > > coordinator
> > > > > > (which could be on a different broker than the txn coordinator)
> > would
> > > > > then
> > > > > > validate if the PID and producer epoch is valid before it writes
> to
> > > the
> > > > > > offset topic along with the associated PID. This is a great
> feature
> > > > > though
> > > > > > I see 2 difficulties
> > > > > >
> > > > > > -- This needs the consumer coordinator to have a consistent view
> of
> > > the
> > > > > > PID/epochs that is same as the view on the txn coordinator.
> > However,
> > > as
> > > > > the
> > > > > > offset and the transaction topics are different, the 2
> coordinators
> > > > might
> > > > > > live on different brokers.
> > > > > > -- We now also have 2 internal topics - a transaction topic and
> the
> > > > > > __consumer_offsets topic.
> > > > > >
> > > > > > Maybe you’ve thought about this already and discarded it ... let
> me
> > > > make
> > > > > a
> > > > > > somewhat crazy proposal — Why don’t we upgrade the transaction
> > topic
> > > to
> > > > > be
> > > > > > the new offsets topic as well? For consumers that want EoS
> > guarantees
> > > > for
> > > > > > a consume-process-produce pattern, the group.id is the same as
> the
> > > > > > transaction.app.id set for the producer. Assume that the
> > transaction
> > > > > topic
> > > > > > also stores consumer offsets. It stores both the transaction
> > metadata
> > > > > > messages as well as offset messages, both for transactional as
> well
> > > as
> > > > > > non-transactional consumers. Since the group.id of the consumer
> > and
> > > > the
> > > > > > app.id of the producer is the same, the offsets associated with
> a
> > > > > consumer
> > > > > > group and topic-partition end up in the same transaction topic
> > > > partition
> > > > > as
> > > > > > the transaction metadata messages. The transaction coordinator
> and
> > > the
> > > > > > consumer coordinator always live on the same broker since they
> both
> > > map
> > > > > to
> > > > > > the same partition in the transaction topic. Even if there are
> > > > failures,
> > > > > > they end up on the same new broker. Hence, they share the same
> and
> > > > > > consistent view of the PIDs, epochs and App IDs, whatever it is.
> > The
> > > > > > consumer coordinator will skip over the transaction metadata
> > messages
> > > > > when
> > > > > > it bootstraps the offsets from this new topic for consumer groups
> > > that
> > > > > are
> > > > > > not involved in a transaction and don’t have a txn id associated
> > with
> > > > the
> > > > > > offset message in the transaction topic. The consumer coordinator
> > > will
> > > > > > expose only committed offsets in cases of consumer groups that
> are
> > > > > involved
> > > > > > in a txn. It will also be able to validate the
> OffsetCommitRequests
> > > > > coming
> > > > > > from a transactional producer by ensuring that it is coming from
> a
> > > > valid
> > > > > > PID, producer epoch since it uses the same view of this data
> > created
> > > by
> > > > > the
> > > > > > transaction coordinator (that lives on the same broker). And we
> > will
> > > > end
> > > > > up
> > > > > > with one internal topic, not too.
> > > > > >
> > > > > > This proposal offers better operational simplicity and fewer
> > internal
> > > > > > topics but there are some downsides that come with it — there
> are 2
> > > > types
> > > > > > of messages in one topic (txn metadata ones and offset ones).
> Since
> > > > this
> > > > > > internal topic serves a dual purpose, it will be harder to name
> it
> > > and
> > > > > also
> > > > > > design a message format that includes the different types of
> > messages
> > > > > that
> > > > > > will live in the topic. Though the transaction topic already
> needs
> > to
> > > > > write
> > > > > > 5 different types of messages (the AppID->PID mapping, the
> BeginTxn
> > > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe
> > adding
> > > > the
> > > > > > offset message isn't a big deal?
> > > > > >
> > > > > > Back when we introduced the offsets topic, we had discussed
> making
> > it
> > > > > more
> > > > > > general and allowing the producer to send offset commit messages
> to
> > > it
> > > > > but
> > > > > > ended up creating a specialized topic to allow the consumer
> > > coordinator
> > > > > to
> > > > > > wall off and prevent unauthorized writes from consumers outside
> of
> > a
> > > > > group.
> > > > > > Jason can comment on the specifics but I don't believe that goal
> of
> > > the
> > > > > new
> > > > > > consumer protocol was quite achieved.
> > > > > >
> > > > > > I have other comments on the message format, request names etc
> but
> > > > wanted
> > > > > > to get your thoughts on these 2 issues first :-)
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <
> wangguoz@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > > delivery
> > > > > > > semantics:
> > > > > > >
> > > > > > > *
> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > <
> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > >*
> > > > > > >
> > > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > > idempotent
> > > > > > > producer implementation to make sure that 1) duplicated
> messages
> > > sent
> > > > > > from
> > > > > > > the same identified producer can be detected on the broker
> side,
> > > and
> > > > > 2) a
> > > > > > > group of messages sent within a transaction will atomically be
> > > either
> > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > >
> > > > > > > The above wiki page provides a high-level view of the proposed
> > > > changes
> > > > > as
> > > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > > implementation
> > > > > > > design is described in this Google doc:
> > > > > > >
> > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8
> > > > > > > <https://docs.google.com/document/d/11Jqy_
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8>
> > > > > > >
> > > > > > >
> > > > > > > We would love to hear your comments and suggestions.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > -- Guozhang
> > > > > > >
> > > > > > --
> > > > > > Thanks,
> > > > > > Neha
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Thanks for your comment, I updated the document. Let me know if it is clear
now.

Apurva

On Wed, Nov 30, 2016 at 9:42 PM, Onur Karaman <on...@gmail.com>
wrote:

> @Apurva yep that's what I was trying to say.
>
> Original message:
> If there is already an entry with the AppID in the mapping, increment the
> epoch number and go on to the next step. If there is no entry with the
> AppID in the mapping, construct a PID with initialized epoch number; append
> an AppID message into the transaction topic, insert into the mapping and
> reply with the PID / epoch / timestamp.
>
> Just wanted to make it explicit because:
> 1. The "append an AppID message..." chunk was ambiguous on whether it
> applied to the "if exists" or "if not exists" condition
> 2. I think the google doc is pretty explicit on appending to the log
> everywhere else.
>
> On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > The first line in step 2 of that section is: "If there is already an
> entry
> > with the AppID in the mapping, increment the epoch number and go on to
> the
> > next step."
> >
> > Are you suggesting that it be made explicit that 'increment the epoch
> > number' includes persisting the updated value to the log?
> >
> > Thanks,
> > Apurva
> >
> > On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> > onurkaraman.apache@gmail.com>
> > wrote:
> >
> > > Nice google doc!
> > >
> > > Probably need to go over the google doc a few more times, but a minor
> > > comment from the first pass:
> > >
> > > In Transaction Coordinator Request Handling (
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > > step 2 mentions that if the Transaction Coordinator doesn't already
> see a
> > > producer with the same app-id, it creates a pid and appends (app-id,
> pid,
> > > epoch) into the transaction log.
> > >
> > > What about if the app-id/pid pair already exists and we increment the
> > > epoch? Should we append (app-id, pid, epoch++) to the transaction log?
> I
> > > think we should, but step 2 doesn't mention this.
> > >
> > > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> > >
> > > > Thanks for your comments, let me deal with your second point
> regarding
> > > > merging the __consumer-offsets and transactions topic.
> > > >
> > > > Needless to say, we considered doing this, but chose to keep them
> > > separate
> > > > for the following reasons:
> > > >
> > > >    1. Your assumption that group.id and transaction.app.id can be
> the
> > > same
> > > >    does not hold for streams applications. All colocated tasks of a
> > > streams
> > > >    application will share the same consumer (and hence implicitly
> will
> > > have
> > > >    the same group.id), but each task will have its own producer
> > > instance.
> > > >    The transaction.app.id for each producer instance will still have
> > to
> > > be
> > > >    distinct. So to colocate the transaction and consumer group
> > > > coordinators,
> > > >    we will have to now introduce a 'group.id' config in the producer
> > and
> > > >    require it to be the same as the consumer. This seemed like a very
> > > > fragile
> > > >    option.
> > > >    2. Following on from the above, the transaction coordinator and
> > group
> > > >    coordinator would _have_ to be colocated inorder to be the leader
> > for
> > > > the
> > > >    same TopicPartition, unless we wanted to make even more
> fundamental
> > > > changes
> > > >    to Kafka.
> > > >    3. We don't require that the consumer coordinator and the
> > transaction
> > > >    coordinator have the same view of the current PID/Epoch pair. If a
> > > > producer
> > > >    instance is bounced, the epoch will be bumped. Any transactions
> > > > initiated
> > > >    by the previous instance would either be fully committed or fully
> > > rolled
> > > >    back. Since the writes to the offset topics are just like writes
> to
> > a
> > > >    regular topic, these would enjoy the same guarantees, and the
> > > > inconsistency
> > > >    will be eventually resolved.
> > > >    4. Finally, every application will have consumers, and hence
> record
> > > >    consumer offsets. But a very small fraction of applications would
> > use
> > > >    transactions. Blending the two topics would make recovering
> > > transaction
> > > >    coordinator state unnecessarily inefficient since it has to read
> > from
> > > > the
> > > >    beginning of the topic to reconstruct its data structures -- it
> > would
> > > > have
> > > >    to inspect and skip a majority of the messages if the offsets were
> > in
> > > > the
> > > >    same topic.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io>
> > > wrote:
> > > >
> > > > > Thanks for initiating this KIP! I think it is well written and I'm
> > > > excited
> > > > > to see the first step towards adding an important feature in Kafka.
> > > > >
> > > > > I had a few initial thoughts on the KIP, mostly not as deeply
> thought
> > > > > through than what you've done -
> > > > >
> > > > > 1. Perhaps you’ve thought about how this would work already — since
> > we
> > > > now
> > > > > require a producer to specify a unique AppID across different
> > instances
> > > > of
> > > > > an application, how would applications that run in the cloud use
> this
> > > > > feature with auto scaling?
> > > > >
> > > > > 2. Making it easy for applications to get exactly-once semantics
> for
> > a
> > > > > consume-process-produce workflow is a great feature to have. To
> > enable
> > > > > this, the proposal now includes letting a producer initiate a write
> > to
> > > > the
> > > > > offset topic as well (just like consumers do). The consumer
> > coordinator
> > > > > (which could be on a different broker than the txn coordinator)
> would
> > > > then
> > > > > validate if the PID and producer epoch is valid before it writes to
> > the
> > > > > offset topic along with the associated PID. This is a great feature
> > > > though
> > > > > I see 2 difficulties
> > > > >
> > > > > -- This needs the consumer coordinator to have a consistent view of
> > the
> > > > > PID/epochs that is same as the view on the txn coordinator.
> However,
> > as
> > > > the
> > > > > offset and the transaction topics are different, the 2 coordinators
> > > might
> > > > > live on different brokers.
> > > > > -- We now also have 2 internal topics - a transaction topic and the
> > > > > __consumer_offsets topic.
> > > > >
> > > > > Maybe you’ve thought about this already and discarded it ... let me
> > > make
> > > > a
> > > > > somewhat crazy proposal — Why don’t we upgrade the transaction
> topic
> > to
> > > > be
> > > > > the new offsets topic as well? For consumers that want EoS
> guarantees
> > > for
> > > > > a consume-process-produce pattern, the group.id is the same as the
> > > > > transaction.app.id set for the producer. Assume that the
> transaction
> > > > topic
> > > > > also stores consumer offsets. It stores both the transaction
> metadata
> > > > > messages as well as offset messages, both for transactional as well
> > as
> > > > > non-transactional consumers. Since the group.id of the consumer
> and
> > > the
> > > > > app.id of the producer is the same, the offsets associated with a
> > > > consumer
> > > > > group and topic-partition end up in the same transaction topic
> > > partition
> > > > as
> > > > > the transaction metadata messages. The transaction coordinator and
> > the
> > > > > consumer coordinator always live on the same broker since they both
> > map
> > > > to
> > > > > the same partition in the transaction topic. Even if there are
> > > failures,
> > > > > they end up on the same new broker. Hence, they share the same and
> > > > > consistent view of the PIDs, epochs and App IDs, whatever it is.
> The
> > > > > consumer coordinator will skip over the transaction metadata
> messages
> > > > when
> > > > > it bootstraps the offsets from this new topic for consumer groups
> > that
> > > > are
> > > > > not involved in a transaction and don’t have a txn id associated
> with
> > > the
> > > > > offset message in the transaction topic. The consumer coordinator
> > will
> > > > > expose only committed offsets in cases of consumer groups that are
> > > > involved
> > > > > in a txn. It will also be able to validate the OffsetCommitRequests
> > > > coming
> > > > > from a transactional producer by ensuring that it is coming from a
> > > valid
> > > > > PID, producer epoch since it uses the same view of this data
> created
> > by
> > > > the
> > > > > transaction coordinator (that lives on the same broker). And we
> will
> > > end
> > > > up
> > > > > with one internal topic, not too.
> > > > >
> > > > > This proposal offers better operational simplicity and fewer
> internal
> > > > > topics but there are some downsides that come with it — there are 2
> > > types
> > > > > of messages in one topic (txn metadata ones and offset ones). Since
> > > this
> > > > > internal topic serves a dual purpose, it will be harder to name it
> > and
> > > > also
> > > > > design a message format that includes the different types of
> messages
> > > > that
> > > > > will live in the topic. Though the transaction topic already needs
> to
> > > > write
> > > > > 5 different types of messages (the AppID->PID mapping, the BeginTxn
> > > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe
> adding
> > > the
> > > > > offset message isn't a big deal?
> > > > >
> > > > > Back when we introduced the offsets topic, we had discussed making
> it
> > > > more
> > > > > general and allowing the producer to send offset commit messages to
> > it
> > > > but
> > > > > ended up creating a specialized topic to allow the consumer
> > coordinator
> > > > to
> > > > > wall off and prevent unauthorized writes from consumers outside of
> a
> > > > group.
> > > > > Jason can comment on the specifics but I don't believe that goal of
> > the
> > > > new
> > > > > consumer protocol was quite achieved.
> > > > >
> > > > > I have other comments on the message format, request names etc but
> > > wanted
> > > > > to get your thoughts on these 2 issues first :-)
> > > > >
> > > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > delivery
> > > > > > semantics:
> > > > > >
> > > > > > *
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > <
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > >*
> > > > > >
> > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > idempotent
> > > > > > producer implementation to make sure that 1) duplicated messages
> > sent
> > > > > from
> > > > > > the same identified producer can be detected on the broker side,
> > and
> > > > 2) a
> > > > > > group of messages sent within a transaction will atomically be
> > either
> > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > >
> > > > > > The above wiki page provides a high-level view of the proposed
> > > changes
> > > > as
> > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > implementation
> > > > > > design is described in this Google doc:
> > > > > >
> > > > > > https://docs.google.com/document/d/11Jqy_
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8
> > > > > > <https://docs.google.com/document/d/11Jqy_
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8>
> > > > > >
> > > > > >
> > > > > > We would love to hear your comments and suggestions.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > -- Guozhang
> > > > > >
> > > > > --
> > > > > Thanks,
> > > > > Neha
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Onur Karaman <on...@gmail.com>.
@Apurva yep that's what I was trying to say.

Original message:
If there is already an entry with the AppID in the mapping, increment the
epoch number and go on to the next step. If there is no entry with the
AppID in the mapping, construct a PID with initialized epoch number; append
an AppID message into the transaction topic, insert into the mapping and
reply with the PID / epoch / timestamp.

Just wanted to make it explicit because:
1. The "append an AppID message..." chunk was ambiguous on whether it
applied to the "if exists" or "if not exists" condition
2. I think the google doc is pretty explicit on appending to the log
everywhere else.

On Wed, Nov 30, 2016 at 9:36 PM, Apurva Mehta <ap...@confluent.io> wrote:

> The first line in step 2 of that section is: "If there is already an entry
> with the AppID in the mapping, increment the epoch number and go on to the
> next step."
>
> Are you suggesting that it be made explicit that 'increment the epoch
> number' includes persisting the updated value to the log?
>
> Thanks,
> Apurva
>
> On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > Nice google doc!
> >
> > Probably need to go over the google doc a few more times, but a minor
> > comment from the first pass:
> >
> > In Transaction Coordinator Request Handling (
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> > step 2 mentions that if the Transaction Coordinator doesn't already see a
> > producer with the same app-id, it creates a pid and appends (app-id, pid,
> > epoch) into the transaction log.
> >
> > What about if the app-id/pid pair already exists and we increment the
> > epoch? Should we append (app-id, pid, epoch++) to the transaction log? I
> > think we should, but step 2 doesn't mention this.
> >
> > On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Thanks for your comments, let me deal with your second point regarding
> > > merging the __consumer-offsets and transactions topic.
> > >
> > > Needless to say, we considered doing this, but chose to keep them
> > separate
> > > for the following reasons:
> > >
> > >    1. Your assumption that group.id and transaction.app.id can be the
> > same
> > >    does not hold for streams applications. All colocated tasks of a
> > streams
> > >    application will share the same consumer (and hence implicitly will
> > have
> > >    the same group.id), but each task will have its own producer
> > instance.
> > >    The transaction.app.id for each producer instance will still have
> to
> > be
> > >    distinct. So to colocate the transaction and consumer group
> > > coordinators,
> > >    we will have to now introduce a 'group.id' config in the producer
> and
> > >    require it to be the same as the consumer. This seemed like a very
> > > fragile
> > >    option.
> > >    2. Following on from the above, the transaction coordinator and
> group
> > >    coordinator would _have_ to be colocated inorder to be the leader
> for
> > > the
> > >    same TopicPartition, unless we wanted to make even more fundamental
> > > changes
> > >    to Kafka.
> > >    3. We don't require that the consumer coordinator and the
> transaction
> > >    coordinator have the same view of the current PID/Epoch pair. If a
> > > producer
> > >    instance is bounced, the epoch will be bumped. Any transactions
> > > initiated
> > >    by the previous instance would either be fully committed or fully
> > rolled
> > >    back. Since the writes to the offset topics are just like writes to
> a
> > >    regular topic, these would enjoy the same guarantees, and the
> > > inconsistency
> > >    will be eventually resolved.
> > >    4. Finally, every application will have consumers, and hence record
> > >    consumer offsets. But a very small fraction of applications would
> use
> > >    transactions. Blending the two topics would make recovering
> > transaction
> > >    coordinator state unnecessarily inefficient since it has to read
> from
> > > the
> > >    beginning of the topic to reconstruct its data structures -- it
> would
> > > have
> > >    to inspect and skip a majority of the messages if the offsets were
> in
> > > the
> > >    same topic.
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io>
> > wrote:
> > >
> > > > Thanks for initiating this KIP! I think it is well written and I'm
> > > excited
> > > > to see the first step towards adding an important feature in Kafka.
> > > >
> > > > I had a few initial thoughts on the KIP, mostly not as deeply thought
> > > > through than what you've done -
> > > >
> > > > 1. Perhaps you’ve thought about how this would work already — since
> we
> > > now
> > > > require a producer to specify a unique AppID across different
> instances
> > > of
> > > > an application, how would applications that run in the cloud use this
> > > > feature with auto scaling?
> > > >
> > > > 2. Making it easy for applications to get exactly-once semantics for
> a
> > > > consume-process-produce workflow is a great feature to have. To
> enable
> > > > this, the proposal now includes letting a producer initiate a write
> to
> > > the
> > > > offset topic as well (just like consumers do). The consumer
> coordinator
> > > > (which could be on a different broker than the txn coordinator) would
> > > then
> > > > validate if the PID and producer epoch is valid before it writes to
> the
> > > > offset topic along with the associated PID. This is a great feature
> > > though
> > > > I see 2 difficulties
> > > >
> > > > -- This needs the consumer coordinator to have a consistent view of
> the
> > > > PID/epochs that is same as the view on the txn coordinator. However,
> as
> > > the
> > > > offset and the transaction topics are different, the 2 coordinators
> > might
> > > > live on different brokers.
> > > > -- We now also have 2 internal topics - a transaction topic and the
> > > > __consumer_offsets topic.
> > > >
> > > > Maybe you’ve thought about this already and discarded it ... let me
> > make
> > > a
> > > > somewhat crazy proposal — Why don’t we upgrade the transaction topic
> to
> > > be
> > > > the new offsets topic as well? For consumers that want EoS guarantees
> > for
> > > > a consume-process-produce pattern, the group.id is the same as the
> > > > transaction.app.id set for the producer. Assume that the transaction
> > > topic
> > > > also stores consumer offsets. It stores both the transaction metadata
> > > > messages as well as offset messages, both for transactional as well
> as
> > > > non-transactional consumers. Since the group.id of the consumer and
> > the
> > > > app.id of the producer is the same, the offsets associated with a
> > > consumer
> > > > group and topic-partition end up in the same transaction topic
> > partition
> > > as
> > > > the transaction metadata messages. The transaction coordinator and
> the
> > > > consumer coordinator always live on the same broker since they both
> map
> > > to
> > > > the same partition in the transaction topic. Even if there are
> > failures,
> > > > they end up on the same new broker. Hence, they share the same and
> > > > consistent view of the PIDs, epochs and App IDs, whatever it is. The
> > > > consumer coordinator will skip over the transaction metadata messages
> > > when
> > > > it bootstraps the offsets from this new topic for consumer groups
> that
> > > are
> > > > not involved in a transaction and don’t have a txn id associated with
> > the
> > > > offset message in the transaction topic. The consumer coordinator
> will
> > > > expose only committed offsets in cases of consumer groups that are
> > > involved
> > > > in a txn. It will also be able to validate the OffsetCommitRequests
> > > coming
> > > > from a transactional producer by ensuring that it is coming from a
> > valid
> > > > PID, producer epoch since it uses the same view of this data created
> by
> > > the
> > > > transaction coordinator (that lives on the same broker). And we will
> > end
> > > up
> > > > with one internal topic, not too.
> > > >
> > > > This proposal offers better operational simplicity and fewer internal
> > > > topics but there are some downsides that come with it — there are 2
> > types
> > > > of messages in one topic (txn metadata ones and offset ones). Since
> > this
> > > > internal topic serves a dual purpose, it will be harder to name it
> and
> > > also
> > > > design a message format that includes the different types of messages
> > > that
> > > > will live in the topic. Though the transaction topic already needs to
> > > write
> > > > 5 different types of messages (the AppID->PID mapping, the BeginTxn
> > > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe adding
> > the
> > > > offset message isn't a big deal?
> > > >
> > > > Back when we introduced the offsets topic, we had discussed making it
> > > more
> > > > general and allowing the producer to send offset commit messages to
> it
> > > but
> > > > ended up creating a specialized topic to allow the consumer
> coordinator
> > > to
> > > > wall off and prevent unauthorized writes from consumers outside of a
> > > group.
> > > > Jason can comment on the specifics but I don't believe that goal of
> the
> > > new
> > > > consumer protocol was quite achieved.
> > > >
> > > > I have other comments on the message format, request names etc but
> > wanted
> > > > to get your thoughts on these 2 issues first :-)
> > > >
> > > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > delivery
> > > > > semantics:
> > > > >
> > > > > *
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > <
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > >*
> > > > >
> > > > > This KIP adds a transactional messaging mechanism along with an
> > > > idempotent
> > > > > producer implementation to make sure that 1) duplicated messages
> sent
> > > > from
> > > > > the same identified producer can be detected on the broker side,
> and
> > > 2) a
> > > > > group of messages sent within a transaction will atomically be
> either
> > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > The above wiki page provides a high-level view of the proposed
> > changes
> > > as
> > > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > > design is described in this Google doc:
> > > > >
> > > > > https://docs.google.com/document/d/11Jqy_
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8
> > > > > <https://docs.google.com/document/d/11Jqy_
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8>
> > > > >
> > > > >
> > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > -- Guozhang
> > > > >
> > > > --
> > > > Thanks,
> > > > Neha
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
The first line in step 2 of that section is: "If there is already an entry
with the AppID in the mapping, increment the epoch number and go on to the
next step."

Are you suggesting that it be made explicit that 'increment the epoch
number' includes persisting the updated value to the log?

Thanks,
Apurva

On Wed, Nov 30, 2016 at 9:21 PM, Onur Karaman <on...@gmail.com>
wrote:

> Nice google doc!
>
> Probably need to go over the google doc a few more times, but a minor
> comment from the first pass:
>
> In Transaction Coordinator Request Handling (
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#bookmark=id.jro89lml46du),
> step 2 mentions that if the Transaction Coordinator doesn't already see a
> producer with the same app-id, it creates a pid and appends (app-id, pid,
> epoch) into the transaction log.
>
> What about if the app-id/pid pair already exists and we increment the
> epoch? Should we append (app-id, pid, epoch++) to the transaction log? I
> think we should, but step 2 doesn't mention this.
>
> On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Thanks for your comments, let me deal with your second point regarding
> > merging the __consumer-offsets and transactions topic.
> >
> > Needless to say, we considered doing this, but chose to keep them
> separate
> > for the following reasons:
> >
> >    1. Your assumption that group.id and transaction.app.id can be the
> same
> >    does not hold for streams applications. All colocated tasks of a
> streams
> >    application will share the same consumer (and hence implicitly will
> have
> >    the same group.id), but each task will have its own producer
> instance.
> >    The transaction.app.id for each producer instance will still have to
> be
> >    distinct. So to colocate the transaction and consumer group
> > coordinators,
> >    we will have to now introduce a 'group.id' config in the producer and
> >    require it to be the same as the consumer. This seemed like a very
> > fragile
> >    option.
> >    2. Following on from the above, the transaction coordinator and group
> >    coordinator would _have_ to be colocated inorder to be the leader for
> > the
> >    same TopicPartition, unless we wanted to make even more fundamental
> > changes
> >    to Kafka.
> >    3. We don't require that the consumer coordinator and the transaction
> >    coordinator have the same view of the current PID/Epoch pair. If a
> > producer
> >    instance is bounced, the epoch will be bumped. Any transactions
> > initiated
> >    by the previous instance would either be fully committed or fully
> rolled
> >    back. Since the writes to the offset topics are just like writes to a
> >    regular topic, these would enjoy the same guarantees, and the
> > inconsistency
> >    will be eventually resolved.
> >    4. Finally, every application will have consumers, and hence record
> >    consumer offsets. But a very small fraction of applications would use
> >    transactions. Blending the two topics would make recovering
> transaction
> >    coordinator state unnecessarily inefficient since it has to read from
> > the
> >    beginning of the topic to reconstruct its data structures -- it would
> > have
> >    to inspect and skip a majority of the messages if the offsets were in
> > the
> >    same topic.
> >
> > Thanks,
> > Apurva
> >
> > On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io>
> wrote:
> >
> > > Thanks for initiating this KIP! I think it is well written and I'm
> > excited
> > > to see the first step towards adding an important feature in Kafka.
> > >
> > > I had a few initial thoughts on the KIP, mostly not as deeply thought
> > > through than what you've done -
> > >
> > > 1. Perhaps you’ve thought about how this would work already — since we
> > now
> > > require a producer to specify a unique AppID across different instances
> > of
> > > an application, how would applications that run in the cloud use this
> > > feature with auto scaling?
> > >
> > > 2. Making it easy for applications to get exactly-once semantics for a
> > > consume-process-produce workflow is a great feature to have. To enable
> > > this, the proposal now includes letting a producer initiate a write to
> > the
> > > offset topic as well (just like consumers do). The consumer coordinator
> > > (which could be on a different broker than the txn coordinator) would
> > then
> > > validate if the PID and producer epoch is valid before it writes to the
> > > offset topic along with the associated PID. This is a great feature
> > though
> > > I see 2 difficulties
> > >
> > > -- This needs the consumer coordinator to have a consistent view of the
> > > PID/epochs that is same as the view on the txn coordinator. However, as
> > the
> > > offset and the transaction topics are different, the 2 coordinators
> might
> > > live on different brokers.
> > > -- We now also have 2 internal topics - a transaction topic and the
> > > __consumer_offsets topic.
> > >
> > > Maybe you’ve thought about this already and discarded it ... let me
> make
> > a
> > > somewhat crazy proposal — Why don’t we upgrade the transaction topic to
> > be
> > > the new offsets topic as well? For consumers that want EoS guarantees
> for
> > > a consume-process-produce pattern, the group.id is the same as the
> > > transaction.app.id set for the producer. Assume that the transaction
> > topic
> > > also stores consumer offsets. It stores both the transaction metadata
> > > messages as well as offset messages, both for transactional as well as
> > > non-transactional consumers. Since the group.id of the consumer and
> the
> > > app.id of the producer is the same, the offsets associated with a
> > consumer
> > > group and topic-partition end up in the same transaction topic
> partition
> > as
> > > the transaction metadata messages. The transaction coordinator and the
> > > consumer coordinator always live on the same broker since they both map
> > to
> > > the same partition in the transaction topic. Even if there are
> failures,
> > > they end up on the same new broker. Hence, they share the same and
> > > consistent view of the PIDs, epochs and App IDs, whatever it is. The
> > > consumer coordinator will skip over the transaction metadata messages
> > when
> > > it bootstraps the offsets from this new topic for consumer groups that
> > are
> > > not involved in a transaction and don’t have a txn id associated with
> the
> > > offset message in the transaction topic. The consumer coordinator will
> > > expose only committed offsets in cases of consumer groups that are
> > involved
> > > in a txn. It will also be able to validate the OffsetCommitRequests
> > coming
> > > from a transactional producer by ensuring that it is coming from a
> valid
> > > PID, producer epoch since it uses the same view of this data created by
> > the
> > > transaction coordinator (that lives on the same broker). And we will
> end
> > up
> > > with one internal topic, not too.
> > >
> > > This proposal offers better operational simplicity and fewer internal
> > > topics but there are some downsides that come with it — there are 2
> types
> > > of messages in one topic (txn metadata ones and offset ones). Since
> this
> > > internal topic serves a dual purpose, it will be harder to name it and
> > also
> > > design a message format that includes the different types of messages
> > that
> > > will live in the topic. Though the transaction topic already needs to
> > write
> > > 5 different types of messages (the AppID->PID mapping, the BeginTxn
> > > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe adding
> the
> > > offset message isn't a big deal?
> > >
> > > Back when we introduced the offsets topic, we had discussed making it
> > more
> > > general and allowing the producer to send offset commit messages to it
> > but
> > > ended up creating a specialized topic to allow the consumer coordinator
> > to
> > > wall off and prevent unauthorized writes from consumers outside of a
> > group.
> > > Jason can comment on the specifics but I don't believe that goal of the
> > new
> > > consumer protocol was quite achieved.
> > >
> > > I have other comments on the message format, request names etc but
> wanted
> > > to get your thoughts on these 2 issues first :-)
> > >
> > > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > > <https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8>
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > > --
> > > Thanks,
> > > Neha
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Onur Karaman <on...@gmail.com>.
Nice google doc!

Probably need to go over the google doc a few more times, but a minor
comment from the first pass:

In Transaction Coordinator Request Handling (
https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.jro89lml46du),
step 2 mentions that if the Transaction Coordinator doesn't already see a
producer with the same app-id, it creates a pid and appends (app-id, pid,
epoch) into the transaction log.

What about if the app-id/pid pair already exists and we increment the
epoch? Should we append (app-id, pid, epoch++) to the transaction log? I
think we should, but step 2 doesn't mention this.

On Wed, Nov 30, 2016 at 5:35 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Thanks for your comments, let me deal with your second point regarding
> merging the __consumer-offsets and transactions topic.
>
> Needless to say, we considered doing this, but chose to keep them separate
> for the following reasons:
>
>    1. Your assumption that group.id and transaction.app.id can be the same
>    does not hold for streams applications. All colocated tasks of a streams
>    application will share the same consumer (and hence implicitly will have
>    the same group.id), but each task will have its own producer instance.
>    The transaction.app.id for each producer instance will still have to be
>    distinct. So to colocate the transaction and consumer group
> coordinators,
>    we will have to now introduce a 'group.id' config in the producer and
>    require it to be the same as the consumer. This seemed like a very
> fragile
>    option.
>    2. Following on from the above, the transaction coordinator and group
>    coordinator would _have_ to be colocated inorder to be the leader for
> the
>    same TopicPartition, unless we wanted to make even more fundamental
> changes
>    to Kafka.
>    3. We don't require that the consumer coordinator and the transaction
>    coordinator have the same view of the current PID/Epoch pair. If a
> producer
>    instance is bounced, the epoch will be bumped. Any transactions
> initiated
>    by the previous instance would either be fully committed or fully rolled
>    back. Since the writes to the offset topics are just like writes to a
>    regular topic, these would enjoy the same guarantees, and the
> inconsistency
>    will be eventually resolved.
>    4. Finally, every application will have consumers, and hence record
>    consumer offsets. But a very small fraction of applications would use
>    transactions. Blending the two topics would make recovering transaction
>    coordinator state unnecessarily inefficient since it has to read from
> the
>    beginning of the topic to reconstruct its data structures -- it would
> have
>    to inspect and skip a majority of the messages if the offsets were in
> the
>    same topic.
>
> Thanks,
> Apurva
>
> On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io> wrote:
>
> > Thanks for initiating this KIP! I think it is well written and I'm
> excited
> > to see the first step towards adding an important feature in Kafka.
> >
> > I had a few initial thoughts on the KIP, mostly not as deeply thought
> > through than what you've done -
> >
> > 1. Perhaps you’ve thought about how this would work already — since we
> now
> > require a producer to specify a unique AppID across different instances
> of
> > an application, how would applications that run in the cloud use this
> > feature with auto scaling?
> >
> > 2. Making it easy for applications to get exactly-once semantics for a
> > consume-process-produce workflow is a great feature to have. To enable
> > this, the proposal now includes letting a producer initiate a write to
> the
> > offset topic as well (just like consumers do). The consumer coordinator
> > (which could be on a different broker than the txn coordinator) would
> then
> > validate if the PID and producer epoch is valid before it writes to the
> > offset topic along with the associated PID. This is a great feature
> though
> > I see 2 difficulties
> >
> > -- This needs the consumer coordinator to have a consistent view of the
> > PID/epochs that is same as the view on the txn coordinator. However, as
> the
> > offset and the transaction topics are different, the 2 coordinators might
> > live on different brokers.
> > -- We now also have 2 internal topics - a transaction topic and the
> > __consumer_offsets topic.
> >
> > Maybe you’ve thought about this already and discarded it ... let me make
> a
> > somewhat crazy proposal — Why don’t we upgrade the transaction topic to
> be
> > the new offsets topic as well? For consumers that want EoS guarantees for
> > a consume-process-produce pattern, the group.id is the same as the
> > transaction.app.id set for the producer. Assume that the transaction
> topic
> > also stores consumer offsets. It stores both the transaction metadata
> > messages as well as offset messages, both for transactional as well as
> > non-transactional consumers. Since the group.id of the consumer and the
> > app.id of the producer is the same, the offsets associated with a
> consumer
> > group and topic-partition end up in the same transaction topic partition
> as
> > the transaction metadata messages. The transaction coordinator and the
> > consumer coordinator always live on the same broker since they both map
> to
> > the same partition in the transaction topic. Even if there are failures,
> > they end up on the same new broker. Hence, they share the same and
> > consistent view of the PIDs, epochs and App IDs, whatever it is. The
> > consumer coordinator will skip over the transaction metadata messages
> when
> > it bootstraps the offsets from this new topic for consumer groups that
> are
> > not involved in a transaction and don’t have a txn id associated with the
> > offset message in the transaction topic. The consumer coordinator will
> > expose only committed offsets in cases of consumer groups that are
> involved
> > in a txn. It will also be able to validate the OffsetCommitRequests
> coming
> > from a transactional producer by ensuring that it is coming from a valid
> > PID, producer epoch since it uses the same view of this data created by
> the
> > transaction coordinator (that lives on the same broker). And we will end
> up
> > with one internal topic, not too.
> >
> > This proposal offers better operational simplicity and fewer internal
> > topics but there are some downsides that come with it — there are 2 types
> > of messages in one topic (txn metadata ones and offset ones). Since this
> > internal topic serves a dual purpose, it will be harder to name it and
> also
> > design a message format that includes the different types of messages
> that
> > will live in the topic. Though the transaction topic already needs to
> write
> > 5 different types of messages (the AppID->PID mapping, the BeginTxn
> > message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe adding the
> > offset message isn't a big deal?
> >
> > Back when we introduced the offsets topic, we had discussed making it
> more
> > general and allowing the producer to send offset commit messages to it
> but
> > ended up creating a specialized topic to allow the consumer coordinator
> to
> > wall off and prevent unauthorized writes from consumers outside of a
> group.
> > Jason can comment on the specifics but I don't believe that goal of the
> new
> > consumer protocol was quite achieved.
> >
> > I have other comments on the message format, request names etc but wanted
> > to get your thoughts on these 2 issues first :-)
> >
> > On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > > <https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8>
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > --
> > Thanks,
> > Neha
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Thanks for your comments, let me deal with your second point regarding
merging the __consumer-offsets and transactions topic.

Needless to say, we considered doing this, but chose to keep them separate
for the following reasons:

   1. Your assumption that group.id and transaction.app.id can be the same
   does not hold for streams applications. All colocated tasks of a streams
   application will share the same consumer (and hence implicitly will have
   the same group.id), but each task will have its own producer instance.
   The transaction.app.id for each producer instance will still have to be
   distinct. So to colocate the transaction and consumer group coordinators,
   we will have to now introduce a 'group.id' config in the producer and
   require it to be the same as the consumer. This seemed like a very fragile
   option.
   2. Following on from the above, the transaction coordinator and group
   coordinator would _have_ to be colocated inorder to be the leader for the
   same TopicPartition, unless we wanted to make even more fundamental changes
   to Kafka.
   3. We don't require that the consumer coordinator and the transaction
   coordinator have the same view of the current PID/Epoch pair. If a producer
   instance is bounced, the epoch will be bumped. Any transactions initiated
   by the previous instance would either be fully committed or fully rolled
   back. Since the writes to the offset topics are just like writes to a
   regular topic, these would enjoy the same guarantees, and the inconsistency
   will be eventually resolved.
   4. Finally, every application will have consumers, and hence record
   consumer offsets. But a very small fraction of applications would use
   transactions. Blending the two topics would make recovering transaction
   coordinator state unnecessarily inefficient since it has to read from the
   beginning of the topic to reconstruct its data structures -- it would have
   to inspect and skip a majority of the messages if the offsets were in the
   same topic.

Thanks,
Apurva

On Wed, Nov 30, 2016 at 4:47 PM, Neha Narkhede <ne...@confluent.io> wrote:

> Thanks for initiating this KIP! I think it is well written and I'm excited
> to see the first step towards adding an important feature in Kafka.
>
> I had a few initial thoughts on the KIP, mostly not as deeply thought
> through than what you've done -
>
> 1. Perhaps you’ve thought about how this would work already — since we now
> require a producer to specify a unique AppID across different instances of
> an application, how would applications that run in the cloud use this
> feature with auto scaling?
>
> 2. Making it easy for applications to get exactly-once semantics for a
> consume-process-produce workflow is a great feature to have. To enable
> this, the proposal now includes letting a producer initiate a write to the
> offset topic as well (just like consumers do). The consumer coordinator
> (which could be on a different broker than the txn coordinator) would then
> validate if the PID and producer epoch is valid before it writes to the
> offset topic along with the associated PID. This is a great feature though
> I see 2 difficulties
>
> -- This needs the consumer coordinator to have a consistent view of the
> PID/epochs that is same as the view on the txn coordinator. However, as the
> offset and the transaction topics are different, the 2 coordinators might
> live on different brokers.
> -- We now also have 2 internal topics - a transaction topic and the
> __consumer_offsets topic.
>
> Maybe you’ve thought about this already and discarded it ... let me make a
> somewhat crazy proposal — Why don’t we upgrade the transaction topic to be
> the new offsets topic as well? For consumers that want EoS guarantees for
> a consume-process-produce pattern, the group.id is the same as the
> transaction.app.id set for the producer. Assume that the transaction topic
> also stores consumer offsets. It stores both the transaction metadata
> messages as well as offset messages, both for transactional as well as
> non-transactional consumers. Since the group.id of the consumer and the
> app.id of the producer is the same, the offsets associated with a consumer
> group and topic-partition end up in the same transaction topic partition as
> the transaction metadata messages. The transaction coordinator and the
> consumer coordinator always live on the same broker since they both map to
> the same partition in the transaction topic. Even if there are failures,
> they end up on the same new broker. Hence, they share the same and
> consistent view of the PIDs, epochs and App IDs, whatever it is. The
> consumer coordinator will skip over the transaction metadata messages when
> it bootstraps the offsets from this new topic for consumer groups that are
> not involved in a transaction and don’t have a txn id associated with the
> offset message in the transaction topic. The consumer coordinator will
> expose only committed offsets in cases of consumer groups that are involved
> in a txn. It will also be able to validate the OffsetCommitRequests coming
> from a transactional producer by ensuring that it is coming from a valid
> PID, producer epoch since it uses the same view of this data created by the
> transaction coordinator (that lives on the same broker). And we will end up
> with one internal topic, not too.
>
> This proposal offers better operational simplicity and fewer internal
> topics but there are some downsides that come with it — there are 2 types
> of messages in one topic (txn metadata ones and offset ones). Since this
> internal topic serves a dual purpose, it will be harder to name it and also
> design a message format that includes the different types of messages that
> will live in the topic. Though the transaction topic already needs to write
> 5 different types of messages (the AppID->PID mapping, the BeginTxn
> message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe adding the
> offset message isn't a big deal?
>
> Back when we introduced the offsets topic, we had discussed making it more
> general and allowing the producer to send offset commit messages to it but
> ended up creating a specialized topic to allow the consumer coordinator to
> wall off and prevent unauthorized writes from consumers outside of a group.
> Jason can comment on the specifics but I don't believe that goal of the new
> consumer protocol was quite achieved.
>
> I have other comments on the message format, request names etc but wanted
> to get your thoughts on these 2 issues first :-)
>
> On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> > <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8>
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
> --
> Thanks,
> Neha
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Neha Narkhede <ne...@confluent.io>.
Thanks for initiating this KIP! I think it is well written and I'm excited
to see the first step towards adding an important feature in Kafka.

I had a few initial thoughts on the KIP, mostly not as deeply thought
through than what you've done -

1. Perhaps you’ve thought about how this would work already — since we now
require a producer to specify a unique AppID across different instances of
an application, how would applications that run in the cloud use this
feature with auto scaling?

2. Making it easy for applications to get exactly-once semantics for a
consume-process-produce workflow is a great feature to have. To enable
this, the proposal now includes letting a producer initiate a write to the
offset topic as well (just like consumers do). The consumer coordinator
(which could be on a different broker than the txn coordinator) would then
validate if the PID and producer epoch is valid before it writes to the
offset topic along with the associated PID. This is a great feature though
I see 2 difficulties

-- This needs the consumer coordinator to have a consistent view of the
PID/epochs that is same as the view on the txn coordinator. However, as the
offset and the transaction topics are different, the 2 coordinators might
live on different brokers.
-- We now also have 2 internal topics - a transaction topic and the
__consumer_offsets topic.

Maybe you’ve thought about this already and discarded it ... let me make a
somewhat crazy proposal — Why don’t we upgrade the transaction topic to be
the new offsets topic as well? For consumers that want EoS guarantees for
a consume-process-produce pattern, the group.id is the same as the
transaction.app.id set for the producer. Assume that the transaction topic
also stores consumer offsets. It stores both the transaction metadata
messages as well as offset messages, both for transactional as well as
non-transactional consumers. Since the group.id of the consumer and the
app.id of the producer is the same, the offsets associated with a consumer
group and topic-partition end up in the same transaction topic partition as
the transaction metadata messages. The transaction coordinator and the
consumer coordinator always live on the same broker since they both map to
the same partition in the transaction topic. Even if there are failures,
they end up on the same new broker. Hence, they share the same and
consistent view of the PIDs, epochs and App IDs, whatever it is. The
consumer coordinator will skip over the transaction metadata messages when
it bootstraps the offsets from this new topic for consumer groups that are
not involved in a transaction and don’t have a txn id associated with the
offset message in the transaction topic. The consumer coordinator will
expose only committed offsets in cases of consumer groups that are involved
in a txn. It will also be able to validate the OffsetCommitRequests coming
from a transactional producer by ensuring that it is coming from a valid
PID, producer epoch since it uses the same view of this data created by the
transaction coordinator (that lives on the same broker). And we will end up
with one internal topic, not too.

This proposal offers better operational simplicity and fewer internal
topics but there are some downsides that come with it — there are 2 types
of messages in one topic (txn metadata ones and offset ones). Since this
internal topic serves a dual purpose, it will be harder to name it and also
design a message format that includes the different types of messages that
will live in the topic. Though the transaction topic already needs to write
5 different types of messages (the AppID->PID mapping, the BeginTxn
message, InsertTxn, PrepareCommit, Committed/Aborted) so maybe adding the
offset message isn't a big deal?

Back when we introduced the offsets topic, we had discussed making it more
general and allowing the producer to send offset commit messages to it but
ended up creating a specialized topic to allow the consumer coordinator to
wall off and prevent unauthorized writes from consumers outside of a group.
Jason can comment on the specifics but I don't believe that goal of the new
consumer protocol was quite achieved.

I have other comments on the message format, request names etc but wanted
to get your thoughts on these 2 issues first :-)

On Wed, Nov 30, 2016 at 2:19 PM Guozhang Wang <wa...@gmail.com> wrote:

> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8>
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>
-- 
Thanks,
Neha

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Hi all

So having gone through a few extra failure scenarios it appears it is still
possible for logs to diverge if the unclean.leader.election setting is
enabled. The protocol could be evolved further to protect against this. The
issue is that it adds significant complexity, and potentially impacts other
primitives like log compaction. As a result the most pragmatic solution is
to *limit the guarantees this KIP provides to clusters where unclean leader
election is disabled*.

If anyone has any strong feelings on this, or useful insights, that would
be awesome. Otherwise I'll update the KIP to reflect this stance (along
with the example below).

All the best
B

*Divergent Logs with Leader Epochs & Unclean Leader Election*
It should be possible to still corrupt the log, even with Leader epochs, if
min.isr=1 and unclean.leader.election=true. Consider two brokers A,B, a
single topic, a single partition, reps=2, min.isr=1.

Intuitively the issue can be seen as:
-> The first two writes create a divergent log at offset 0 on completely
isolated brokers.
-> The second two writes “cover up” that first divergent write so the
LeaderEpoch request doesn’t see it.

Scenario:
1. [LeaderEpoch0] Write a message to A (offset A:0), Stop broker A. Bring
up broker B which becomes leader
2. [LeaderEpoch1] Write a message to B (offset B:0), Stop broker B. Bring
up broker A which becomes leader
3. [LeaderEpoch2] Write a message to A (offset A:1), Stop broker A. Bring
up broker B which becomes leader
4. [LeaderEpoch3] Write a message to B (offset B:1),
5. Bring up broker A. It sends a Epoch Request for Epoch 2 to broker B. B
has only epochs 1,3, not 2, so it replies with the first offset of Epoch 3
(which is 1). So offset 0 is divergent.

The underlying problem here is that, whilst B can tell something is wrong,
it can't tell where in the log the divergence started.

One solution is to detect the break, by comparing complete epoch lineage
between brokers, then truncate either to (a) zero or (b) the point of
divergence, then refetch. However compacted topics make both of these
options hard as arbitrary epochs & offset information can be 'lost' from
the log. This information could be retained and managed in the LeaderEpoch
file instead, but the whole solution is becoming quite complex. Hence it
seems sensible to forgo this guarantee for the unclean leader election
case, or at least push it to a subsequent kip.


On Wed, Dec 14, 2016 at 6:45 PM Jun Rao <ju...@confluent.io> wrote:

Hi, Onur,

The reason for keeping track of the CZXID of the broker registration path
is the following. There is one corner case bug (KAFKA-1120) that Ben
mentioned where the controller could miss a ZK watcher event if the broker
deregisters and registers quickly. Always triggering a leader election (and
thus increasing the leader epoch) on broker registration event may work,
but we have to think through the controller failover logic. When the
controller initializes, it simply reads all current broker registration
from ZK. The controller doesn't know whether any broker registration has
changed since the previous controller has failed. Just blindly forcing
leader election on all partitions during the controller failover probably
adds too much overhead.

So, the idea is to have the broker tracks the broker -> CZXID mapping in
memory. Every time the controller changes the leader for a partition, the
controller stores the CZXID of the leader together with the leader broker
id (and leader epoch, controller epoch etc) in memory and in
/brokers/topics/[topic]/partitions/[partitionId]/state
(this is missing in the KIP wiki). Now if the controller gets a broker
registration event or when there is a controller failover, the controller
just needs to force a leader election if the CZXID of the broker
registration doesn't match the CZXID associated with the leader in
/brokers/topics/[topic]/partitions/[partitionId]/state.
This way, we will only do leader election when it's truly necessary.

The reason why this change is related to this KIP is that it also addresses
the issue of keeping the replicas identical during correlated failures. If
all replicas are down and the leader replica is the first being restarted,
by forcing the increase of leader epoch even though the leader remains on
the same replica, we can distinguish the data written since the leader
replica is restarted from those written by the same leader replica before
it's restarted. This allows us to maintain all replicas to be identical
even in the correlated failure case.

Thanks,

Jun

On Sun, Dec 11, 2016 at 3:54 PM, Onur Karaman <on...@gmail.com>
wrote:

> Pretty happy to see a KIP tackling this problem! One comment below.
>
> The "Extending LeaderEpoch to include Returning Leaders" states:
> "To protect against this eventuality the controller will maintain a cached
> mapping of [broker -> Zookeeper CZXID] (CZXID is a unique and monotonic
> 64-bit number) for the broker’s registration in Zookeeper
> (/brokers/ids/[brokerId]). If the controller receives a Broker
Registration
> where the CZXID has changed it will increment the Leader Epoch and
> propagate that value to the broker via the Leader and ISR Request (in the
> normal way), then update the cached CZXID for that broker."
>
> In general I think kafka underutilizes zookeeper's various flavors of
zxids
> but this time it's not clear to me what the motivation is for maintaining
> the broker to czxid mapping. It seems that the following check is
> redundant: "If the controller receives a Broker Registration where the
> CZXID has changed". By definition, the czxid of the
/brokers/ids/[brokerId]
> znode will always change upon successful broker registration (
> https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_
> zkStatStructure).
> Why maintain the mapping at all? Why not just always update leader epochs
> and propagate every time the controller receives the broker registration
zk
> event?
>
> On Sun, Dec 11, 2016 at 2:30 PM, Neha Narkhede <ne...@confluent.io> wrote:
>
> > Good to see this KIP being proposed. Back when I added the epoch to the
> > replication protocol, we discussed adding it to the log due to the
> failure
> > scenarios listed in the KIP but I failed to convince people that it was
> > worth the effort needed to upgrade the cluster (especially after we
asked
> > people to go through a painful backwards incompatible upgrade for 0.8
> :-))
> > The lack of including the leader epoch/generation in the log has also
> been
> > one of the biggest critiques of Kafka's replication protocol by the
> > distributed systems community.
> >
> > I'm in favor of this work though I think we shouldn't end up with 2
> notions
> > of representing a leader's generation. When we added the epoch, we
wanted
> > to add it to the log but we didn't. Now that we are adding the
generation
> > id to the log, I think we should revisit calling it the epoch at all.
> Have
> > you thought about a way to evolve the epoch to the generation id
> throughout
> > and what it will take?
> >
> > On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:
> >
> > > Hi All
> > >
> > > Please find the below KIP which describes a proposed solution to a
> couple
> > > of issues that have been observed with the replication protocol.
> > >
> > > In short, the proposal replaces the use of the High Watermark, for
> > > follower log trunctation, with an alternate Generation Marker. This
> > > uniquely defines which leader messages were acknowledged by.
> > >
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
> > Generation+rather+than+High+Watermark+for+Truncation
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
> > Generation+rather+than+High+Watermark+for+Truncation
> > > >
> > >
> > > All comments and suggestions greatly appreciated.
> > >
> > > Ben Stopford
> > > Confluent, http://www.confluent.io <http://www.confluent.io/>
> > >
> > > --
> > Thanks,
> > Neha
> >
>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Jun Rao <ju...@confluent.io>.
Hi, Onur,

The reason for keeping track of the CZXID of the broker registration path
is the following. There is one corner case bug (KAFKA-1120) that Ben
mentioned where the controller could miss a ZK watcher event if the broker
deregisters and registers quickly. Always triggering a leader election (and
thus increasing the leader epoch) on broker registration event may work,
but we have to think through the controller failover logic. When the
controller initializes, it simply reads all current broker registration
from ZK. The controller doesn't know whether any broker registration has
changed since the previous controller has failed. Just blindly forcing
leader election on all partitions during the controller failover probably
adds too much overhead.

So, the idea is to have the broker tracks the broker -> CZXID mapping in
memory. Every time the controller changes the leader for a partition, the
controller stores the CZXID of the leader together with the leader broker
id (and leader epoch, controller epoch etc) in memory and in
/brokers/topics/[topic]/partitions/[partitionId]/state
(this is missing in the KIP wiki). Now if the controller gets a broker
registration event or when there is a controller failover, the controller
just needs to force a leader election if the CZXID of the broker
registration doesn't match the CZXID associated with the leader in
/brokers/topics/[topic]/partitions/[partitionId]/state.
This way, we will only do leader election when it's truly necessary.

The reason why this change is related to this KIP is that it also addresses
the issue of keeping the replicas identical during correlated failures. If
all replicas are down and the leader replica is the first being restarted,
by forcing the increase of leader epoch even though the leader remains on
the same replica, we can distinguish the data written since the leader
replica is restarted from those written by the same leader replica before
it's restarted. This allows us to maintain all replicas to be identical
even in the correlated failure case.

Thanks,

Jun

On Sun, Dec 11, 2016 at 3:54 PM, Onur Karaman <on...@gmail.com>
wrote:

> Pretty happy to see a KIP tackling this problem! One comment below.
>
> The "Extending LeaderEpoch to include Returning Leaders" states:
> "To protect against this eventuality the controller will maintain a cached
> mapping of [broker -> Zookeeper CZXID] (CZXID is a unique and monotonic
> 64-bit number) for the broker’s registration in Zookeeper
> (/brokers/ids/[brokerId]). If the controller receives a Broker Registration
> where the CZXID has changed it will increment the Leader Epoch and
> propagate that value to the broker via the Leader and ISR Request (in the
> normal way), then update the cached CZXID for that broker."
>
> In general I think kafka underutilizes zookeeper's various flavors of zxids
> but this time it's not clear to me what the motivation is for maintaining
> the broker to czxid mapping. It seems that the following check is
> redundant: "If the controller receives a Broker Registration where the
> CZXID has changed". By definition, the czxid of the /brokers/ids/[brokerId]
> znode will always change upon successful broker registration (
> https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_
> zkStatStructure).
> Why maintain the mapping at all? Why not just always update leader epochs
> and propagate every time the controller receives the broker registration zk
> event?
>
> On Sun, Dec 11, 2016 at 2:30 PM, Neha Narkhede <ne...@confluent.io> wrote:
>
> > Good to see this KIP being proposed. Back when I added the epoch to the
> > replication protocol, we discussed adding it to the log due to the
> failure
> > scenarios listed in the KIP but I failed to convince people that it was
> > worth the effort needed to upgrade the cluster (especially after we asked
> > people to go through a painful backwards incompatible upgrade for 0.8
> :-))
> > The lack of including the leader epoch/generation in the log has also
> been
> > one of the biggest critiques of Kafka's replication protocol by the
> > distributed systems community.
> >
> > I'm in favor of this work though I think we shouldn't end up with 2
> notions
> > of representing a leader's generation. When we added the epoch, we wanted
> > to add it to the log but we didn't. Now that we are adding the generation
> > id to the log, I think we should revisit calling it the epoch at all.
> Have
> > you thought about a way to evolve the epoch to the generation id
> throughout
> > and what it will take?
> >
> > On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:
> >
> > > Hi All
> > >
> > > Please find the below KIP which describes a proposed solution to a
> couple
> > > of issues that have been observed with the replication protocol.
> > >
> > > In short, the proposal replaces the use of the High Watermark, for
> > > follower log trunctation, with an alternate Generation Marker. This
> > > uniquely defines which leader messages were acknowledged by.
> > >
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
> > Generation+rather+than+High+Watermark+for+Truncation
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
> > Generation+rather+than+High+Watermark+for+Truncation
> > > >
> > >
> > > All comments and suggestions greatly appreciated.
> > >
> > > Ben Stopford
> > > Confluent, http://www.confluent.io <http://www.confluent.io/>
> > >
> > > --
> > Thanks,
> > Neha
> >
>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
To clarify slightly, the case described in the KIP doesn’t necessitate an extra mapping to the CZXID. But there is an issue filed against the controller, which would also affect the LeaderGeneration correctness. The suggested fix for this includes such a mapping, according to Jun’s reasoning in the Jira comments: https://issues.apache.org/jira/browse/KAFKA-1120 <https://issues.apache.org/jira/browse/KAFKA-1120>. Strictly speaking this is a separate issue though and I’ve updated the KIP accordingly. 

B
Ben Stopford
Confluent, http://www.confluent.io <http://www.confluent.io/>



> On 14 Dec 2016, at 11:37, Ben Stopford <be...@confluent.io> wrote:
> 
> Thanks Onur. That’s correct, we no longer nee that extra mapping. I’ll update the KIP. 
> B
> 
> Ben Stopford
> Confluent, http://www.confluent.io <http://www.confluent.io/>
> 
> 
> 
>> On 11 Dec 2016, at 23:54, Onur Karaman <onurkaraman.apache@gmail.com <ma...@gmail.com>> wrote:
>> 
>> Pretty happy to see a KIP tackling this problem! One comment below.
>> 
>> The "Extending LeaderEpoch to include Returning Leaders" states:
>> "To protect against this eventuality the controller will maintain a cached
>> mapping of [broker -> Zookeeper CZXID] (CZXID is a unique and monotonic
>> 64-bit number) for the broker’s registration in Zookeeper
>> (/brokers/ids/[brokerId]). If the controller receives a Broker Registration
>> where the CZXID has changed it will increment the Leader Epoch and
>> propagate that value to the broker via the Leader and ISR Request (in the
>> normal way), then update the cached CZXID for that broker."
>> 
>> In general I think kafka underutilizes zookeeper's various flavors of zxids
>> but this time it's not clear to me what the motivation is for maintaining
>> the broker to czxid mapping. It seems that the following check is
>> redundant: "If the controller receives a Broker Registration where the
>> CZXID has changed". By definition, the czxid of the /brokers/ids/[brokerId]
>> znode will always change upon successful broker registration (
>> https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_zkStatStructure <https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_zkStatStructure>).
>> Why maintain the mapping at all? Why not just always update leader epochs
>> and propagate every time the controller receives the broker registration zk
>> event?
>> 
>> On Sun, Dec 11, 2016 at 2:30 PM, Neha Narkhede <neha@confluent.io <ma...@confluent.io>> wrote:
>> 
>>> Good to see this KIP being proposed. Back when I added the epoch to the
>>> replication protocol, we discussed adding it to the log due to the failure
>>> scenarios listed in the KIP but I failed to convince people that it was
>>> worth the effort needed to upgrade the cluster (especially after we asked
>>> people to go through a painful backwards incompatible upgrade for 0.8 :-))
>>> The lack of including the leader epoch/generation in the log has also been
>>> one of the biggest critiques of Kafka's replication protocol by the
>>> distributed systems community.
>>> 
>>> I'm in favor of this work though I think we shouldn't end up with 2 notions
>>> of representing a leader's generation. When we added the epoch, we wanted
>>> to add it to the log but we didn't. Now that we are adding the generation
>>> id to the log, I think we should revisit calling it the epoch at all. Have
>>> you thought about a way to evolve the epoch to the generation id throughout
>>> and what it will take?
>>> 
>>> On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <ben@confluent.io <ma...@confluent.io>> wrote:
>>> 
>>>> Hi All
>>>> 
>>>> Please find the below KIP which describes a proposed solution to a couple
>>>> of issues that have been observed with the replication protocol.
>>>> 
>>>> In short, the proposal replaces the use of the High Watermark, for
>>>> follower log trunctation, with an alternate Generation Marker. This
>>>> uniquely defines which leader messages were acknowledged by.
>>>> 
>>>> 
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>> 101+-+Alter+Replication+Protocol+to+use+Leader+
>>> Generation+rather+than+High+Watermark+for+Truncation
>>>> <
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>> 101+-+Alter+Replication+Protocol+to+use+Leader+
>>> Generation+rather+than+High+Watermark+for+Truncation
>>>>> 
>>>> 
>>>> All comments and suggestions greatly appreciated.
>>>> 
>>>> Ben Stopford
>>>> Confluent, http://www.confluent.io <http://www.confluent.io/> <http://www.confluent.io/ <http://www.confluent.io/>>
>>>> 
>>>> --
>>> Thanks,
>>> Neha
>>> 
> 


Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Thanks Onur. That’s correct, we no longer nee that extra mapping. I’ll update the KIP. 
B

Ben Stopford
Confluent, http://www.confluent.io <http://www.confluent.io/>



> On 11 Dec 2016, at 23:54, Onur Karaman <on...@gmail.com> wrote:
> 
> Pretty happy to see a KIP tackling this problem! One comment below.
> 
> The "Extending LeaderEpoch to include Returning Leaders" states:
> "To protect against this eventuality the controller will maintain a cached
> mapping of [broker -> Zookeeper CZXID] (CZXID is a unique and monotonic
> 64-bit number) for the broker’s registration in Zookeeper
> (/brokers/ids/[brokerId]). If the controller receives a Broker Registration
> where the CZXID has changed it will increment the Leader Epoch and
> propagate that value to the broker via the Leader and ISR Request (in the
> normal way), then update the cached CZXID for that broker."
> 
> In general I think kafka underutilizes zookeeper's various flavors of zxids
> but this time it's not clear to me what the motivation is for maintaining
> the broker to czxid mapping. It seems that the following check is
> redundant: "If the controller receives a Broker Registration where the
> CZXID has changed". By definition, the czxid of the /brokers/ids/[brokerId]
> znode will always change upon successful broker registration (
> https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_zkStatStructure).
> Why maintain the mapping at all? Why not just always update leader epochs
> and propagate every time the controller receives the broker registration zk
> event?
> 
> On Sun, Dec 11, 2016 at 2:30 PM, Neha Narkhede <ne...@confluent.io> wrote:
> 
>> Good to see this KIP being proposed. Back when I added the epoch to the
>> replication protocol, we discussed adding it to the log due to the failure
>> scenarios listed in the KIP but I failed to convince people that it was
>> worth the effort needed to upgrade the cluster (especially after we asked
>> people to go through a painful backwards incompatible upgrade for 0.8 :-))
>> The lack of including the leader epoch/generation in the log has also been
>> one of the biggest critiques of Kafka's replication protocol by the
>> distributed systems community.
>> 
>> I'm in favor of this work though I think we shouldn't end up with 2 notions
>> of representing a leader's generation. When we added the epoch, we wanted
>> to add it to the log but we didn't. Now that we are adding the generation
>> id to the log, I think we should revisit calling it the epoch at all. Have
>> you thought about a way to evolve the epoch to the generation id throughout
>> and what it will take?
>> 
>> On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:
>> 
>>> Hi All
>>> 
>>> Please find the below KIP which describes a proposed solution to a couple
>>> of issues that have been observed with the replication protocol.
>>> 
>>> In short, the proposal replaces the use of the High Watermark, for
>>> follower log trunctation, with an alternate Generation Marker. This
>>> uniquely defines which leader messages were acknowledged by.
>>> 
>>> 
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 101+-+Alter+Replication+Protocol+to+use+Leader+
>> Generation+rather+than+High+Watermark+for+Truncation
>>> <
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 101+-+Alter+Replication+Protocol+to+use+Leader+
>> Generation+rather+than+High+Watermark+for+Truncation
>>>> 
>>> 
>>> All comments and suggestions greatly appreciated.
>>> 
>>> Ben Stopford
>>> Confluent, http://www.confluent.io <http://www.confluent.io/>
>>> 
>>> --
>> Thanks,
>> Neha
>> 


Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Onur Karaman <on...@gmail.com>.
Pretty happy to see a KIP tackling this problem! One comment below.

The "Extending LeaderEpoch to include Returning Leaders" states:
"To protect against this eventuality the controller will maintain a cached
mapping of [broker -> Zookeeper CZXID] (CZXID is a unique and monotonic
64-bit number) for the broker’s registration in Zookeeper
(/brokers/ids/[brokerId]). If the controller receives a Broker Registration
where the CZXID has changed it will increment the Leader Epoch and
propagate that value to the broker via the Leader and ISR Request (in the
normal way), then update the cached CZXID for that broker."

In general I think kafka underutilizes zookeeper's various flavors of zxids
but this time it's not clear to me what the motivation is for maintaining
the broker to czxid mapping. It seems that the following check is
redundant: "If the controller receives a Broker Registration where the
CZXID has changed". By definition, the czxid of the /brokers/ids/[brokerId]
znode will always change upon successful broker registration (
https://zookeeper.apache.org/doc/r3.4.8/zookeeperProgrammers.html#sc_zkStatStructure).
Why maintain the mapping at all? Why not just always update leader epochs
and propagate every time the controller receives the broker registration zk
event?

On Sun, Dec 11, 2016 at 2:30 PM, Neha Narkhede <ne...@confluent.io> wrote:

> Good to see this KIP being proposed. Back when I added the epoch to the
> replication protocol, we discussed adding it to the log due to the failure
> scenarios listed in the KIP but I failed to convince people that it was
> worth the effort needed to upgrade the cluster (especially after we asked
> people to go through a painful backwards incompatible upgrade for 0.8 :-))
> The lack of including the leader epoch/generation in the log has also been
> one of the biggest critiques of Kafka's replication protocol by the
> distributed systems community.
>
> I'm in favor of this work though I think we shouldn't end up with 2 notions
> of representing a leader's generation. When we added the epoch, we wanted
> to add it to the log but we didn't. Now that we are adding the generation
> id to the log, I think we should revisit calling it the epoch at all. Have
> you thought about a way to evolve the epoch to the generation id throughout
> and what it will take?
>
> On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:
>
> > Hi All
> >
> > Please find the below KIP which describes a proposed solution to a couple
> > of issues that have been observed with the replication protocol.
> >
> > In short, the proposal replaces the use of the High Watermark, for
> > follower log trunctation, with an alternate Generation Marker. This
> > uniquely defines which leader messages were acknowledged by.
> >
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 101+-+Alter+Replication+Protocol+to+use+Leader+
> Generation+rather+than+High+Watermark+for+Truncation
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 101+-+Alter+Replication+Protocol+to+use+Leader+
> Generation+rather+than+High+Watermark+for+Truncation
> > >
> >
> > All comments and suggestions greatly appreciated.
> >
> > Ben Stopford
> > Confluent, http://www.confluent.io <http://www.confluent.io/>
> >
> > --
> Thanks,
> Neha
>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Thanks Neha. It’s a fair point. We don’t need two anymore. 

There isn’t much to be gained from changing the existing nomenclature, unless people feel strongly about it the word epoch. LeaderEpoch is currently fairly ingrained (spans Zookeeper, the controller, the metadata requests etc). I suggest ditch LeaderGeneration.

I’ll update the KIP.

B

Ben Stopford
Confluent, http://www.confluent.io <http://www.confluent.io/>



> On 11 Dec 2016, at 22:30, Neha Narkhede <ne...@confluent.io> wrote:
> 
> Good to see this KIP being proposed. Back when I added the epoch to the
> replication protocol, we discussed adding it to the log due to the failure
> scenarios listed in the KIP but I failed to convince people that it was
> worth the effort needed to upgrade the cluster (especially after we asked
> people to go through a painful backwards incompatible upgrade for 0.8 :-))
> The lack of including the leader epoch/generation in the log has also been
> one of the biggest critiques of Kafka's replication protocol by the
> distributed systems community.
> 
> I'm in favor of this work though I think we shouldn't end up with 2 notions
> of representing a leader's generation. When we added the epoch, we wanted
> to add it to the log but we didn't. Now that we are adding the generation
> id to the log, I think we should revisit calling it the epoch at all. Have
> you thought about a way to evolve the epoch to the generation id throughout
> and what it will take?
> 
> On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:
> 
>> Hi All
>> 
>> Please find the below KIP which describes a proposed solution to a couple
>> of issues that have been observed with the replication protocol.
>> 
>> In short, the proposal replaces the use of the High Watermark, for
>> follower log trunctation, with an alternate Generation Marker. This
>> uniquely defines which leader messages were acknowledged by.
>> 
>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation
>> <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation
>>> 
>> 
>> All comments and suggestions greatly appreciated.
>> 
>> Ben Stopford
>> Confluent, http://www.confluent.io <http://www.confluent.io/>
>> 
>> --
> Thanks,
> Neha


Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Neha Narkhede <ne...@confluent.io>.
Good to see this KIP being proposed. Back when I added the epoch to the
replication protocol, we discussed adding it to the log due to the failure
scenarios listed in the KIP but I failed to convince people that it was
worth the effort needed to upgrade the cluster (especially after we asked
people to go through a painful backwards incompatible upgrade for 0.8 :-))
The lack of including the leader epoch/generation in the log has also been
one of the biggest critiques of Kafka's replication protocol by the
distributed systems community.

I'm in favor of this work though I think we shouldn't end up with 2 notions
of representing a leader's generation. When we added the epoch, we wanted
to add it to the log but we didn't. Now that we are adding the generation
id to the log, I think we should revisit calling it the epoch at all. Have
you thought about a way to evolve the epoch to the generation id throughout
and what it will take?

On Sun, Dec 11, 2016 at 4:31 AM Ben Stopford <be...@confluent.io> wrote:

> Hi All
>
> Please find the below KIP which describes a proposed solution to a couple
> of issues that have been observed with the replication protocol.
>
> In short, the proposal replaces the use of the High Watermark, for
> follower log trunctation, with an alternate Generation Marker. This
> uniquely defines which leader messages were acknowledged by.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation
> >
>
> All comments and suggestions greatly appreciated.
>
> Ben Stopford
> Confluent, http://www.confluent.io <http://www.confluent.io/>
>
> --
Thanks,
Neha

Re: [VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Closing this thread.

On Tue, Jan 3, 2017 at 6:00 PM Ben Stopford <be...@confluent.io> wrote:

> Hi All
>
> Happy New Year!
>
> Hopefully everyone has had an opportunity to review this KIP now, if they
> wished to, so I'd like to call a vote.
>
> As a reminder the KIP proposes a change to the replication protocol to
> remove the potential for replicas to diverge.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation
>
> Thanks
>
> B
>
> On Wed, Dec 21, 2016 at 1:55 PM Ben Stopford <be...@confluent.io> wrote:
>
> Thanks Apurva - yes that's one of those funny english phrases which are
> often read one way, but are really quite ambiguous. I have reworded.
>
> Thanks for pointing this one out. Appreciated.
>
> B
>
> On Tue, 20 Dec 2016 at 23:49, Apurva Mehta <ap...@confluent.io> wrote:
>
> Hi Ben,
>
>
>
> Thanks for the KIP. It is very well written and explains the problem and
>
> solution very nicely. I have one --very minor-- question. In the 'steps'
>
> section, you write:
>
>
>
> > 4.6 The follower starts fetching from the leader from its log end offset.
>
>
>
> The use of 'its' is a bit ambiguous here. I presume that you mean that the
>
> follower fetches from the log end offset of the follower (and not the
>
> leader). Might be worth clarifying whose log end offset is referred to
>
> here.
>
>
>
> While the perceived ambiguity may be put down to my english skills, I still
>
> feet it would be better to leave no room for doubt.
>
>
>
> Thanks,
>
> Apurva
>
>
>
> On Sun, Dec 11, 2016 at 4:30 AM, Ben Stopford <be...@confluent.io> wrote:
>
>
>
> > Hi All
>
> >
>
> > Please find the below KIP which describes a proposed solution to a couple
>
> > of issues that have been observed with the replication protocol.
>
> >
>
> > In short, the proposal replaces the use of the High Watermark, for
>
> > follower log trunctation, with an alternate Generation Marker. This
>
> > uniquely defines which leader messages were acknowledged by.
>
> >
>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
>
> > Generation+rather+than+High+Watermark+for+Truncation <
>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
>
> > Generation+rather+than+High+Watermark+for+Truncation>
>
> >
>
> > All comments and suggestions greatly appreciated.
>
> >
>
> > Ben Stopford
>
> > Confluent, http://www.confluent.io <http://www.confluent.io/>
>
> >
>
> >
>
>

[VOTE] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Hi All

Happy New Year!

Hopefully everyone has had an opportunity to review this KIP now, if they
wished to, so I'd like to call a vote.

As a reminder the KIP proposes a change to the replication protocol to
remove the potential for replicas to diverge.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation

Thanks

B

On Wed, Dec 21, 2016 at 1:55 PM Ben Stopford <be...@confluent.io> wrote:

> Thanks Apurva - yes that's one of those funny english phrases which are
> often read one way, but are really quite ambiguous. I have reworded.
>
> Thanks for pointing this one out. Appreciated.
>
> B
>
> On Tue, 20 Dec 2016 at 23:49, Apurva Mehta <ap...@confluent.io> wrote:
>
> Hi Ben,
>
>
>
> Thanks for the KIP. It is very well written and explains the problem and
>
> solution very nicely. I have one --very minor-- question. In the 'steps'
>
> section, you write:
>
>
>
> > 4.6 The follower starts fetching from the leader from its log end offset.
>
>
>
> The use of 'its' is a bit ambiguous here. I presume that you mean that the
>
> follower fetches from the log end offset of the follower (and not the
>
> leader). Might be worth clarifying whose log end offset is referred to
>
> here.
>
>
>
> While the perceived ambiguity may be put down to my english skills, I still
>
> feet it would be better to leave no room for doubt.
>
>
>
> Thanks,
>
> Apurva
>
>
>
> On Sun, Dec 11, 2016 at 4:30 AM, Ben Stopford <be...@confluent.io> wrote:
>
>
>
> > Hi All
>
> >
>
> > Please find the below KIP which describes a proposed solution to a couple
>
> > of issues that have been observed with the replication protocol.
>
> >
>
> > In short, the proposal replaces the use of the High Watermark, for
>
> > follower log trunctation, with an alternate Generation Marker. This
>
> > uniquely defines which leader messages were acknowledged by.
>
> >
>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
>
> > Generation+rather+than+High+Watermark+for+Truncation <
>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > 101+-+Alter+Replication+Protocol+to+use+Leader+
>
> > Generation+rather+than+High+Watermark+for+Truncation>
>
> >
>
> > All comments and suggestions greatly appreciated.
>
> >
>
> > Ben Stopford
>
> > Confluent, http://www.confluent.io <http://www.confluent.io/>
>
> >
>
> >
>
>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Thanks Apurva - yes that's one of those funny english phrases which are
often read one way, but are really quite ambiguous. I have reworded.

Thanks for pointing this one out. Appreciated.

B

On Tue, 20 Dec 2016 at 23:49, Apurva Mehta <ap...@confluent.io> wrote:

Hi Ben,



Thanks for the KIP. It is very well written and explains the problem and

solution very nicely. I have one --very minor-- question. In the 'steps'

section, you write:



> 4.6 The follower starts fetching from the leader from its log end offset.



The use of 'its' is a bit ambiguous here. I presume that you mean that the

follower fetches from the log end offset of the follower (and not the

leader). Might be worth clarifying whose log end offset is referred to

here.



While the perceived ambiguity may be put down to my english skills, I still

feet it would be better to leave no room for doubt.



Thanks,

Apurva



On Sun, Dec 11, 2016 at 4:30 AM, Ben Stopford <be...@confluent.io> wrote:



> Hi All

>

> Please find the below KIP which describes a proposed solution to a couple

> of issues that have been observed with the replication protocol.

>

> In short, the proposal replaces the use of the High Watermark, for

> follower log trunctation, with an alternate Generation Marker. This

> uniquely defines which leader messages were acknowledged by.

>

> https://cwiki.apache.org/confluence/display/KAFKA/KIP-

> 101+-+Alter+Replication+Protocol+to+use+Leader+

> Generation+rather+than+High+Watermark+for+Truncation <

> https://cwiki.apache.org/confluence/display/KAFKA/KIP-

> 101+-+Alter+Replication+Protocol+to+use+Leader+

> Generation+rather+than+High+Watermark+for+Truncation>

>

> All comments and suggestions greatly appreciated.

>

> Ben Stopford

> Confluent, http://www.confluent.io <http://www.confluent.io/>

>

>

Re: [DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Ben,

Thanks for the KIP. It is very well written and explains the problem and
solution very nicely. I have one --very minor-- question. In the 'steps'
section, you write:

> 4.6 The follower starts fetching from the leader from its log end offset.

The use of 'its' is a bit ambiguous here. I presume that you mean that the
follower fetches from the log end offset of the follower (and not the
leader). Might be worth clarifying whose log end offset is referred to
here.

While the perceived ambiguity may be put down to my english skills, I still
feet it would be better to leave no room for doubt.

Thanks,
Apurva

On Sun, Dec 11, 2016 at 4:30 AM, Ben Stopford <be...@confluent.io> wrote:

> Hi All
>
> Please find the below KIP which describes a proposed solution to a couple
> of issues that have been observed with the replication protocol.
>
> In short, the proposal replaces the use of the High Watermark, for
> follower log trunctation, with an alternate Generation Marker. This
> uniquely defines which leader messages were acknowledged by.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 101+-+Alter+Replication+Protocol+to+use+Leader+
> Generation+rather+than+High+Watermark+for+Truncation <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 101+-+Alter+Replication+Protocol+to+use+Leader+
> Generation+rather+than+High+Watermark+for+Truncation>
>
> All comments and suggestions greatly appreciated.
>
> Ben Stopford
> Confluent, http://www.confluent.io <http://www.confluent.io/>
>
>

[DISCUSS] KIP-101: Alter Replication Protocol to use Leader Generation rather than High Watermark for Truncation

Posted by Ben Stopford <be...@confluent.io>.
Hi All

Please find the below KIP which describes a proposed solution to a couple of issues that have been observed with the replication protocol. 

In short, the proposal replaces the use of the High Watermark, for follower log trunctation, with an alternate Generation Marker. This uniquely defines which leader messages were acknowledged by. 

https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation <https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Generation+rather+than+High+Watermark+for+Truncation>

All comments and suggestions greatly appreciated. 

Ben Stopford
Confluent, http://www.confluent.io <http://www.confluent.io/>


Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
@Onur

I think you are asking about this bullet point right?

https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.ia8lrje8xifh

To me "durability" means that once a transaction has been acknowledged
committed, it will never be lost. In this sense the current design does
gurantee this property since the coordinator will not return the
EndTxnRequest until it has completed writing the marker to each of the
partitions. The scenario you were concerning is more about "availability",
which KIP-98 does not ever try to improve on: it is purely dependent on the
replication mechanism that Kafka already has.


@Andrew

The exactly-once delivery semantics in Kafka is not designed to replace
existing transactional systems, like a DBMS. And I agree with you that it
is "different" with what ACID properties provides. Myself has also worked
on some in-memory DBMS systems which tries to provided "queuing operations"
in transactions such like:

beginTxn;

updateRow1();
updateRow2();
sendtoQueue();

endTxn;


And from my experience, I feel that if users want to leverage Kafka as the
queuing services in the above example, they need to make those Kafka topics
"purely owned" by the applications integrating with the transactional
systems instead of making them shared among different producers or
different applications.

And about the various error cases you mentioned above:

* A message is published to a topic which crashes the leader Kafka node, as
  it's replicated across the cluster, it crashes all of the other Kafka
nodes
  (we've really had this - SEGV, our fault and we've fixed it, but it
happened)
  so this is a kind of rolling node crash in a cluster
* Out of memory error in one or more Kafka nodes
* Disk fills in one or more Kafka nodes
* Uncontrolled power-off to all nodes in the cluster

I think they will all be handled as an aborted transaction so atomicity is
still guaranteed.


@Michael

About the naming. As I mentioned before, it is indeed different to what
DBMS has for transactions (for example, we do not provide any
"serializbility" isolation level, or any application-specific consistency
guarantees), but I feel it still makes to use the term "transaction" to
emphasize its properties in terms of atomicity and durability, and it
actually makes audience who are familiar with WAL etc easier to understand
this proposal.



Guozhang



On Fri, Dec 9, 2016 at 10:48 AM, Onur Karaman <on...@gmail.com>
wrote:

> I had a similar comment to Andrew's in terms of the system's safety. He may
> have stated this scenario in a slightly different way in his first failure
> scenario. Specifically I'm wondering if the transaction can appear complete
> and then later go into a bad state. A transaction can involve messages
> spanning multiple partitions and therefore topics, which can have different
> configurations. As part of EndTxnRequest, we send a COMMIT or ABORT marker
> to each partition involved.
>
> Does this mean that the transaction is only as durable as the weakest
> partition?
>
> For instance, a transaction may involve a partition whose RF=3 and
> min.insync.replicas=2 and another partition whose RF=1 with no
> min.insync.replicas specified.
>
> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Hi Jay,
> >
> > I can't go too deep into exact implantation due to no NDA. So apologies
> > here.
> >
> > Essentially we have multiple processes each owning selection of accounts
> > so on general flows an action for an account just needs to be managed
> local
> > to the owning node, happy days ever change is handled as a tick tock
> change.
> >
> > Unfortunately when a transfer occurs we need the two processes to
> > co-ordinate their transaction, we also need to ensure both don't continue
> > other actions/changesl, we do this using a data grid technology. This
> grid
> > technology supports transaction manager that we couple into currently our
> > jms provider which supports full XA transactions as such we can manage
> the
> > production of the change messages out the system transactionally as well
> as
> > the in grid state.
> >
> > The obvious arguement here is should we even look to move this flow off
> > JMS then. We prob shouldn't nor will do this.
> >
> > The point is that I think saying Kafka supports transactions but then not
> > supporting it as per the traditional sense leads to developers expecting
> > similar behaviour and will cause issues in prod when they find it doesn't
> > work as they're used to.
> >
> > As my other response earlier, is there a better name to describe this
> > feature, if we're not implementing transactions to the traditional
> > transaction expected, to avoid this confusion?
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Friday, December 9, 2016 6:08:07 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Michael,
> >
> > Doesn't that example have more to do with applying the update against two
> > rows in a single transaction? That is, clearly the write to Kafka needs
> to
> > be "transactional" and the write to the destination needs to be
> > transactional, but it's not clear to me that you need isolation that
> spans
> > both operations. Can you dive into the system architecture a bit more and
> > explain why Kafka needs to participate in the same transaction as the
> > destination system?
> >
> > -Jay
> >
> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> > wrote:
> >
> > > Usecase in IG:
> > >
> > > Fund transfer between accounts. When we debit one account and fund
> > another
> > > we must ensure the records to both occur as an acid action, and as a
> > single
> > > transaction.
> > >
> > > Today we achieve this because we have jms, as such we can do the
> actions
> > > needed in an xa transaction across both the accounts. To move this flow
> > to
> > > Kafka we would need support of XA transaction.
> > >
> > >
> > >
> > > Sent using OWA for iPhone
> > > ________________________________________
> > > From: Michael Pearce <Mi...@ig.com>
> > > Sent: Friday, December 9, 2016 6:09:06 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > > Messaging
> > >
> > > Hi Jay,
> > >
> > > For me having an XA transaction allows for ensuring ACID across my
> > > application.
> > >
> > > I believe it is part of the JMS api, and obviously JMS still is in
> > > enterprise very widely adopted for Messaging transport , so obviously
> to
> > > say it isn't widely used i think is ignoring a whole range of users.
> Like
> > > wise I believe frameworks like spring etc fully support it more
> evidence
> > of
> > > its wide adoption.
> > >
> > > On this note personally we try to avoid transactions entirely in our
> > flows
> > > for performance and simplicity. but we do alas unfortunately have one
> or
> > > two places we cannot ignore it.
> > >
> > > Cheers
> > > Mike
> > >
> > > Sent using OWA for iPhone
> > > ________________________________________
> > > From: Jay Kreps <ja...@confluent.io>
> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > > Messaging
> > >
> > > Hey Edoardo,
> > >
> > > For (3) can you outline what you think the benefit and use cases for a
> > more
> > > general cross-system XA feature would be an what changes to the
> proposal
> > > would be required to enable it? When I have asked people who wanted
> > > cross-system XA in the past what they wanted it for, I haven't really
> > > gotten any answers that made sense. Every person really wanted
> something
> > > that would be better solved by a transactional (or idempotent) write to
> > > Kafka followed by an independent transactional (or idempotent)
> > consumption
> > > (which this proposal enables). For the use cases they described tying
> > these
> > > two things together had no advantage and many disadvantages.
> > >
> > > I have one use case which would be accomplished by cross-system XA
> which
> > is
> > > allowing the producer to block on the synchronous processing of the
> > message
> > > by (all? some?) consumers. However I'm not convinced that cross-system
> XA
> > > is the best solution to this problem, and I'm also not convinced this
> is
> > an
> > > important problem to solve. But maybe you have something in mind here.
> > >
> > > -Jay
> > >
> > >
> > >
> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
> wrote:
> > >
> > > > Hi,
> > > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > > >
> > > > We have many users who choose not to use the Java client,  so I have
> > > > concerns about the added complexity in developing the clients.
> > > > A few questions.
> > > >
> > > > 1 - is mixing transactional and non transactional messages on the
> *same
> > > > topic-partition* really a requirement ?
> > > > What use case does it satisfy?
> > > >
> > > > 2 - I guess some clients may only be interested to implement the
> > producer
> > > > idempotency.
> > > > It's not clear how they could be implemented without having to add
> the
> > > > transaction capabilities.
> > > > As others on this list have said, I too would like to see idempotency
> > as
> > > a
> > > > more basic feature, on top which txns can be built.
> > > >
> > > > 3 - The KIP seems focused on a use case where consumption from a
> topic
> > > and
> > > > subsequent production are part of the producer transaction.
> > > >
> > > > It'd be great to see a way to extend the producer transaction to
> > include
> > > > additional transactional resources,
> > > > so that the consumption from another topic just becomes a special
> case
> > of
> > > > a more general "distributed" txn.
> > > >
> > > > Edo
> > > > --------------------------------------------------
> > > > Edoardo Comar
> > > > IBM MessageHub
> > > > ecomar@uk.ibm.com
> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > > >
> > > > IBM United Kingdom Limited Registered in England and Wales with
> number
> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
> Hants.
> > > PO6
> > > > 3AU
> > > >
> > > >
> > > >
> > > > From:   Guozhang Wang <wa...@gmail.com>
> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > > Date:   30/11/2016 22:20
> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > > > Messaging
> > > >
> > > >
> > > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > > >
> > > >
> > > > Unless stated otherwise above:
> > > > IBM United Kingdom Limited - Registered in England and Wales with
> > number
> > > > 741598.
> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
> PO6
> > > 3AU
> > > >
> > > The information contained in this email is strictly confidential and
> for
> > > the use of the addressee only, unless otherwise indicated. If you are
> not
> > > the intended recipient, please do not read, copy, use or disclose to
> > others
> > > this message or any attachment. Please also notify the sender by
> replying
> > > to this email or by telephone (+44(020 7896 0011) and then delete the
> > email
> > > and any copies of it. Opinions, conclusion (etc) that do not relate to
> > the
> > > official business of this company shall be understood as neither given
> > nor
> > > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > > registered in England and Wales, company number 04008957) and IG Index
> > > Limited (a company registered in England and Wales, company number
> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > > London EC4R 2YA. Both IG Markets Limited (register number 195355) and
> IG
> > > Index Limited (register number 114059) are authorised and regulated by
> > the
> > > Financial Conduct Authority.
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Onur Karaman <on...@gmail.com>.
I had a similar comment to Andrew's in terms of the system's safety. He may
have stated this scenario in a slightly different way in his first failure
scenario. Specifically I'm wondering if the transaction can appear complete
and then later go into a bad state. A transaction can involve messages
spanning multiple partitions and therefore topics, which can have different
configurations. As part of EndTxnRequest, we send a COMMIT or ABORT marker
to each partition involved.

Does this mean that the transaction is only as durable as the weakest
partition?

For instance, a transaction may involve a partition whose RF=3 and
min.insync.replicas=2 and another partition whose RF=1 with no
min.insync.replicas specified.

On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Hi Jay,
>
> I can't go too deep into exact implantation due to no NDA. So apologies
> here.
>
> Essentially we have multiple processes each owning selection of accounts
> so on general flows an action for an account just needs to be managed local
> to the owning node, happy days ever change is handled as a tick tock change.
>
> Unfortunately when a transfer occurs we need the two processes to
> co-ordinate their transaction, we also need to ensure both don't continue
> other actions/changesl, we do this using a data grid technology. This grid
> technology supports transaction manager that we couple into currently our
> jms provider which supports full XA transactions as such we can manage the
> production of the change messages out the system transactionally as well as
> the in grid state.
>
> The obvious arguement here is should we even look to move this flow off
> JMS then. We prob shouldn't nor will do this.
>
> The point is that I think saying Kafka supports transactions but then not
> supporting it as per the traditional sense leads to developers expecting
> similar behaviour and will cause issues in prod when they find it doesn't
> work as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 6:08:07 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Doesn't that example have more to do with applying the update against two
> rows in a single transaction? That is, clearly the write to Kafka needs to
> be "transactional" and the write to the destination needs to be
> transactional, but it's not clear to me that you need isolation that spans
> both operations. Can you dive into the system architecture a bit more and
> explain why Kafka needs to participate in the same transaction as the
> destination system?
>
> -Jay
>
> On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund transfer between accounts. When we debit one account and fund
> another
> > we must ensure the records to both occur as an acid action, and as a
> single
> > transaction.
> >
> > Today we achieve this because we have jms, as such we can do the actions
> > needed in an xa transaction across both the accounts. To move this flow
> to
> > Kafka we would need support of XA transaction.
> >
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Michael Pearce <Mi...@ig.com>
> > Sent: Friday, December 9, 2016 6:09:06 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hi Jay,
> >
> > For me having an XA transaction allows for ensuring ACID across my
> > application.
> >
> > I believe it is part of the JMS api, and obviously JMS still is in
> > enterprise very widely adopted for Messaging transport , so obviously to
> > say it isn't widely used i think is ignoring a whole range of users. Like
> > wise I believe frameworks like spring etc fully support it more evidence
> of
> > its wide adoption.
> >
> > On this note personally we try to avoid transactions entirely in our
> flows
> > for performance and simplicity. but we do alas unfortunately have one or
> > two places we cannot ignore it.
> >
> > Cheers
> > Mike
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Thursday, December 8, 2016 11:25:53 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Edoardo,
> >
> > For (3) can you outline what you think the benefit and use cases for a
> more
> > general cross-system XA feature would be an what changes to the proposal
> > would be required to enable it? When I have asked people who wanted
> > cross-system XA in the past what they wanted it for, I haven't really
> > gotten any answers that made sense. Every person really wanted something
> > that would be better solved by a transactional (or idempotent) write to
> > Kafka followed by an independent transactional (or idempotent)
> consumption
> > (which this proposal enables). For the use cases they described tying
> these
> > two things together had no advantage and many disadvantages.
> >
> > I have one use case which would be accomplished by cross-system XA which
> is
> > allowing the producer to block on the synchronous processing of the
> message
> > by (all? some?) consumers. However I'm not convinced that cross-system XA
> > is the best solution to this problem, and I'm also not convinced this is
> an
> > important problem to solve. But maybe you have something in mind here.
> >
> > -Jay
> >
> >
> >
> > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> >
> > > Hi,
> > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > >
> > > We have many users who choose not to use the Java client,  so I have
> > > concerns about the added complexity in developing the clients.
> > > A few questions.
> > >
> > > 1 - is mixing transactional and non transactional messages on the *same
> > > topic-partition* really a requirement ?
> > > What use case does it satisfy?
> > >
> > > 2 - I guess some clients may only be interested to implement the
> producer
> > > idempotency.
> > > It's not clear how they could be implemented without having to add the
> > > transaction capabilities.
> > > As others on this list have said, I too would like to see idempotency
> as
> > a
> > > more basic feature, on top which txns can be built.
> > >
> > > 3 - The KIP seems focused on a use case where consumption from a topic
> > and
> > > subsequent production are part of the producer transaction.
> > >
> > > It'd be great to see a way to extend the producer transaction to
> include
> > > additional transactional resources,
> > > so that the consumption from another topic just becomes a special case
> of
> > > a more general "distributed" txn.
> > >
> > > Edo
> > > --------------------------------------------------
> > > Edoardo Comar
> > > IBM MessageHub
> > > ecomar@uk.ibm.com
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > > IBM United Kingdom Limited Registered in England and Wales with number
> > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> > PO6
> > > 3AU
> > >
> > >
> > >
> > > From:   Guozhang Wang <wa...@gmail.com>
> > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date:   30/11/2016 22:20
> > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > Messaging
> > >
> > >
> > >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Sriram Subramanian <ra...@confluent.io>.
I am not sure if it is a good idea to support both init() and lazy
initialization. The ideal state would have been to implement init as a non
blocking api and have the rest of the methods throw uninitialized exception
if init was not called. This would ensure that init can still be used by
other non blocking frameworks but at the same time enforces that other apis
are not called till init is complete. The problem with supporting both is
that it is going to confuse users. Why should I call init over lazy
initialization? The lazy initialization of transactions will not work if
you want to recover transactions, read offsets and then start a new
transaction. In such cases, you would have to resort to the init api.
Client developers need to provide support for both the lazy initialization
and init method. This problem might be solved if we just renamed the
initTransaction api (eg: recoverTransactions).

On Mon, Dec 12, 2016 at 11:36 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi Jay,
>
> I like the idea of having a single `init`, but I am not sure about the
> specifics of the metadata initialisation (as Jason alluded to). More
> inline.
>
> On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> >    1. Add a generic init() call which initializes both transactions and
> >    metadata
> >
>
> Would this initialise metadata for all topics? One advantage of doing the
> metadata call during `send` is that we only retrieve metadata for the
> subset of topics that you are producing to. For large clusters, retrieving
> the metadata for all the topics is relatively expensive and I think users
> would prefer to avoid that unless there are some concrete benefits. We
> could pass the topics to `init`, but that seems a bit clunky.
>
>
> >    2. If you don't call init(), metadata is initialized on the first send
> >    (as now)
>
>
> We need to maintain the logic to refresh the metadata on `send` anyway if
> you try to send to a topic that is missing from the metadata (e.g. if it's
> added after the `init` method is called, assuming that we don't expect
> people to call `init` more than once) so that seems fine.
>
>
> > and transactions are lazily initialized at the first beginTransaction()
> > call.
>
>
> I'll leave it to Jason to say if this is feasible. However, if it is, it
> seems like we can just do this and avoid the `init` method altogether?
>
> Ismael
>
> On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Jason/Neha,
> >
> > Yeah, clearly having a mandatory, generic init() method that initializes
> > both transactions and topic metadata would be the ideal solution. This
> > would solve the occasional complaint about blocking behavior during
> > initialization of metadata (or at least shift it to a new complaint about
> > an inability to initialize when the cluster isn't up in test
> environments).
> > The challenge is that we can't do this because it isn't backwards
> > compatible with existing apps that don't call init.
> >
> > The alternative of having an optional generic init() call is a bit odd
> > because to figure out if you need to call it you need to discover what it
> > does, which is not generic, it initializes transactions. We can't really
> > add more logic to init because it only gets invoked by transaction users
> so
> > it doesn't really function as a generic init.
> >
> > What do you think of this solution:
> >
> >    1. Add a generic init() call which initializes both transactions and
> >    metadata
> >    2. If you don't call init(), metadata is initialized on the first send
> >    (as now) and transactions are lazily initialized at the first
> >    beginTransaction() call.
> >
> > -Jay
> >
> >
> >
> >
> >
> >
> >
> > On Mon, Dec 12, 2016 at 9:17 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > @Neha
> > >
> > >
> > > 1. I think we should consider renaming initTransactions to just init()
> > and
> > > > moving the metadata initialization there. Let's make sure we don't
> add
> > > APIs
> > > > that are relevant to this proposal only. Instead, try to think what
> > we'd
> > > > propose if we were writing the producer from scratch today. I suspect
> > we
> > > > would end up with an init() API that would do the metadata
> > initialization
> > > > as well as the transaction stuff lazily. If so, let's make that
> change
> > > now.
> > >
> > >
> > > I think the only awkwardness with `init()` is that it would probably
> have
> > > to be an optional API for non-transactional usage to support existing
> > code.
> > > I'm also not sure what metadata we can actually initialize at that
> point
> > > since we don't know which topics will be produced to. That said, I'm
> also
> > > not fond of the `initTransactions` name, and we may find other uses
> for a
> > > generic `init()` in the future, so I'm in favor this renaming.
> > >
> > >
> > > > 2. Along the same lines, let's think about the role of each id that
> the
> > > > producer will have and see if everything still makes sense. For
> > instance,
> > > > we have quite a few per-producer-instance notions -- client.id, a
> > > producer
> > > > id and a transaction.app.id, some set via config and some generated
> > > > on-the-fly. What role does each play, how do they relate to each
> other
> > > and
> > > > is there an opportunity to get rid of any.
> > >
> > >
> > > The abundance of ids is super annoying. The producer ID is not actually
> > > exposed in either the producer or consumer, but I'm not sure how
> > successful
> > > we'll be in hiding its existence from the user (you probably need to
> know
> > > about it for debugging and administrative purposes at least). This
> issue
> > > has been a continual thorn and I'm not sure I have a great answer. We
> > have
> > > been tempted to use client.id as the AppID at one point or another,
> but
> > > its
> > > current usage is to have the same value for all producers in an
> > > application. The lack of an AppID meant that we would have to expose
> the
> > > producer ID and the application would be responsible for persisting it.
> > In
> > > the use cases we looked at, it was simpler to let the application
> provide
> > > its own ID through configuration. And in use cases where there was no
> > > obvious ID to serve as the AppID, it seemed simple enough to let the
> > > application generate its own. We also looked at removing the producer
> ID.
> > > This was discussed somewhere above, but basically the idea is to store
> > the
> > > AppID in the message set header directly and avoid the mapping to
> > producer
> > > ID altogether. As long as batching isn't too bad, the impact on total
> > size
> > > may not be too bad, but we were ultimately more comfortable with a
> fixed
> > > size ID.
> > >
> > > 3. I think we should definitely consider renaming transaction.app.id
> to
> > > > something else. Given that we already have a notion of
> application.id
> > > and
> > > > it represents the entire Streams application, having
> > transaction.app.id
> > > > that represents a producer instance is confusing. I do understand
> that,
> > > for
> > > > Streams, the user doesn't have to set transaction.app.id as it will
> > > likely
> > > > be application.id+taskId (am I understanding that correctly?)
> > >
> > >
> > > Your understanding is correct. The "transaction" prefix was intended to
> > > make it clear that it was only needed for transactional usage. We've
> also
> > > referred to the AppID as a producer "instance ID." This is more
> > suggestive
> > > of the fact that it needs to be unique within the producers of a
> > particular
> > > application. Maybe we could drop the "transaction" and use "
> instance.id"
> > > or
> > > "app.instance.id"? Not sure that's any better, but perhaps it avoids
> the
> > > confusion with application.id?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Mon, Dec 12, 2016 at 8:37 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > @Becket
> > > >
> > > > It has been a pain in many cases that we do not know the number of
> > > >>    messages in a message set, not sure if the OffsetDelta field in
> the
> > > >> wrapper
> > > >>    message will address this.
> > > >
> > > >
> > > > Interestingly, we had this in one of the design iterations, but we
> > found
> > > > in the prototype that we weren't really using it. Did you have a
> > > particular
> > > > use case in mind? I share the intuition that it may be helpful to
> know,
> > > but
> > > > I don't have a clear example in mind. In fact, in the initial
> version,
> > we
> > > > attempted to let the message set always represent a contiguous
> sequence
> > > of
> > > > messages. In that case, the message set only needed a base offset
> and a
> > > > count of the number of messages, and the individual messages no
> longer
> > > > needed the offset delta. We ultimately abandoned that because we were
> > > > uncomfortable with its impact on compaction.
> > > >
> > > > -Jason
> > > >
> > > > On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > >> Andrew,
> > > >>
> > > >> As I mentioned above, in Kafka durability is supported via data
> > > >> replication
> > > >> instead of sync-flushing to disks. KIP-98 does not try to change
> that
> > > part
> > > >> of the Kafka: if all your replicas are gone at the same time before
> > the
> > > >> data was ever flushed to disks, then your data is lost today, and it
> > > will
> > > >> be still the case after KIP-98.
> > > >>
> > > >> As for atomicity, KIP-98 does provide all-or-nothing guarantee for
> > > writes
> > > >> to multiple partitions, and it is based on its existing durability
> > > >> guarantees. So it is possible that if your durability breaks, then
> > > >> atomicity will be violated: some of the committed transaction's
> > messages
> > > >> could be lost if the above scenarios happen while others can be
> > > >> successfully appended. My take is that, if you have concerns that
> > > Kafka's
> > > >> replication mechanism i not good enough for your durability
> > requirements
> > > >> as
> > > >> of today, then you should have the same level of concerns with
> > > durability
> > > >> if you want to use Kafka with KIP-98 as your transactional queuing
> > > system
> > > >> as well.
> > > >>
> > > >>
> > > >> Guozhang
> > > >>
> > > >>
> > > >> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> > > >> andrew_schofield@live.com
> > > >> > wrote:
> > > >>
> > > >> > Guozhang,
> > > >> > Exactly. This is the crux of the matter. Because it's async, the
> log
> > > is
> > > >> > basically
> > > >> > slightly out of date wrt to the run-time state and a failure of
> all
> > > >> > replicas might
> > > >> > take the data slightly back in time.
> > > >> >
> > > >> > Given this, do you think that KIP-98 gives an all-or-nothing,
> > > >> > no-matter-what guarantee
> > > >> > for Kafka transactions? I think the key is whether the data which
> is
> > > >> > asynchronously
> > > >> > flushed is guaranteed to be recovered atomically in all cases.
> > > >> > Asynchronous but
> > > >> > atomic would be good.
> > > >> >
> > > >> > Andrew Schofield
> > > >> > IBM Watson and Cloud Platform
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > From: Guozhang Wang <wa...@gmail.com>
> > > >> > > Sent: 09 December 2016 22:59
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > Messaging
> > > >> > >
> > > >> > > Onur,
> > > >> > >
> > > >> > > I understand your question now. So it is indeed possible that
> > after
> > > >> > > commitTxn() returned the messages could still be lost
> permanently
> > if
> > > >> all
> > > >> > > replicas failed before the data was flushed to disk. This is the
> > > >> virtue
> > > >> > of
> > > >> > > Kafka's design to reply on replication (probably in memory) for
> > high
> > > >> > > availability, hence async flushing. This scenario already exist
> > > today
> > > >> and
> > > >> > > KIP-98 did not intend to change this factor in any ways.
> > > >> > >
> > > >> > > Guozhang
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> -- Guozhang
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Ismael,

Yes, you are correct, I remember now why didn't do that. I rescind that
suggestion. I still think lazy initialization is more in keeping with what
we've done if feasible.

-Jay

On Mon, Dec 12, 2016 at 11:36 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi Jay,
>
> I like the idea of having a single `init`, but I am not sure about the
> specifics of the metadata initialisation (as Jason alluded to). More
> inline.
>
> On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> >    1. Add a generic init() call which initializes both transactions and
> >    metadata
> >
>
> Would this initialise metadata for all topics? One advantage of doing the
> metadata call during `send` is that we only retrieve metadata for the
> subset of topics that you are producing to. For large clusters, retrieving
> the metadata for all the topics is relatively expensive and I think users
> would prefer to avoid that unless there are some concrete benefits. We
> could pass the topics to `init`, but that seems a bit clunky.
>
>
> >    2. If you don't call init(), metadata is initialized on the first send
> >    (as now)
>
>
> We need to maintain the logic to refresh the metadata on `send` anyway if
> you try to send to a topic that is missing from the metadata (e.g. if it's
> added after the `init` method is called, assuming that we don't expect
> people to call `init` more than once) so that seems fine.
>
>
> > and transactions are lazily initialized at the first beginTransaction()
> > call.
>
>
> I'll leave it to Jason to say if this is feasible. However, if it is, it
> seems like we can just do this and avoid the `init` method altogether?
>
> Ismael
>
> On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Jason/Neha,
> >
> > Yeah, clearly having a mandatory, generic init() method that initializes
> > both transactions and topic metadata would be the ideal solution. This
> > would solve the occasional complaint about blocking behavior during
> > initialization of metadata (or at least shift it to a new complaint about
> > an inability to initialize when the cluster isn't up in test
> environments).
> > The challenge is that we can't do this because it isn't backwards
> > compatible with existing apps that don't call init.
> >
> > The alternative of having an optional generic init() call is a bit odd
> > because to figure out if you need to call it you need to discover what it
> > does, which is not generic, it initializes transactions. We can't really
> > add more logic to init because it only gets invoked by transaction users
> so
> > it doesn't really function as a generic init.
> >
> > What do you think of this solution:
> >
> >    1. Add a generic init() call which initializes both transactions and
> >    metadata
> >    2. If you don't call init(), metadata is initialized on the first send
> >    (as now) and transactions are lazily initialized at the first
> >    beginTransaction() call.
> >
> > -Jay
> >
> >
> >
> >
> >
> >
> >
> > On Mon, Dec 12, 2016 at 9:17 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > @Neha
> > >
> > >
> > > 1. I think we should consider renaming initTransactions to just init()
> > and
> > > > moving the metadata initialization there. Let's make sure we don't
> add
> > > APIs
> > > > that are relevant to this proposal only. Instead, try to think what
> > we'd
> > > > propose if we were writing the producer from scratch today. I suspect
> > we
> > > > would end up with an init() API that would do the metadata
> > initialization
> > > > as well as the transaction stuff lazily. If so, let's make that
> change
> > > now.
> > >
> > >
> > > I think the only awkwardness with `init()` is that it would probably
> have
> > > to be an optional API for non-transactional usage to support existing
> > code.
> > > I'm also not sure what metadata we can actually initialize at that
> point
> > > since we don't know which topics will be produced to. That said, I'm
> also
> > > not fond of the `initTransactions` name, and we may find other uses
> for a
> > > generic `init()` in the future, so I'm in favor this renaming.
> > >
> > >
> > > > 2. Along the same lines, let's think about the role of each id that
> the
> > > > producer will have and see if everything still makes sense. For
> > instance,
> > > > we have quite a few per-producer-instance notions -- client.id, a
> > > producer
> > > > id and a transaction.app.id, some set via config and some generated
> > > > on-the-fly. What role does each play, how do they relate to each
> other
> > > and
> > > > is there an opportunity to get rid of any.
> > >
> > >
> > > The abundance of ids is super annoying. The producer ID is not actually
> > > exposed in either the producer or consumer, but I'm not sure how
> > successful
> > > we'll be in hiding its existence from the user (you probably need to
> know
> > > about it for debugging and administrative purposes at least). This
> issue
> > > has been a continual thorn and I'm not sure I have a great answer. We
> > have
> > > been tempted to use client.id as the AppID at one point or another,
> but
> > > its
> > > current usage is to have the same value for all producers in an
> > > application. The lack of an AppID meant that we would have to expose
> the
> > > producer ID and the application would be responsible for persisting it.
> > In
> > > the use cases we looked at, it was simpler to let the application
> provide
> > > its own ID through configuration. And in use cases where there was no
> > > obvious ID to serve as the AppID, it seemed simple enough to let the
> > > application generate its own. We also looked at removing the producer
> ID.
> > > This was discussed somewhere above, but basically the idea is to store
> > the
> > > AppID in the message set header directly and avoid the mapping to
> > producer
> > > ID altogether. As long as batching isn't too bad, the impact on total
> > size
> > > may not be too bad, but we were ultimately more comfortable with a
> fixed
> > > size ID.
> > >
> > > 3. I think we should definitely consider renaming transaction.app.id
> to
> > > > something else. Given that we already have a notion of
> application.id
> > > and
> > > > it represents the entire Streams application, having
> > transaction.app.id
> > > > that represents a producer instance is confusing. I do understand
> that,
> > > for
> > > > Streams, the user doesn't have to set transaction.app.id as it will
> > > likely
> > > > be application.id+taskId (am I understanding that correctly?)
> > >
> > >
> > > Your understanding is correct. The "transaction" prefix was intended to
> > > make it clear that it was only needed for transactional usage. We've
> also
> > > referred to the AppID as a producer "instance ID." This is more
> > suggestive
> > > of the fact that it needs to be unique within the producers of a
> > particular
> > > application. Maybe we could drop the "transaction" and use "
> instance.id"
> > > or
> > > "app.instance.id"? Not sure that's any better, but perhaps it avoids
> the
> > > confusion with application.id?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Mon, Dec 12, 2016 at 8:37 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > @Becket
> > > >
> > > > It has been a pain in many cases that we do not know the number of
> > > >>    messages in a message set, not sure if the OffsetDelta field in
> the
> > > >> wrapper
> > > >>    message will address this.
> > > >
> > > >
> > > > Interestingly, we had this in one of the design iterations, but we
> > found
> > > > in the prototype that we weren't really using it. Did you have a
> > > particular
> > > > use case in mind? I share the intuition that it may be helpful to
> know,
> > > but
> > > > I don't have a clear example in mind. In fact, in the initial
> version,
> > we
> > > > attempted to let the message set always represent a contiguous
> sequence
> > > of
> > > > messages. In that case, the message set only needed a base offset
> and a
> > > > count of the number of messages, and the individual messages no
> longer
> > > > needed the offset delta. We ultimately abandoned that because we were
> > > > uncomfortable with its impact on compaction.
> > > >
> > > > -Jason
> > > >
> > > > On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > >> Andrew,
> > > >>
> > > >> As I mentioned above, in Kafka durability is supported via data
> > > >> replication
> > > >> instead of sync-flushing to disks. KIP-98 does not try to change
> that
> > > part
> > > >> of the Kafka: if all your replicas are gone at the same time before
> > the
> > > >> data was ever flushed to disks, then your data is lost today, and it
> > > will
> > > >> be still the case after KIP-98.
> > > >>
> > > >> As for atomicity, KIP-98 does provide all-or-nothing guarantee for
> > > writes
> > > >> to multiple partitions, and it is based on its existing durability
> > > >> guarantees. So it is possible that if your durability breaks, then
> > > >> atomicity will be violated: some of the committed transaction's
> > messages
> > > >> could be lost if the above scenarios happen while others can be
> > > >> successfully appended. My take is that, if you have concerns that
> > > Kafka's
> > > >> replication mechanism i not good enough for your durability
> > requirements
> > > >> as
> > > >> of today, then you should have the same level of concerns with
> > > durability
> > > >> if you want to use Kafka with KIP-98 as your transactional queuing
> > > system
> > > >> as well.
> > > >>
> > > >>
> > > >> Guozhang
> > > >>
> > > >>
> > > >> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> > > >> andrew_schofield@live.com
> > > >> > wrote:
> > > >>
> > > >> > Guozhang,
> > > >> > Exactly. This is the crux of the matter. Because it's async, the
> log
> > > is
> > > >> > basically
> > > >> > slightly out of date wrt to the run-time state and a failure of
> all
> > > >> > replicas might
> > > >> > take the data slightly back in time.
> > > >> >
> > > >> > Given this, do you think that KIP-98 gives an all-or-nothing,
> > > >> > no-matter-what guarantee
> > > >> > for Kafka transactions? I think the key is whether the data which
> is
> > > >> > asynchronously
> > > >> > flushed is guaranteed to be recovered atomically in all cases.
> > > >> > Asynchronous but
> > > >> > atomic would be good.
> > > >> >
> > > >> > Andrew Schofield
> > > >> > IBM Watson and Cloud Platform
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > From: Guozhang Wang <wa...@gmail.com>
> > > >> > > Sent: 09 December 2016 22:59
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > Messaging
> > > >> > >
> > > >> > > Onur,
> > > >> > >
> > > >> > > I understand your question now. So it is indeed possible that
> > after
> > > >> > > commitTxn() returned the messages could still be lost
> permanently
> > if
> > > >> all
> > > >> > > replicas failed before the data was flushed to disk. This is the
> > > >> virtue
> > > >> > of
> > > >> > > Kafka's design to reply on replication (probably in memory) for
> > high
> > > >> > > availability, hence async flushing. This scenario already exist
> > > today
> > > >> and
> > > >> > > KIP-98 did not intend to change this factor in any ways.
> > > >> > >
> > > >> > > Guozhang
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> -- Guozhang
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Jay,

I like the idea of having a single `init`, but I am not sure about the
specifics of the metadata initialisation (as Jason alluded to). More inline.

On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:

>    1. Add a generic init() call which initializes both transactions and
>    metadata
>

Would this initialise metadata for all topics? One advantage of doing the
metadata call during `send` is that we only retrieve metadata for the
subset of topics that you are producing to. For large clusters, retrieving
the metadata for all the topics is relatively expensive and I think users
would prefer to avoid that unless there are some concrete benefits. We
could pass the topics to `init`, but that seems a bit clunky.


>    2. If you don't call init(), metadata is initialized on the first send
>    (as now)


We need to maintain the logic to refresh the metadata on `send` anyway if
you try to send to a topic that is missing from the metadata (e.g. if it's
added after the `init` method is called, assuming that we don't expect
people to call `init` more than once) so that seems fine.


> and transactions are lazily initialized at the first beginTransaction()
> call.


I'll leave it to Jason to say if this is feasible. However, if it is, it
seems like we can just do this and avoid the `init` method altogether?

Ismael

On Tue, Dec 13, 2016 at 6:01 AM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Jason/Neha,
>
> Yeah, clearly having a mandatory, generic init() method that initializes
> both transactions and topic metadata would be the ideal solution. This
> would solve the occasional complaint about blocking behavior during
> initialization of metadata (or at least shift it to a new complaint about
> an inability to initialize when the cluster isn't up in test environments).
> The challenge is that we can't do this because it isn't backwards
> compatible with existing apps that don't call init.
>
> The alternative of having an optional generic init() call is a bit odd
> because to figure out if you need to call it you need to discover what it
> does, which is not generic, it initializes transactions. We can't really
> add more logic to init because it only gets invoked by transaction users so
> it doesn't really function as a generic init.
>
> What do you think of this solution:
>
>    1. Add a generic init() call which initializes both transactions and
>    metadata
>    2. If you don't call init(), metadata is initialized on the first send
>    (as now) and transactions are lazily initialized at the first
>    beginTransaction() call.
>
> -Jay
>
>
>
>
>
>
>
> On Mon, Dec 12, 2016 at 9:17 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > @Neha
> >
> >
> > 1. I think we should consider renaming initTransactions to just init()
> and
> > > moving the metadata initialization there. Let's make sure we don't add
> > APIs
> > > that are relevant to this proposal only. Instead, try to think what
> we'd
> > > propose if we were writing the producer from scratch today. I suspect
> we
> > > would end up with an init() API that would do the metadata
> initialization
> > > as well as the transaction stuff lazily. If so, let's make that change
> > now.
> >
> >
> > I think the only awkwardness with `init()` is that it would probably have
> > to be an optional API for non-transactional usage to support existing
> code.
> > I'm also not sure what metadata we can actually initialize at that point
> > since we don't know which topics will be produced to. That said, I'm also
> > not fond of the `initTransactions` name, and we may find other uses for a
> > generic `init()` in the future, so I'm in favor this renaming.
> >
> >
> > > 2. Along the same lines, let's think about the role of each id that the
> > > producer will have and see if everything still makes sense. For
> instance,
> > > we have quite a few per-producer-instance notions -- client.id, a
> > producer
> > > id and a transaction.app.id, some set via config and some generated
> > > on-the-fly. What role does each play, how do they relate to each other
> > and
> > > is there an opportunity to get rid of any.
> >
> >
> > The abundance of ids is super annoying. The producer ID is not actually
> > exposed in either the producer or consumer, but I'm not sure how
> successful
> > we'll be in hiding its existence from the user (you probably need to know
> > about it for debugging and administrative purposes at least). This issue
> > has been a continual thorn and I'm not sure I have a great answer. We
> have
> > been tempted to use client.id as the AppID at one point or another, but
> > its
> > current usage is to have the same value for all producers in an
> > application. The lack of an AppID meant that we would have to expose the
> > producer ID and the application would be responsible for persisting it.
> In
> > the use cases we looked at, it was simpler to let the application provide
> > its own ID through configuration. And in use cases where there was no
> > obvious ID to serve as the AppID, it seemed simple enough to let the
> > application generate its own. We also looked at removing the producer ID.
> > This was discussed somewhere above, but basically the idea is to store
> the
> > AppID in the message set header directly and avoid the mapping to
> producer
> > ID altogether. As long as batching isn't too bad, the impact on total
> size
> > may not be too bad, but we were ultimately more comfortable with a fixed
> > size ID.
> >
> > 3. I think we should definitely consider renaming transaction.app.id to
> > > something else. Given that we already have a notion of application.id
> > and
> > > it represents the entire Streams application, having
> transaction.app.id
> > > that represents a producer instance is confusing. I do understand that,
> > for
> > > Streams, the user doesn't have to set transaction.app.id as it will
> > likely
> > > be application.id+taskId (am I understanding that correctly?)
> >
> >
> > Your understanding is correct. The "transaction" prefix was intended to
> > make it clear that it was only needed for transactional usage. We've also
> > referred to the AppID as a producer "instance ID." This is more
> suggestive
> > of the fact that it needs to be unique within the producers of a
> particular
> > application. Maybe we could drop the "transaction" and use "instance.id"
> > or
> > "app.instance.id"? Not sure that's any better, but perhaps it avoids the
> > confusion with application.id?
> >
> > Thanks,
> > Jason
> >
> > On Mon, Dec 12, 2016 at 8:37 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > @Becket
> > >
> > > It has been a pain in many cases that we do not know the number of
> > >>    messages in a message set, not sure if the OffsetDelta field in the
> > >> wrapper
> > >>    message will address this.
> > >
> > >
> > > Interestingly, we had this in one of the design iterations, but we
> found
> > > in the prototype that we weren't really using it. Did you have a
> > particular
> > > use case in mind? I share the intuition that it may be helpful to know,
> > but
> > > I don't have a clear example in mind. In fact, in the initial version,
> we
> > > attempted to let the message set always represent a contiguous sequence
> > of
> > > messages. In that case, the message set only needed a base offset and a
> > > count of the number of messages, and the individual messages no longer
> > > needed the offset delta. We ultimately abandoned that because we were
> > > uncomfortable with its impact on compaction.
> > >
> > > -Jason
> > >
> > > On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > >> Andrew,
> > >>
> > >> As I mentioned above, in Kafka durability is supported via data
> > >> replication
> > >> instead of sync-flushing to disks. KIP-98 does not try to change that
> > part
> > >> of the Kafka: if all your replicas are gone at the same time before
> the
> > >> data was ever flushed to disks, then your data is lost today, and it
> > will
> > >> be still the case after KIP-98.
> > >>
> > >> As for atomicity, KIP-98 does provide all-or-nothing guarantee for
> > writes
> > >> to multiple partitions, and it is based on its existing durability
> > >> guarantees. So it is possible that if your durability breaks, then
> > >> atomicity will be violated: some of the committed transaction's
> messages
> > >> could be lost if the above scenarios happen while others can be
> > >> successfully appended. My take is that, if you have concerns that
> > Kafka's
> > >> replication mechanism i not good enough for your durability
> requirements
> > >> as
> > >> of today, then you should have the same level of concerns with
> > durability
> > >> if you want to use Kafka with KIP-98 as your transactional queuing
> > system
> > >> as well.
> > >>
> > >>
> > >> Guozhang
> > >>
> > >>
> > >> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> > >> andrew_schofield@live.com
> > >> > wrote:
> > >>
> > >> > Guozhang,
> > >> > Exactly. This is the crux of the matter. Because it's async, the log
> > is
> > >> > basically
> > >> > slightly out of date wrt to the run-time state and a failure of all
> > >> > replicas might
> > >> > take the data slightly back in time.
> > >> >
> > >> > Given this, do you think that KIP-98 gives an all-or-nothing,
> > >> > no-matter-what guarantee
> > >> > for Kafka transactions? I think the key is whether the data which is
> > >> > asynchronously
> > >> > flushed is guaranteed to be recovered atomically in all cases.
> > >> > Asynchronous but
> > >> > atomic would be good.
> > >> >
> > >> > Andrew Schofield
> > >> > IBM Watson and Cloud Platform
> > >> >
> > >> >
> > >> > >
> > >> > > From: Guozhang Wang <wa...@gmail.com>
> > >> > > Sent: 09 December 2016 22:59
> > >> > > To: dev@kafka.apache.org
> > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > >> > Messaging
> > >> > >
> > >> > > Onur,
> > >> > >
> > >> > > I understand your question now. So it is indeed possible that
> after
> > >> > > commitTxn() returned the messages could still be lost permanently
> if
> > >> all
> > >> > > replicas failed before the data was flushed to disk. This is the
> > >> virtue
> > >> > of
> > >> > > Kafka's design to reply on replication (probably in memory) for
> high
> > >> > > availability, hence async flushing. This scenario already exist
> > today
> > >> and
> > >> > > KIP-98 did not intend to change this factor in any ways.
> > >> > >
> > >> > > Guozhang
> > >> > >
> > >> > >
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -- Guozhang
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Jason/Neha,

Yeah, clearly having a mandatory, generic init() method that initializes
both transactions and topic metadata would be the ideal solution. This
would solve the occasional complaint about blocking behavior during
initialization of metadata (or at least shift it to a new complaint about
an inability to initialize when the cluster isn't up in test environments).
The challenge is that we can't do this because it isn't backwards
compatible with existing apps that don't call init.

The alternative of having an optional generic init() call is a bit odd
because to figure out if you need to call it you need to discover what it
does, which is not generic, it initializes transactions. We can't really
add more logic to init because it only gets invoked by transaction users so
it doesn't really function as a generic init.

What do you think of this solution:

   1. Add a generic init() call which initializes both transactions and
   metadata
   2. If you don't call init(), metadata is initialized on the first send
   (as now) and transactions are lazily initialized at the first
   beginTransaction() call.

-Jay







On Mon, Dec 12, 2016 at 9:17 PM, Jason Gustafson <ja...@confluent.io> wrote:

> @Neha
>
>
> 1. I think we should consider renaming initTransactions to just init() and
> > moving the metadata initialization there. Let's make sure we don't add
> APIs
> > that are relevant to this proposal only. Instead, try to think what we'd
> > propose if we were writing the producer from scratch today. I suspect we
> > would end up with an init() API that would do the metadata initialization
> > as well as the transaction stuff lazily. If so, let's make that change
> now.
>
>
> I think the only awkwardness with `init()` is that it would probably have
> to be an optional API for non-transactional usage to support existing code.
> I'm also not sure what metadata we can actually initialize at that point
> since we don't know which topics will be produced to. That said, I'm also
> not fond of the `initTransactions` name, and we may find other uses for a
> generic `init()` in the future, so I'm in favor this renaming.
>
>
> > 2. Along the same lines, let's think about the role of each id that the
> > producer will have and see if everything still makes sense. For instance,
> > we have quite a few per-producer-instance notions -- client.id, a
> producer
> > id and a transaction.app.id, some set via config and some generated
> > on-the-fly. What role does each play, how do they relate to each other
> and
> > is there an opportunity to get rid of any.
>
>
> The abundance of ids is super annoying. The producer ID is not actually
> exposed in either the producer or consumer, but I'm not sure how successful
> we'll be in hiding its existence from the user (you probably need to know
> about it for debugging and administrative purposes at least). This issue
> has been a continual thorn and I'm not sure I have a great answer. We have
> been tempted to use client.id as the AppID at one point or another, but
> its
> current usage is to have the same value for all producers in an
> application. The lack of an AppID meant that we would have to expose the
> producer ID and the application would be responsible for persisting it. In
> the use cases we looked at, it was simpler to let the application provide
> its own ID through configuration. And in use cases where there was no
> obvious ID to serve as the AppID, it seemed simple enough to let the
> application generate its own. We also looked at removing the producer ID.
> This was discussed somewhere above, but basically the idea is to store the
> AppID in the message set header directly and avoid the mapping to producer
> ID altogether. As long as batching isn't too bad, the impact on total size
> may not be too bad, but we were ultimately more comfortable with a fixed
> size ID.
>
> 3. I think we should definitely consider renaming transaction.app.id to
> > something else. Given that we already have a notion of application.id
> and
> > it represents the entire Streams application, having transaction.app.id
> > that represents a producer instance is confusing. I do understand that,
> for
> > Streams, the user doesn't have to set transaction.app.id as it will
> likely
> > be application.id+taskId (am I understanding that correctly?)
>
>
> Your understanding is correct. The "transaction" prefix was intended to
> make it clear that it was only needed for transactional usage. We've also
> referred to the AppID as a producer "instance ID." This is more suggestive
> of the fact that it needs to be unique within the producers of a particular
> application. Maybe we could drop the "transaction" and use "instance.id"
> or
> "app.instance.id"? Not sure that's any better, but perhaps it avoids the
> confusion with application.id?
>
> Thanks,
> Jason
>
> On Mon, Dec 12, 2016 at 8:37 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > @Becket
> >
> > It has been a pain in many cases that we do not know the number of
> >>    messages in a message set, not sure if the OffsetDelta field in the
> >> wrapper
> >>    message will address this.
> >
> >
> > Interestingly, we had this in one of the design iterations, but we found
> > in the prototype that we weren't really using it. Did you have a
> particular
> > use case in mind? I share the intuition that it may be helpful to know,
> but
> > I don't have a clear example in mind. In fact, in the initial version, we
> > attempted to let the message set always represent a contiguous sequence
> of
> > messages. In that case, the message set only needed a base offset and a
> > count of the number of messages, and the individual messages no longer
> > needed the offset delta. We ultimately abandoned that because we were
> > uncomfortable with its impact on compaction.
> >
> > -Jason
> >
> > On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> >> Andrew,
> >>
> >> As I mentioned above, in Kafka durability is supported via data
> >> replication
> >> instead of sync-flushing to disks. KIP-98 does not try to change that
> part
> >> of the Kafka: if all your replicas are gone at the same time before the
> >> data was ever flushed to disks, then your data is lost today, and it
> will
> >> be still the case after KIP-98.
> >>
> >> As for atomicity, KIP-98 does provide all-or-nothing guarantee for
> writes
> >> to multiple partitions, and it is based on its existing durability
> >> guarantees. So it is possible that if your durability breaks, then
> >> atomicity will be violated: some of the committed transaction's messages
> >> could be lost if the above scenarios happen while others can be
> >> successfully appended. My take is that, if you have concerns that
> Kafka's
> >> replication mechanism i not good enough for your durability requirements
> >> as
> >> of today, then you should have the same level of concerns with
> durability
> >> if you want to use Kafka with KIP-98 as your transactional queuing
> system
> >> as well.
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> >> andrew_schofield@live.com
> >> > wrote:
> >>
> >> > Guozhang,
> >> > Exactly. This is the crux of the matter. Because it's async, the log
> is
> >> > basically
> >> > slightly out of date wrt to the run-time state and a failure of all
> >> > replicas might
> >> > take the data slightly back in time.
> >> >
> >> > Given this, do you think that KIP-98 gives an all-or-nothing,
> >> > no-matter-what guarantee
> >> > for Kafka transactions? I think the key is whether the data which is
> >> > asynchronously
> >> > flushed is guaranteed to be recovered atomically in all cases.
> >> > Asynchronous but
> >> > atomic would be good.
> >> >
> >> > Andrew Schofield
> >> > IBM Watson and Cloud Platform
> >> >
> >> >
> >> > >
> >> > > From: Guozhang Wang <wa...@gmail.com>
> >> > > Sent: 09 December 2016 22:59
> >> > > To: dev@kafka.apache.org
> >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> >> > Messaging
> >> > >
> >> > > Onur,
> >> > >
> >> > > I understand your question now. So it is indeed possible that after
> >> > > commitTxn() returned the messages could still be lost permanently if
> >> all
> >> > > replicas failed before the data was flushed to disk. This is the
> >> virtue
> >> > of
> >> > > Kafka's design to reply on replication (probably in memory) for high
> >> > > availability, hence async flushing. This scenario already exist
> today
> >> and
> >> > > KIP-98 did not intend to change this factor in any ways.
> >> > >
> >> > > Guozhang
> >> > >
> >> > >
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
@Neha


1. I think we should consider renaming initTransactions to just init() and
> moving the metadata initialization there. Let's make sure we don't add APIs
> that are relevant to this proposal only. Instead, try to think what we'd
> propose if we were writing the producer from scratch today. I suspect we
> would end up with an init() API that would do the metadata initialization
> as well as the transaction stuff lazily. If so, let's make that change now.


I think the only awkwardness with `init()` is that it would probably have
to be an optional API for non-transactional usage to support existing code.
I'm also not sure what metadata we can actually initialize at that point
since we don't know which topics will be produced to. That said, I'm also
not fond of the `initTransactions` name, and we may find other uses for a
generic `init()` in the future, so I'm in favor this renaming.


> 2. Along the same lines, let's think about the role of each id that the
> producer will have and see if everything still makes sense. For instance,
> we have quite a few per-producer-instance notions -- client.id, a producer
> id and a transaction.app.id, some set via config and some generated
> on-the-fly. What role does each play, how do they relate to each other and
> is there an opportunity to get rid of any.


The abundance of ids is super annoying. The producer ID is not actually
exposed in either the producer or consumer, but I'm not sure how successful
we'll be in hiding its existence from the user (you probably need to know
about it for debugging and administrative purposes at least). This issue
has been a continual thorn and I'm not sure I have a great answer. We have
been tempted to use client.id as the AppID at one point or another, but its
current usage is to have the same value for all producers in an
application. The lack of an AppID meant that we would have to expose the
producer ID and the application would be responsible for persisting it. In
the use cases we looked at, it was simpler to let the application provide
its own ID through configuration. And in use cases where there was no
obvious ID to serve as the AppID, it seemed simple enough to let the
application generate its own. We also looked at removing the producer ID.
This was discussed somewhere above, but basically the idea is to store the
AppID in the message set header directly and avoid the mapping to producer
ID altogether. As long as batching isn't too bad, the impact on total size
may not be too bad, but we were ultimately more comfortable with a fixed
size ID.

3. I think we should definitely consider renaming transaction.app.id to
> something else. Given that we already have a notion of application.id and
> it represents the entire Streams application, having transaction.app.id
> that represents a producer instance is confusing. I do understand that, for
> Streams, the user doesn't have to set transaction.app.id as it will likely
> be application.id+taskId (am I understanding that correctly?)


Your understanding is correct. The "transaction" prefix was intended to
make it clear that it was only needed for transactional usage. We've also
referred to the AppID as a producer "instance ID." This is more suggestive
of the fact that it needs to be unique within the producers of a particular
application. Maybe we could drop the "transaction" and use "instance.id" or
"app.instance.id"? Not sure that's any better, but perhaps it avoids the
confusion with application.id?

Thanks,
Jason

On Mon, Dec 12, 2016 at 8:37 PM, Jason Gustafson <ja...@confluent.io> wrote:

> @Becket
>
> It has been a pain in many cases that we do not know the number of
>>    messages in a message set, not sure if the OffsetDelta field in the
>> wrapper
>>    message will address this.
>
>
> Interestingly, we had this in one of the design iterations, but we found
> in the prototype that we weren't really using it. Did you have a particular
> use case in mind? I share the intuition that it may be helpful to know, but
> I don't have a clear example in mind. In fact, in the initial version, we
> attempted to let the message set always represent a contiguous sequence of
> messages. In that case, the message set only needed a base offset and a
> count of the number of messages, and the individual messages no longer
> needed the offset delta. We ultimately abandoned that because we were
> uncomfortable with its impact on compaction.
>
> -Jason
>
> On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> Andrew,
>>
>> As I mentioned above, in Kafka durability is supported via data
>> replication
>> instead of sync-flushing to disks. KIP-98 does not try to change that part
>> of the Kafka: if all your replicas are gone at the same time before the
>> data was ever flushed to disks, then your data is lost today, and it will
>> be still the case after KIP-98.
>>
>> As for atomicity, KIP-98 does provide all-or-nothing guarantee for writes
>> to multiple partitions, and it is based on its existing durability
>> guarantees. So it is possible that if your durability breaks, then
>> atomicity will be violated: some of the committed transaction's messages
>> could be lost if the above scenarios happen while others can be
>> successfully appended. My take is that, if you have concerns that Kafka's
>> replication mechanism i not good enough for your durability requirements
>> as
>> of today, then you should have the same level of concerns with durability
>> if you want to use Kafka with KIP-98 as your transactional queuing system
>> as well.
>>
>>
>> Guozhang
>>
>>
>> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
>> andrew_schofield@live.com
>> > wrote:
>>
>> > Guozhang,
>> > Exactly. This is the crux of the matter. Because it's async, the log is
>> > basically
>> > slightly out of date wrt to the run-time state and a failure of all
>> > replicas might
>> > take the data slightly back in time.
>> >
>> > Given this, do you think that KIP-98 gives an all-or-nothing,
>> > no-matter-what guarantee
>> > for Kafka transactions? I think the key is whether the data which is
>> > asynchronously
>> > flushed is guaranteed to be recovered atomically in all cases.
>> > Asynchronous but
>> > atomic would be good.
>> >
>> > Andrew Schofield
>> > IBM Watson and Cloud Platform
>> >
>> >
>> > >
>> > > From: Guozhang Wang <wa...@gmail.com>
>> > > Sent: 09 December 2016 22:59
>> > > To: dev@kafka.apache.org
>> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> > Messaging
>> > >
>> > > Onur,
>> > >
>> > > I understand your question now. So it is indeed possible that after
>> > > commitTxn() returned the messages could still be lost permanently if
>> all
>> > > replicas failed before the data was flushed to disk. This is the
>> virtue
>> > of
>> > > Kafka's design to reply on replication (probably in memory) for high
>> > > availability, hence async flushing. This scenario already exist today
>> and
>> > > KIP-98 did not intend to change this factor in any ways.
>> > >
>> > > Guozhang
>> > >
>> > >
>> > >
>> >
>>
>>
>>
>> --
>> -- Guozhang
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
@Becket

It has been a pain in many cases that we do not know the number of
>    messages in a message set, not sure if the OffsetDelta field in the
> wrapper
>    message will address this.


Interestingly, we had this in one of the design iterations, but we found in
the prototype that we weren't really using it. Did you have a particular
use case in mind? I share the intuition that it may be helpful to know, but
I don't have a clear example in mind. In fact, in the initial version, we
attempted to let the message set always represent a contiguous sequence of
messages. In that case, the message set only needed a base offset and a
count of the number of messages, and the individual messages no longer
needed the offset delta. We ultimately abandoned that because we were
uncomfortable with its impact on compaction.

-Jason

On Mon, Dec 12, 2016 at 5:55 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Andrew,
>
> As I mentioned above, in Kafka durability is supported via data replication
> instead of sync-flushing to disks. KIP-98 does not try to change that part
> of the Kafka: if all your replicas are gone at the same time before the
> data was ever flushed to disks, then your data is lost today, and it will
> be still the case after KIP-98.
>
> As for atomicity, KIP-98 does provide all-or-nothing guarantee for writes
> to multiple partitions, and it is based on its existing durability
> guarantees. So it is possible that if your durability breaks, then
> atomicity will be violated: some of the committed transaction's messages
> could be lost if the above scenarios happen while others can be
> successfully appended. My take is that, if you have concerns that Kafka's
> replication mechanism i not good enough for your durability requirements as
> of today, then you should have the same level of concerns with durability
> if you want to use Kafka with KIP-98 as your transactional queuing system
> as well.
>
>
> Guozhang
>
>
> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> andrew_schofield@live.com
> > wrote:
>
> > Guozhang,
> > Exactly. This is the crux of the matter. Because it's async, the log is
> > basically
> > slightly out of date wrt to the run-time state and a failure of all
> > replicas might
> > take the data slightly back in time.
> >
> > Given this, do you think that KIP-98 gives an all-or-nothing,
> > no-matter-what guarantee
> > for Kafka transactions? I think the key is whether the data which is
> > asynchronously
> > flushed is guaranteed to be recovered atomically in all cases.
> > Asynchronous but
> > atomic would be good.
> >
> > Andrew Schofield
> > IBM Watson and Cloud Platform
> >
> >
> > >
> > > From: Guozhang Wang <wa...@gmail.com>
> > > Sent: 09 December 2016 22:59
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> > >
> > > Onur,
> > >
> > > I understand your question now. So it is indeed possible that after
> > > commitTxn() returned the messages could still be lost permanently if
> all
> > > replicas failed before the data was flushed to disk. This is the virtue
> > of
> > > Kafka's design to reply on replication (probably in memory) for high
> > > availability, hence async flushing. This scenario already exist today
> and
> > > KIP-98 did not intend to change this factor in any ways.
> > >
> > > Guozhang
> > >
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Andrew,

As I mentioned above, in Kafka durability is supported via data replication
instead of sync-flushing to disks. KIP-98 does not try to change that part
of the Kafka: if all your replicas are gone at the same time before the
data was ever flushed to disks, then your data is lost today, and it will
be still the case after KIP-98.

As for atomicity, KIP-98 does provide all-or-nothing guarantee for writes
to multiple partitions, and it is based on its existing durability
guarantees. So it is possible that if your durability breaks, then
atomicity will be violated: some of the committed transaction's messages
could be lost if the above scenarios happen while others can be
successfully appended. My take is that, if you have concerns that Kafka's
replication mechanism i not good enough for your durability requirements as
of today, then you should have the same level of concerns with durability
if you want to use Kafka with KIP-98 as your transactional queuing system
as well.


Guozhang


On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <andrew_schofield@live.com
> wrote:

> Guozhang,
> Exactly. This is the crux of the matter. Because it's async, the log is
> basically
> slightly out of date wrt to the run-time state and a failure of all
> replicas might
> take the data slightly back in time.
>
> Given this, do you think that KIP-98 gives an all-or-nothing,
> no-matter-what guarantee
> for Kafka transactions? I think the key is whether the data which is
> asynchronously
> flushed is guaranteed to be recovered atomically in all cases.
> Asynchronous but
> atomic would be good.
>
> Andrew Schofield
> IBM Watson and Cloud Platform
>
>
> >
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: 09 December 2016 22:59
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
> >
> > Onur,
> >
> > I understand your question now. So it is indeed possible that after
> > commitTxn() returned the messages could still be lost permanently if all
> > replicas failed before the data was flushed to disk. This is the virtue
> of
> > Kafka's design to reply on replication (probably in memory) for high
> > availability, hence async flushing. This scenario already exist today and
> > KIP-98 did not intend to change this factor in any ways.
> >
> > Guozhang
> >
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Andrew Schofield <an...@live.com>.
Guozhang,
Exactly. This is the crux of the matter. Because it's async, the log is basically
slightly out of date wrt to the run-time state and a failure of all replicas might
take the data slightly back in time.

Given this, do you think that KIP-98 gives an all-or-nothing, no-matter-what guarantee
for Kafka transactions? I think the key is whether the data which is asynchronously
flushed is guaranteed to be recovered atomically in all cases. Asynchronous but
atomic would be good.

Andrew Schofield
IBM Watson and Cloud Platform


>
> From: Guozhang Wang <wa...@gmail.com>
> Sent: 09 December 2016 22:59
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
>  
> Onur,
> 
> I understand your question now. So it is indeed possible that after
> commitTxn() returned the messages could still be lost permanently if all
> replicas failed before the data was flushed to disk. This is the virtue of
> Kafka's design to reply on replication (probably in memory) for high
> availability, hence async flushing. This scenario already exist today and
> KIP-98 did not intend to change this factor in any ways.
> 
> Guozhang
>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
@Becket and @Rajini,

Thanks for those comments. You raise some very astute points. I will
address a subset of them here.

One common thread across your emails has to do with the notion of a 'batch'
of messages from the consumer's point of view. In particular, Rajini's
points 12 and 16, and Becket's point 7, all have to do with the semantics
of transactional message delivery to consumers.

You are both correct in that the wording on the KIP regarding message
delivery to consumers is stronger than the actual semantics. In particular,
it is possible for a consumer to partially read a batch for two reasons:
log segment deletion / log compaction, and because the consumer may
explicitly seek to the middle of transaction.

See the discussion on seeks
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.h19xy0kiexf0>
in
the design doc to learn how consumer seeks relate to transactions. The
behavior in the case that log segments are rolled or compacted is exactly
the same: it is possible for consumers to see some tail of the batch, and
not the entire batch.

As such, consumer applications are not 'transaction aware': they don't know
if they are reading batched messages or not. This is similar to databases.
Reading applications don't know which records were written transactionally
and which were not. These are excellent points, and we will update the KIP
and design doc to make this clear.

 * *

@Rajini:

14a: That is a very interesting suggestion. However, these are topic level
configs, and `transaction.app.id` is a producer config. As such, it seems
like it will be very hard to actually enforce this kind of condition in
practice.

14b: Fixed.

14c: This is a tricky balance.. We want to keep the KIP to user facing
details as much as possible. But I can see how a discussion of durability
requirements for each message in a transaction would be useful to the user,
and it should be easy to succinctly call out the messages which require
higher durability requirements in the KIP it self. Will look into doing so.

15.
All transactional messages (all messages written to the transaction log as
well as the commit / abort markers written to the regular topic partitions)
need to be replicated and flushed in order for the system to work. See the
thread between Onur and Guozhang for the details.

18: you are correct. I will update the document.

19.
If a send fails, what happens next depends on the reason for a failure. If
it fails due to an 'UnrecognizedMessageException', it is a fatal error, and
all further sends to that TopicPartition will have the same error. In this
case, the application must abort the transaction (if any) and then exit or
reinitialize the producer to reset the state.

On the other hand, for retriable exceptions, it is possibly worth requiring
infinite retries for idempotence and transactions, otherwise it seems
impossible to deliver either guarantee.

 * *

@Becket:

2: yes, we plan on doing that once the message format and RPCs get
finalized over the course of this discussion.

5: what do you mean by 'concrete example' ? What would you like it to cover?

 * *

Thanks once more for your comments. The rest will be tackled in the coming
days.

Apurva

On Mon, Dec 12, 2016 at 5:35 AM, Rajini Sivaram <
rajinisivaram@googlemail.com> wrote:

> A few more comments:
>
> 16) There are a couple of statements in the doc which don't quite sound
> right:
> Motivation Section: In particular, if a set of consumed messages c are
> transformed to result in set of produced messages p, then all the messages
> in c will be considered consumed if and only if all messages in p are
> reliably published. *Additionally, all messages in **p will be delivered to
> downstream consumers atomically. *
> Section 5.2: *This message indicates to consumers that messages with the
> given PID must be delivered to the user all together, or not at all.*
>
> The two statements suggest that poll() returns a committed batch to the
> application. But my understanding is that poll() returns only committed
> messages, but the application does not see them as a single committed batch
> and the application is not aware of which messages are in a committed
> batch.
>
> 17) Is there a reason why transaction.app.ids were not made authorizable
> entities? Obviously it can be added later if required, but the single
> transaction authorization feels like a very broad scope.
>
> 18) KIP says: *For every acknowledged message the producer will increment
> the sequence number.*
> Aren't sequence numbers incremented in the producer earlier than that?
>
> 19) What happens when a send fails? Will the producer abort the transaction
> at that point? In the example in the KIP, would further sends continue
> until it came upto the offset commit?
>
>
> On Mon, Dec 12, 2016 at 10:33 AM, Rajini Sivaram <
> rajinisivaram@googlemail.com> wrote:
>
> > Thank you for the really well thought out KIP. This is a very neat
> > approach to solve a rather complex problem.
> >
> > I haven't yet digested the doc. But I have a few comments and questions.
> >
> > 11 a). It feels like memory usage in the consumer is unbounded. If an
> > application crashed mid-transaction, lots of messages from other
> > transactions could be written before the transaction timeout eventually
> > aborted the first transaction. Any consumer reading the log would need to
> > wait and buffer all the messages before actually consuming any. You could
> > end up with a consumer that can never consume from the log even if
> > restarted since it doesn't have sufficient heap space.
> >
> > It feels like at the very least, consumers should discard buffered
> records
> > beyond a certain memory limit and continue to consume and discard until
> > commit markers. And then refetch committed records. In most cases, this
> > would fallback to current behavior  providing low latency, but it would
> > ensure that memory usage is limited and all logs are consumable.
> >
> >
> > 11 b) How will the new buffer.memory being introduced in KIP-81 to
> control
> > consumer memory usage be used in the transactional scenario? Would it be
> > completely ignored? We need to ensure that we don't stop reading from
> > sockets while waiting for a commit to complete.
> >
> >
> > 12. How will normal log deletion work? Presumably we can have log
> segments
> > which contain partial transactions. What happens when that log segment
> was
> > deleted (expired)? New consumers replaying logs start consuming partial
> > transactions?
> >
> >
> > 13. From the discussion thread, I gather that the KIP is based on real
> use
> > cases. The KIP does feel very useful. Even so, for some of us who come
> from
> > a more traditional messaging background, it will be very useful to
> > understand the use cases. Can a few be added to the KIP?
> >
> >
> > 14. I think this was mentioned in the doc, but not clear from the KIP.
> >
> > 14a) To use transactions, you need unclean.leader.election=false and
> > min.insync.replicas > 1 in the broker and producer acks=all - perhaps any
> > other value with transaction.app.id ought to to be an invalid config for
> > the producer?
> >
> > 14b) Javadoc in the KIP says producer.app.id - should be
> > transaction.app.id?
> >
> > 14c) To understand the data flow in the KIP, I had to go to the doc to
> > figure out which flows wait for replication to complete. I did find the
> > information in the docs, but it will be good to include this in the KIP.
> >
> >
> > 15) As others have mentioned earlier, it may be good to consider a
> > higher level of durability for transactions. Perhaps for a later KIP, but
> > it looks like only the messages from the transaction coordinator need to
> be
> > flushed to disk to provide that guarantee?
> >
> >
> >
> >
> >
> > On Mon, Dec 12, 2016 at 6:33 AM, Becket Qin <be...@gmail.com>
> wrote:
> >
> >> Thanks for the KIP. It looks I am late to the party :)
> >>
> >> I just went through the KIP and doc for one pass. Some quick feedback:
> >>
> >>
> >>    1.
> >>
> >>    UnrecognizedMessageException is a synchronous exception? It is good
> for
> >>    it to catch the attention, but seems not intuitive because the
> >> exception is
> >>    not caused by the message users are currently sending.
> >>    2.
> >>
> >>    We probably want to have the new protocols and message formats to be
> >>    added to the wiki, at least a separate link to a dedicated google
> doc.
> >> The
> >>    description of some of the new fields seem missing.
> >>    3.
> >>
> >>    It has been a pain in many cases that we do not know the number of
> >>    messages in a message set, not sure if the OffsetDelta field in the
> >> wrapper
> >>    message will address this.
> >>    4. Maybe it worth clarifying the targeting topology, does the
> >>    transaction persists after mirrored? Do the consumer and producer of
> >> the
> >>    consume-process-produce loop have to talk to the same cluster?
> >>    5.
> >>
> >>    It would be good to give one concrete producing example involves a
> >>    transaction
> >>    6.
> >>
> >>    It seems that if there is one long transaction (in terms of the
> offsets
> >>    span, but not number of messages), the consumer side memory pressure
> >> would
> >>    be big because we do not allow LCO to go beyond LSO.
> >>    7.
> >>
> >>    What happens if a consumer starts up and seeks to the middle of a
> >>    transaction?
> >>
> >>
> >> Thanks,
> >>
> >> Jiangjie (Becket) Qin
> >>
> >> On Sun, Dec 11, 2016 at 5:15 PM, Neha Narkhede <ne...@confluent.io>
> wrote:
> >>
> >> > Apurva and Jason -- appreciate the detailed explanation, thank you!
> >> >
> >> > Jason -
> >> >
> >> > > In many applications, there is already a
> >> > natural identifier which is already used to divide the workload. For
> >> > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> >> > applications where there is no natural ID, the user can generate a
> UUID
> >> and
> >> > persist it locally, which is as good as having the server generate it
> >> >
> >> > I think for stateful applications that aren't Streams or Connect, it
> is
> >> > more work to leverage the strongest guarantees that this proposal
> >> offers.
> >> > But as you say, stateless applications, on the other hand, can still
> >> get a
> >> > lot of it by getting deduplication as well as transactional semantics
> >> for
> >> > the lifetime of the producer instance. I think this is fine since I
> >> think
> >> > that the vast majority of producer apps are stateless and will benefit
> >> from
> >> > the stronger guarantees. And for the minority of apps that need to
> >> > carefully use the application id can just fall back to using Streams
> >> > instead and expressing the send as a simple Streams topology, if
> needed.
> >> >
> >> > I have a few comments on the naming --
> >> >
> >> > 1. I think we should consider renaming initTransactions to just init()
> >> and
> >> > moving the metadata initialization there. Let's make sure we don't add
> >> APIs
> >> > that are relevant to this proposal only. Instead, try to think what
> we'd
> >> > propose if we were writing the producer from scratch today. I suspect
> we
> >> > would end up with an init() API that would do the metadata
> >> initialization
> >> > as well as the transaction stuff lazily. If so, let's make that change
> >> now.
> >> > 2. Along the same lines, let's think about the role of each id that
> the
> >> > producer will have and see if everything still makes sense. For
> >> instance,
> >> > we have quite a few per-producer-instance notions -- client.id, a
> >> producer
> >> > id and a transaction.app.id, some set via config and some generated
> >> > on-the-fly. What role does each play, how do they relate to each other
> >> and
> >> > is there an opportunity to get rid of any.
> >> > 3. I think we should definitely consider renaming transaction.app.id
> to
> >> > something else. Given that we already have a notion of application.id
> >> and
> >> > it represents the entire Streams application, having
> transaction.app.id
> >> > that represents a producer instance is confusing. I do understand
> that,
> >> for
> >> > Streams, the user doesn't have to set transaction.app.id as it will
> >> likely
> >> > be application.id+taskId (am I understanding that correctly?)
> >> >
> >> > On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >> >
> >> > Onur,
> >> >
> >> > I understand your question now. So it is indeed possible that after
> >> > commitTxn() returned the messages could still be lost permanently if
> all
> >> > replicas failed before the data was flushed to disk. This is the
> virtue
> >> of
> >> > Kafka's design to reply on replication (probably in memory) for high
> >> > availability, hence async flushing. This scenario already exist today
> >> and
> >> > KIP-98 did not intend to change this factor in any ways.
> >> >
> >> > Guozhang
> >> >
> >> >
> >> > On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <
> >> > onurkaraman.apache@gmail.com>
> >> > wrote:
> >> >
> >> > > In other words, we can see inconsistency when the transaction log
> >> reports
> >> > > the transaction as COMMITTED while the markers and data
> corresponding
> >> to
> >> > > the transaction itself on the user partitions may have been
> partially
> >> > lost
> >> > > after-the-fact because of kafka's durability guarantees.
> >> > >
> >> > > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> >> > > onurkaraman.apache@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > @Guozhang no I actually meant durability concerns over
> COMMIT/ABORT
> >> > > > markers (and a subset of the user's data produced in the
> transaction
> >> > for
> >> > > > that matter) getting lost from the delta between the write and
> >> flush.
> >> > > >
> >> > > > KIP-98 relies on replicas writing to logs, so transaction
> >> durability is
> >> > > > effectively limited by kafka's definition of a "write success"
> >> meaning
> >> > > > written but not flushed to disk.
> >> > > >
> >> > > > I mentioned RF=1 not because of availability but actually to
> >> highlight
> >> > a
> >> > > > corner-case durability scenario where the single replica
> >> participating
> >> > in
> >> > > > the transaction experiences a hard failure after the write but
> >> before
> >> > the
> >> > > > flush, causing the transaction to have partial data loss.
> >> > > >
> >> > > > Is this level of durability okay or do we want stronger guarantees
> >> for
> >> > > the
> >> > > > transaction? Basically what I'm wondering is if KIP-98
> necessitates
> >> > > kafka'a
> >> > > > definition of a "write success" to be extended from "written" to
> an
> >> > > > optional "written and flushed to disk".
> >> > > >
> >> > > > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <
> >> Michael.Pearce@ig.com
> >> > >
> >> > > > wrote:
> >> > > >
> >> > > >> Apologies on the spelling.
> >> > > >>
> >> > > >> *Hi Jay,
> >> > > >> ________________________________________
> >> > > >> From: Michael Pearce <Mi...@ig.com>
> >> > > >> Sent: Friday, December 9, 2016 7:52:25 PM
> >> > > >> To: dev@kafka.apache.org
> >> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> >> Transactional
> >> > > >> Messaging
> >> > > >>
> >> > > >> Hi Jey
> >> > > >>
> >> > > >> 1) I agree, these should be used to add this in a future kip if
> >> ever
> >> > was
> >> > > >> enough of a case. As stated for us I think for these systems we
> >> will
> >> > > keep
> >> > > >> our JMS solutions there.  I think maybe in the docs when this
> >> feature
> >> > is
> >> > > >> written up, one should redirect users to alternative options such
> >> as
> >> > jms
> >> > > >> brokers, for these use cases.
> >> > > >>
> >> > > >> 2) I think this kip needs to be mindful and actually own to make
> >> sure
> >> > > >> things are implemented in a way to make future enchancement
> >> easy/or at
> >> > > >> least extensible. Having to in future rework things and correct
> >> > historic
> >> > > >> decisions is expensive as already finding.
> >> > > >>
> >> > > >> Sent using OWA for iPhone
> >> > > >> ________________________________________
> >> > > >> From: Jay Kreps <ja...@confluent.io>
> >> > > >> Sent: Friday, December 9, 2016 7:19:59 PM
> >> > > >> To: dev@kafka.apache.org
> >> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> >> Transactional
> >> > > >> Messaging
> >> > > >>
> >> > > >> Hey Michael,
> >> > > >>
> >> > > >> Yeah, I don't think you need to go into the details of whatever
> you
> >> > guys
> >> > > >> have. I think several people in the thread said "let's do XA
> >> > > transactions
> >> > > >> too!" Obviously in a world where features were free and always
> >> worked
> >> > > >> perfectly we would! I've probably talked to about 100 people
> about
> >> > their
> >> > > >> use of XA transactions in different systems and my observation
> has
> >> > been
> >> > > >> (a)
> >> > > >> they are a bit of an operational nightmare, (b) the use cases
> i've
> >> > > >> understood don't actually require full XA transactions they
> >> actually
> >> > > >> require a much weaker and easier to guarantee property. The
> result
> >> is
> >> > > you
> >> > > >> pay a big complexity cost for a guarantee much stronger than what
> >> you
> >> > > >> wanted. My sense is that this opinion is broadly shared by the
> >> > > distributed
> >> > > >> systems community at large and by Kafka folks in particular.
> >> > > >>
> >> > > >> I'm a contrarian so I think it is great not to be too swayed by
> >> > "common
> >> > > >> wisdom" though. Five years ago there was a consensus that
> >> distributed
> >> > > >> transactions were too hard to implement in an operationally sound
> >> way,
> >> > > >> which i think was not correct, so the bad reputation for
> >> cross-system
> >> > > >> transactions may be equally wrong!
> >> > > >>
> >> > > >> To build a compelling case this is wrong I think two things need
> >> to be
> >> > > >> done:
> >> > > >>
> >> > > >>    1. Build a case that there are a large/important set of use
> >> cases
> >> > > that
> >> > > >>    cannot be solved with two independent transactions (as i
> >> > described),
> >> > > >> and
> >> > > >>    that these use cases are things Kafka should be able to do.
> >> > > >>    2. Come up with the concrete extensions to the KIP-98 proposal
> >> that
> >> > > >>    would enable an operationally sound implementation for
> pluggable
> >> > > >>    multi-system XA.
> >> > > >>
> >> > > >> -Jay
> >> > > >>
> >> > > >>
> >> > > >>
> >> > > >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <
> >> > Michael.Pearce@ig.com>
> >> > > >> wrote:
> >> > > >>
> >> > > >> > Hi Jay,
> >> > > >> >
> >> > > >> > I can't go too deep into exact implantation due to no NDA. So
> >> > > apologies
> >> > > >> > here.
> >> > > >> >
> >> > > >> > Essentially we have multiple processes each owning selection of
> >> > > accounts
> >> > > >> > so on general flows an action for an account just needs to be
> >> > managed
> >> > > >> local
> >> > > >> > to the owning node, happy days ever change is handled as a tick
> >> tock
> >> > > >> change.
> >> > > >> >
> >> > > >> > Unfortunately when a transfer occurs we need the two processes
> to
> >> > > >> > co-ordinate their transaction, we also need to ensure both
> don't
> >> > > >> continue
> >> > > >> > other actions/changesl, we do this using a data grid
> technology.
> >> > This
> >> > > >> grid
> >> > > >> > technology supports transaction manager that we couple into
> >> > currently
> >> > > >> our
> >> > > >> > jms provider which supports full XA transactions as such we can
> >> > manage
> >> > > >> the
> >> > > >> > production of the change messages out the system
> transactionally
> >> as
> >> > > >> well as
> >> > > >> > the in grid state.
> >> > > >> >
> >> > > >> > The obvious arguement here is should we even look to move this
> >> flow
> >> > > off
> >> > > >> > JMS then. We prob shouldn't nor will do this.
> >> > > >> >
> >> > > >> > The point is that I think saying Kafka supports transactions
> but
> >> > then
> >> > > >> not
> >> > > >> > supporting it as per the traditional sense leads to developers
> >> > > expecting
> >> > > >> > similar behaviour and will cause issues in prod when they find
> it
> >> > > >> doesn't
> >> > > >> > work as they're used to.
> >> > > >> >
> >> > > >> > As my other response earlier, is there a better name to
> describe
> >> > this
> >> > > >> > feature, if we're not implementing transactions to the
> >> traditional
> >> > > >> > transaction expected, to avoid this confusion?
> >> > > >> >
> >> > > >> >
> >> > > >> > Sent using OWA for iPhone
> >> > > >> > ________________________________________
> >> > > >> > From: Jay Kreps <ja...@confluent.io>
> >> > > >> > Sent: Friday, December 9, 2016 6:08:07 PM
> >> > > >> > To: dev@kafka.apache.org
> >> > > >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > Transactional
> >> > > >> > Messaging
> >> > > >> >
> >> > > >> > Hey Michael,
> >> > > >> >
> >> > > >> > Doesn't that example have more to do with applying the update
> >> > against
> >> > > >> two
> >> > > >> > rows in a single transaction? That is, clearly the write to
> Kafka
> >> > > needs
> >> > > >> to
> >> > > >> > be "transactional" and the write to the destination needs to be
> >> > > >> > transactional, but it's not clear to me that you need isolation
> >> that
> >> > > >> spans
> >> > > >> > both operations. Can you dive into the system architecture a
> bit
> >> > more
> >> > > >> and
> >> > > >> > explain why Kafka needs to participate in the same transaction
> as
> >> > the
> >> > > >> > destination system?
> >> > > >> >
> >> > > >> > -Jay
> >> > > >> >
> >> > > >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> >> > > Michael.Pearce@ig.com>
> >> > > >> > wrote:
> >> > > >> >
> >> > > >> > > Usecase in IG:
> >> > > >> > >
> >> > > >> > > Fund transfer between accounts. When we debit one account and
> >> fund
> >> > > >> > another
> >> > > >> > > we must ensure the records to both occur as an acid action,
> >> and as
> >> > a
> >> > > >> > single
> >> > > >> > > transaction.
> >> > > >> > >
> >> > > >> > > Today we achieve this because we have jms, as such we can do
> >> the
> >> > > >> actions
> >> > > >> > > needed in an xa transaction across both the accounts. To move
> >> this
> >> > > >> flow
> >> > > >> > to
> >> > > >> > > Kafka we would need support of XA transaction.
> >> > > >> > >
> >> > > >> > >
> >> > > >> > >
> >> > > >> > > Sent using OWA for iPhone
> >> > > >> > > ________________________________________
> >> > > >> > > From: Michael Pearce <Mi...@ig.com>
> >> > > >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> >> > > >> > > To: dev@kafka.apache.org
> >> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > > Transactional
> >> > > >> > > Messaging
> >> > > >> > >
> >> > > >> > > Hi Jay,
> >> > > >> > >
> >> > > >> > > For me having an XA transaction allows for ensuring ACID
> >> across my
> >> > > >> > > application.
> >> > > >> > >
> >> > > >> > > I believe it is part of the JMS api, and obviously JMS still
> >> is in
> >> > > >> > > enterprise very widely adopted for Messaging transport , so
> >> > > obviously
> >> > > >> to
> >> > > >> > > say it isn't widely used i think is ignoring a whole range of
> >> > users.
> >> > > >> Like
> >> > > >> > > wise I believe frameworks like spring etc fully support it
> more
> >> > > >> evidence
> >> > > >> > of
> >> > > >> > > its wide adoption.
> >> > > >> > >
> >> > > >> > > On this note personally we try to avoid transactions entirely
> >> in
> >> > our
> >> > > >> > flows
> >> > > >> > > for performance and simplicity. but we do alas unfortunately
> >> have
> >> > > one
> >> > > >> or
> >> > > >> > > two places we cannot ignore it.
> >> > > >> > >
> >> > > >> > > Cheers
> >> > > >> > > Mike
> >> > > >> > >
> >> > > >> > > Sent using OWA for iPhone
> >> > > >> > > ________________________________________
> >> > > >> > > From: Jay Kreps <ja...@confluent.io>
> >> > > >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> >> > > >> > > To: dev@kafka.apache.org
> >> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > > Transactional
> >> > > >> > > Messaging
> >> > > >> > >
> >> > > >> > > Hey Edoardo,
> >> > > >> > >
> >> > > >> > > For (3) can you outline what you think the benefit and use
> >> cases
> >> > > for a
> >> > > >> > more
> >> > > >> > > general cross-system XA feature would be an what changes to
> the
> >> > > >> proposal
> >> > > >> > > would be required to enable it? When I have asked people who
> >> > wanted
> >> > > >> > > cross-system XA in the past what they wanted it for, I
> haven't
> >> > > really
> >> > > >> > > gotten any answers that made sense. Every person really
> wanted
> >> > > >> something
> >> > > >> > > that would be better solved by a transactional (or
> idempotent)
> >> > write
> >> > > >> to
> >> > > >> > > Kafka followed by an independent transactional (or
> idempotent)
> >> > > >> > consumption
> >> > > >> > > (which this proposal enables). For the use cases they
> described
> >> > > tying
> >> > > >> > these
> >> > > >> > > two things together had no advantage and many disadvantages.
> >> > > >> > >
> >> > > >> > > I have one use case which would be accomplished by
> >> cross-system XA
> >> > > >> which
> >> > > >> > is
> >> > > >> > > allowing the producer to block on the synchronous processing
> of
> >> > the
> >> > > >> > message
> >> > > >> > > by (all? some?) consumers. However I'm not convinced that
> >> > > >> cross-system XA
> >> > > >> > > is the best solution to this problem, and I'm also not
> >> convinced
> >> > > this
> >> > > >> is
> >> > > >> > an
> >> > > >> > > important problem to solve. But maybe you have something in
> >> mind
> >> > > here.
> >> > > >> > >
> >> > > >> > > -Jay
> >> > > >> > >
> >> > > >> > >
> >> > > >> > >
> >> > > >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <
> >> ECOMAR@uk.ibm.com>
> >> > > >> wrote:
> >> > > >> > >
> >> > > >> > > > Hi,
> >> > > >> > > > thanks, very interesting KIP ... I haven't fully digested
> it
> >> > yet.
> >> > > >> > > >
> >> > > >> > > > We have many users who choose not to use the Java client,
> >> so I
> >> > > have
> >> > > >> > > > concerns about the added complexity in developing the
> >> clients.
> >> > > >> > > > A few questions.
> >> > > >> > > >
> >> > > >> > > > 1 - is mixing transactional and non transactional messages
> on
> >> > the
> >> > > >> *same
> >> > > >> > > > topic-partition* really a requirement ?
> >> > > >> > > > What use case does it satisfy?
> >> > > >> > > >
> >> > > >> > > > 2 - I guess some clients may only be interested to
> implement
> >> the
> >> > > >> > producer
> >> > > >> > > > idempotency.
> >> > > >> > > > It's not clear how they could be implemented without having
> >> to
> >> > add
> >> > > >> the
> >> > > >> > > > transaction capabilities.
> >> > > >> > > > As others on this list have said, I too would like to see
> >> > > >> idempotency
> >> > > >> > as
> >> > > >> > > a
> >> > > >> > > > more basic feature, on top which txns can be built.
> >> > > >> > > >
> >> > > >> > > > 3 - The KIP seems focused on a use case where consumption
> >> from a
> >> > > >> topic
> >> > > >> > > and
> >> > > >> > > > subsequent production are part of the producer transaction.
> >> > > >> > > >
> >> > > >> > > > It'd be great to see a way to extend the producer
> >> transaction to
> >> > > >> > include
> >> > > >> > > > additional transactional resources,
> >> > > >> > > > so that the consumption from another topic just becomes a
> >> > special
> >> > > >> case
> >> > > >> > of
> >> > > >> > > > a more general "distributed" txn.
> >> > > >> > > >
> >> > > >> > > > Edo
> >> > > >> > > > --------------------------------------------------
> >> > > >> > > > Edoardo Comar
> >> > > >> > > > IBM MessageHub
> >> > > >> > > > ecomar@uk.ibm.com
> >> > > >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> >> > > >> > > >
> >> > > >> > > > IBM United Kingdom Limited Registered in England and Wales
> >> with
> >> > > >> number
> >> > > >> > > > 741598 Registered office: PO Box 41, North Harbour,
> >> Portsmouth,
> >> > > >> Hants.
> >> > > >> > > PO6
> >> > > >> > > > 3AU
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > > From:   Guozhang Wang <wa...@gmail.com>
> >> > > >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> >> > > >> > > > Date:   30/11/2016 22:20
> >> > > >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > > >> > Transactional
> >> > > >> > > > Messaging
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > > Hi all,
> >> > > >> > > >
> >> > > >> > > > I have just created KIP-98 to enhance Kafka with exactly
> once
> >> > > >> delivery
> >> > > >> > > > semantics:
> >> > > >> > > >
> >> > > >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > >> > > > <
> >> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > >> > > > >*
> >> > > >> > > >
> >> > > >> > > > This KIP adds a transactional messaging mechanism along
> with
> >> an
> >> > > >> > > idempotent
> >> > > >> > > > producer implementation to make sure that 1) duplicated
> >> messages
> >> > > >> sent
> >> > > >> > > from
> >> > > >> > > > the same identified producer can be detected on the broker
> >> side,
> >> > > and
> >> > > >> > 2) a
> >> > > >> > > > group of messages sent within a transaction will atomically
> >> be
> >> > > >> either
> >> > > >> > > > reflected and fetchable to consumers or not as a whole.
> >> > > >> > > >
> >> > > >> > > > The above wiki page provides a high-level view of the
> >> proposed
> >> > > >> changes
> >> > > >> > as
> >> > > >> > > > well as summarized guarantees. Initial draft of the
> detailed
> >> > > >> > > > implementation
> >> > > >> > > > design is described in this Google doc:
> >> > > >> > > >
> >> > > >> > > > https://docs.google.com/document/d/11Jqy_
> >> > > >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> >> > > >> > > > 0wSw9ra8
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > > We would love to hear your comments and suggestions.
> >> > > >> > > >
> >> > > >> > > > Thanks,
> >> > > >> > > >
> >> > > >> > > > -- Guozhang
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > >
> >> > > >> > > > Unless stated otherwise above:
> >> > > >> > > > IBM United Kingdom Limited - Registered in England and
> Wales
> >> > with
> >> > > >> > number
> >> > > >> > > > 741598.
> >> > > >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
> >> > Hampshire
> >> > > >> PO6
> >> > > >> > > 3AU
> >> > > >> > > >
> >> > > >> > > The information contained in this email is strictly
> >> confidential
> >> > and
> >> > > >> for
> >> > > >> > > the use of the addressee only, unless otherwise indicated. If
> >> you
> >> > > are
> >> > > >> not
> >> > > >> > > the intended recipient, please do not read, copy, use or
> >> disclose
> >> > to
> >> > > >> > others
> >> > > >> > > this message or any attachment. Please also notify the sender
> >> by
> >> > > >> replying
> >> > > >> > > to this email or by telephone (+44(020 7896 0011) and then
> >> delete
> >> > > the
> >> > > >> > email
> >> > > >> > > and any copies of it. Opinions, conclusion (etc) that do not
> >> > relate
> >> > > to
> >> > > >> > the
> >> > > >> > > official business of this company shall be understood as
> >> neither
> >> > > given
> >> > > >> > nor
> >> > > >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> >> > > company
> >> > > >> > > registered in England and Wales, company number 04008957) and
> >> IG
> >> > > Index
> >> > > >> > > Limited (a company registered in England and Wales, company
> >> number
> >> > > >> > > 01190902). Registered address at Cannon Bridge House, 25
> >> Dowgate
> >> > > Hill,
> >> > > >> > > London EC4R 2YA. Both IG Markets Limited (register number
> >> 195355)
> >> > > and
> >> > > >> IG
> >> > > >> > > Index Limited (register number 114059) are authorised and
> >> > regulated
> >> > > by
> >> > > >> > the
> >> > > >> > > Financial Conduct Authority.
> >> > > >> > >
> >> > > >> > The information contained in this email is strictly
> confidential
> >> and
> >> > > for
> >> > > >> > the use of the addressee only, unless otherwise indicated. If
> you
> >> > are
> >> > > >> not
> >> > > >> > the intended recipient, please do not read, copy, use or
> >> disclose to
> >> > > >> others
> >> > > >> > this message or any attachment. Please also notify the sender
> by
> >> > > >> replying
> >> > > >> > to this email or by telephone (+44(020 7896 0011) and then
> delete
> >> > the
> >> > > >> email
> >> > > >> > and any copies of it. Opinions, conclusion (etc) that do not
> >> relate
> >> > to
> >> > > >> the
> >> > > >> > official business of this company shall be understood as
> neither
> >> > given
> >> > > >> nor
> >> > > >> > endorsed by it. IG is a trading name of IG Markets Limited (a
> >> > company
> >> > > >> > registered in England and Wales, company number 04008957) and
> IG
> >> > Index
> >> > > >> > Limited (a company registered in England and Wales, company
> >> number
> >> > > >> > 01190902). Registered address at Cannon Bridge House, 25
> Dowgate
> >> > Hill,
> >> > > >> > London EC4R 2YA. Both IG Markets Limited (register number
> 195355)
> >> > and
> >> > > IG
> >> > > >> > Index Limited (register number 114059) are authorised and
> >> regulated
> >> > by
> >> > > >> the
> >> > > >> > Financial Conduct Authority.
> >> > > >> >
> >> > > >> The information contained in this email is strictly confidential
> >> and
> >> > for
> >> > > >> the use of the addressee only, unless otherwise indicated. If you
> >> are
> >> > > not
> >> > > >> the intended recipient, please do not read, copy, use or disclose
> >> to
> >> > > others
> >> > > >> this message or any attachment. Please also notify the sender by
> >> > > replying
> >> > > >> to this email or by telephone (+44(020 7896 0011) and then delete
> >> the
> >> > > email
> >> > > >> and any copies of it. Opinions, conclusion (etc) that do not
> >> relate to
> >> > > the
> >> > > >> official business of this company shall be understood as neither
> >> given
> >> > > nor
> >> > > >> endorsed by it. IG is a trading name of IG Markets Limited (a
> >> company
> >> > > >> registered in England and Wales, company number 04008957) and IG
> >> Index
> >> > > >> Limited (a company registered in England and Wales, company
> number
> >> > > >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> >> Hill,
> >> > > >> London EC4R 2YA. Both IG Markets Limited (register number 195355)
> >> and
> >> > IG
> >> > > >> Index Limited (register number 114059) are authorised and
> >> regulated by
> >> > > the
> >> > > >> Financial Conduct Authority.
> >> > > >>
> >> > > >
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >> > --
> >> > Thanks,
> >> > Neha
> >> >
> >>
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <ra...@googlemail.com>.
A few more comments:

16) There are a couple of statements in the doc which don't quite sound
right:
Motivation Section: In particular, if a set of consumed messages c are
transformed to result in set of produced messages p, then all the messages
in c will be considered consumed if and only if all messages in p are
reliably published. *Additionally, all messages in **p will be delivered to
downstream consumers atomically. *
Section 5.2: *This message indicates to consumers that messages with the
given PID must be delivered to the user all together, or not at all.*

The two statements suggest that poll() returns a committed batch to the
application. But my understanding is that poll() returns only committed
messages, but the application does not see them as a single committed batch
and the application is not aware of which messages are in a committed batch.

17) Is there a reason why transaction.app.ids were not made authorizable
entities? Obviously it can be added later if required, but the single
transaction authorization feels like a very broad scope.

18) KIP says: *For every acknowledged message the producer will increment
the sequence number.*
Aren't sequence numbers incremented in the producer earlier than that?

19) What happens when a send fails? Will the producer abort the transaction
at that point? In the example in the KIP, would further sends continue
until it came upto the offset commit?


On Mon, Dec 12, 2016 at 10:33 AM, Rajini Sivaram <
rajinisivaram@googlemail.com> wrote:

> Thank you for the really well thought out KIP. This is a very neat
> approach to solve a rather complex problem.
>
> I haven't yet digested the doc. But I have a few comments and questions.
>
> 11 a). It feels like memory usage in the consumer is unbounded. If an
> application crashed mid-transaction, lots of messages from other
> transactions could be written before the transaction timeout eventually
> aborted the first transaction. Any consumer reading the log would need to
> wait and buffer all the messages before actually consuming any. You could
> end up with a consumer that can never consume from the log even if
> restarted since it doesn't have sufficient heap space.
>
> It feels like at the very least, consumers should discard buffered records
> beyond a certain memory limit and continue to consume and discard until
> commit markers. And then refetch committed records. In most cases, this
> would fallback to current behavior  providing low latency, but it would
> ensure that memory usage is limited and all logs are consumable.
>
>
> 11 b) How will the new buffer.memory being introduced in KIP-81 to control
> consumer memory usage be used in the transactional scenario? Would it be
> completely ignored? We need to ensure that we don't stop reading from
> sockets while waiting for a commit to complete.
>
>
> 12. How will normal log deletion work? Presumably we can have log segments
> which contain partial transactions. What happens when that log segment was
> deleted (expired)? New consumers replaying logs start consuming partial
> transactions?
>
>
> 13. From the discussion thread, I gather that the KIP is based on real use
> cases. The KIP does feel very useful. Even so, for some of us who come from
> a more traditional messaging background, it will be very useful to
> understand the use cases. Can a few be added to the KIP?
>
>
> 14. I think this was mentioned in the doc, but not clear from the KIP.
>
> 14a) To use transactions, you need unclean.leader.election=false and
> min.insync.replicas > 1 in the broker and producer acks=all - perhaps any
> other value with transaction.app.id ought to to be an invalid config for
> the producer?
>
> 14b) Javadoc in the KIP says producer.app.id - should be
> transaction.app.id?
>
> 14c) To understand the data flow in the KIP, I had to go to the doc to
> figure out which flows wait for replication to complete. I did find the
> information in the docs, but it will be good to include this in the KIP.
>
>
> 15) As others have mentioned earlier, it may be good to consider a
> higher level of durability for transactions. Perhaps for a later KIP, but
> it looks like only the messages from the transaction coordinator need to be
> flushed to disk to provide that guarantee?
>
>
>
>
>
> On Mon, Dec 12, 2016 at 6:33 AM, Becket Qin <be...@gmail.com> wrote:
>
>> Thanks for the KIP. It looks I am late to the party :)
>>
>> I just went through the KIP and doc for one pass. Some quick feedback:
>>
>>
>>    1.
>>
>>    UnrecognizedMessageException is a synchronous exception? It is good for
>>    it to catch the attention, but seems not intuitive because the
>> exception is
>>    not caused by the message users are currently sending.
>>    2.
>>
>>    We probably want to have the new protocols and message formats to be
>>    added to the wiki, at least a separate link to a dedicated google doc.
>> The
>>    description of some of the new fields seem missing.
>>    3.
>>
>>    It has been a pain in many cases that we do not know the number of
>>    messages in a message set, not sure if the OffsetDelta field in the
>> wrapper
>>    message will address this.
>>    4. Maybe it worth clarifying the targeting topology, does the
>>    transaction persists after mirrored? Do the consumer and producer of
>> the
>>    consume-process-produce loop have to talk to the same cluster?
>>    5.
>>
>>    It would be good to give one concrete producing example involves a
>>    transaction
>>    6.
>>
>>    It seems that if there is one long transaction (in terms of the offsets
>>    span, but not number of messages), the consumer side memory pressure
>> would
>>    be big because we do not allow LCO to go beyond LSO.
>>    7.
>>
>>    What happens if a consumer starts up and seeks to the middle of a
>>    transaction?
>>
>>
>> Thanks,
>>
>> Jiangjie (Becket) Qin
>>
>> On Sun, Dec 11, 2016 at 5:15 PM, Neha Narkhede <ne...@confluent.io> wrote:
>>
>> > Apurva and Jason -- appreciate the detailed explanation, thank you!
>> >
>> > Jason -
>> >
>> > > In many applications, there is already a
>> > natural identifier which is already used to divide the workload. For
>> > example, in Kafka Streams and Kafka Connect, we have a taskId. For
>> > applications where there is no natural ID, the user can generate a UUID
>> and
>> > persist it locally, which is as good as having the server generate it
>> >
>> > I think for stateful applications that aren't Streams or Connect, it is
>> > more work to leverage the strongest guarantees that this proposal
>> offers.
>> > But as you say, stateless applications, on the other hand, can still
>> get a
>> > lot of it by getting deduplication as well as transactional semantics
>> for
>> > the lifetime of the producer instance. I think this is fine since I
>> think
>> > that the vast majority of producer apps are stateless and will benefit
>> from
>> > the stronger guarantees. And for the minority of apps that need to
>> > carefully use the application id can just fall back to using Streams
>> > instead and expressing the send as a simple Streams topology, if needed.
>> >
>> > I have a few comments on the naming --
>> >
>> > 1. I think we should consider renaming initTransactions to just init()
>> and
>> > moving the metadata initialization there. Let's make sure we don't add
>> APIs
>> > that are relevant to this proposal only. Instead, try to think what we'd
>> > propose if we were writing the producer from scratch today. I suspect we
>> > would end up with an init() API that would do the metadata
>> initialization
>> > as well as the transaction stuff lazily. If so, let's make that change
>> now.
>> > 2. Along the same lines, let's think about the role of each id that the
>> > producer will have and see if everything still makes sense. For
>> instance,
>> > we have quite a few per-producer-instance notions -- client.id, a
>> producer
>> > id and a transaction.app.id, some set via config and some generated
>> > on-the-fly. What role does each play, how do they relate to each other
>> and
>> > is there an opportunity to get rid of any.
>> > 3. I think we should definitely consider renaming transaction.app.id to
>> > something else. Given that we already have a notion of application.id
>> and
>> > it represents the entire Streams application, having transaction.app.id
>> > that represents a producer instance is confusing. I do understand that,
>> for
>> > Streams, the user doesn't have to set transaction.app.id as it will
>> likely
>> > be application.id+taskId (am I understanding that correctly?)
>> >
>> > On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wa...@gmail.com>
>> wrote:
>> >
>> > Onur,
>> >
>> > I understand your question now. So it is indeed possible that after
>> > commitTxn() returned the messages could still be lost permanently if all
>> > replicas failed before the data was flushed to disk. This is the virtue
>> of
>> > Kafka's design to reply on replication (probably in memory) for high
>> > availability, hence async flushing. This scenario already exist today
>> and
>> > KIP-98 did not intend to change this factor in any ways.
>> >
>> > Guozhang
>> >
>> >
>> > On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <
>> > onurkaraman.apache@gmail.com>
>> > wrote:
>> >
>> > > In other words, we can see inconsistency when the transaction log
>> reports
>> > > the transaction as COMMITTED while the markers and data corresponding
>> to
>> > > the transaction itself on the user partitions may have been partially
>> > lost
>> > > after-the-fact because of kafka's durability guarantees.
>> > >
>> > > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
>> > > onurkaraman.apache@gmail.com>
>> > > wrote:
>> > >
>> > > > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
>> > > > markers (and a subset of the user's data produced in the transaction
>> > for
>> > > > that matter) getting lost from the delta between the write and
>> flush.
>> > > >
>> > > > KIP-98 relies on replicas writing to logs, so transaction
>> durability is
>> > > > effectively limited by kafka's definition of a "write success"
>> meaning
>> > > > written but not flushed to disk.
>> > > >
>> > > > I mentioned RF=1 not because of availability but actually to
>> highlight
>> > a
>> > > > corner-case durability scenario where the single replica
>> participating
>> > in
>> > > > the transaction experiences a hard failure after the write but
>> before
>> > the
>> > > > flush, causing the transaction to have partial data loss.
>> > > >
>> > > > Is this level of durability okay or do we want stronger guarantees
>> for
>> > > the
>> > > > transaction? Basically what I'm wondering is if KIP-98 necessitates
>> > > kafka'a
>> > > > definition of a "write success" to be extended from "written" to an
>> > > > optional "written and flushed to disk".
>> > > >
>> > > > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <
>> Michael.Pearce@ig.com
>> > >
>> > > > wrote:
>> > > >
>> > > >> Apologies on the spelling.
>> > > >>
>> > > >> *Hi Jay,
>> > > >> ________________________________________
>> > > >> From: Michael Pearce <Mi...@ig.com>
>> > > >> Sent: Friday, December 9, 2016 7:52:25 PM
>> > > >> To: dev@kafka.apache.org
>> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
>> Transactional
>> > > >> Messaging
>> > > >>
>> > > >> Hi Jey
>> > > >>
>> > > >> 1) I agree, these should be used to add this in a future kip if
>> ever
>> > was
>> > > >> enough of a case. As stated for us I think for these systems we
>> will
>> > > keep
>> > > >> our JMS solutions there.  I think maybe in the docs when this
>> feature
>> > is
>> > > >> written up, one should redirect users to alternative options such
>> as
>> > jms
>> > > >> brokers, for these use cases.
>> > > >>
>> > > >> 2) I think this kip needs to be mindful and actually own to make
>> sure
>> > > >> things are implemented in a way to make future enchancement
>> easy/or at
>> > > >> least extensible. Having to in future rework things and correct
>> > historic
>> > > >> decisions is expensive as already finding.
>> > > >>
>> > > >> Sent using OWA for iPhone
>> > > >> ________________________________________
>> > > >> From: Jay Kreps <ja...@confluent.io>
>> > > >> Sent: Friday, December 9, 2016 7:19:59 PM
>> > > >> To: dev@kafka.apache.org
>> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
>> Transactional
>> > > >> Messaging
>> > > >>
>> > > >> Hey Michael,
>> > > >>
>> > > >> Yeah, I don't think you need to go into the details of whatever you
>> > guys
>> > > >> have. I think several people in the thread said "let's do XA
>> > > transactions
>> > > >> too!" Obviously in a world where features were free and always
>> worked
>> > > >> perfectly we would! I've probably talked to about 100 people about
>> > their
>> > > >> use of XA transactions in different systems and my observation has
>> > been
>> > > >> (a)
>> > > >> they are a bit of an operational nightmare, (b) the use cases i've
>> > > >> understood don't actually require full XA transactions they
>> actually
>> > > >> require a much weaker and easier to guarantee property. The result
>> is
>> > > you
>> > > >> pay a big complexity cost for a guarantee much stronger than what
>> you
>> > > >> wanted. My sense is that this opinion is broadly shared by the
>> > > distributed
>> > > >> systems community at large and by Kafka folks in particular.
>> > > >>
>> > > >> I'm a contrarian so I think it is great not to be too swayed by
>> > "common
>> > > >> wisdom" though. Five years ago there was a consensus that
>> distributed
>> > > >> transactions were too hard to implement in an operationally sound
>> way,
>> > > >> which i think was not correct, so the bad reputation for
>> cross-system
>> > > >> transactions may be equally wrong!
>> > > >>
>> > > >> To build a compelling case this is wrong I think two things need
>> to be
>> > > >> done:
>> > > >>
>> > > >>    1. Build a case that there are a large/important set of use
>> cases
>> > > that
>> > > >>    cannot be solved with two independent transactions (as i
>> > described),
>> > > >> and
>> > > >>    that these use cases are things Kafka should be able to do.
>> > > >>    2. Come up with the concrete extensions to the KIP-98 proposal
>> that
>> > > >>    would enable an operationally sound implementation for pluggable
>> > > >>    multi-system XA.
>> > > >>
>> > > >> -Jay
>> > > >>
>> > > >>
>> > > >>
>> > > >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <
>> > Michael.Pearce@ig.com>
>> > > >> wrote:
>> > > >>
>> > > >> > Hi Jay,
>> > > >> >
>> > > >> > I can't go too deep into exact implantation due to no NDA. So
>> > > apologies
>> > > >> > here.
>> > > >> >
>> > > >> > Essentially we have multiple processes each owning selection of
>> > > accounts
>> > > >> > so on general flows an action for an account just needs to be
>> > managed
>> > > >> local
>> > > >> > to the owning node, happy days ever change is handled as a tick
>> tock
>> > > >> change.
>> > > >> >
>> > > >> > Unfortunately when a transfer occurs we need the two processes to
>> > > >> > co-ordinate their transaction, we also need to ensure both don't
>> > > >> continue
>> > > >> > other actions/changesl, we do this using a data grid technology.
>> > This
>> > > >> grid
>> > > >> > technology supports transaction manager that we couple into
>> > currently
>> > > >> our
>> > > >> > jms provider which supports full XA transactions as such we can
>> > manage
>> > > >> the
>> > > >> > production of the change messages out the system transactionally
>> as
>> > > >> well as
>> > > >> > the in grid state.
>> > > >> >
>> > > >> > The obvious arguement here is should we even look to move this
>> flow
>> > > off
>> > > >> > JMS then. We prob shouldn't nor will do this.
>> > > >> >
>> > > >> > The point is that I think saying Kafka supports transactions but
>> > then
>> > > >> not
>> > > >> > supporting it as per the traditional sense leads to developers
>> > > expecting
>> > > >> > similar behaviour and will cause issues in prod when they find it
>> > > >> doesn't
>> > > >> > work as they're used to.
>> > > >> >
>> > > >> > As my other response earlier, is there a better name to describe
>> > this
>> > > >> > feature, if we're not implementing transactions to the
>> traditional
>> > > >> > transaction expected, to avoid this confusion?
>> > > >> >
>> > > >> >
>> > > >> > Sent using OWA for iPhone
>> > > >> > ________________________________________
>> > > >> > From: Jay Kreps <ja...@confluent.io>
>> > > >> > Sent: Friday, December 9, 2016 6:08:07 PM
>> > > >> > To: dev@kafka.apache.org
>> > > >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
>> > Transactional
>> > > >> > Messaging
>> > > >> >
>> > > >> > Hey Michael,
>> > > >> >
>> > > >> > Doesn't that example have more to do with applying the update
>> > against
>> > > >> two
>> > > >> > rows in a single transaction? That is, clearly the write to Kafka
>> > > needs
>> > > >> to
>> > > >> > be "transactional" and the write to the destination needs to be
>> > > >> > transactional, but it's not clear to me that you need isolation
>> that
>> > > >> spans
>> > > >> > both operations. Can you dive into the system architecture a bit
>> > more
>> > > >> and
>> > > >> > explain why Kafka needs to participate in the same transaction as
>> > the
>> > > >> > destination system?
>> > > >> >
>> > > >> > -Jay
>> > > >> >
>> > > >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
>> > > Michael.Pearce@ig.com>
>> > > >> > wrote:
>> > > >> >
>> > > >> > > Usecase in IG:
>> > > >> > >
>> > > >> > > Fund transfer between accounts. When we debit one account and
>> fund
>> > > >> > another
>> > > >> > > we must ensure the records to both occur as an acid action,
>> and as
>> > a
>> > > >> > single
>> > > >> > > transaction.
>> > > >> > >
>> > > >> > > Today we achieve this because we have jms, as such we can do
>> the
>> > > >> actions
>> > > >> > > needed in an xa transaction across both the accounts. To move
>> this
>> > > >> flow
>> > > >> > to
>> > > >> > > Kafka we would need support of XA transaction.
>> > > >> > >
>> > > >> > >
>> > > >> > >
>> > > >> > > Sent using OWA for iPhone
>> > > >> > > ________________________________________
>> > > >> > > From: Michael Pearce <Mi...@ig.com>
>> > > >> > > Sent: Friday, December 9, 2016 6:09:06 AM
>> > > >> > > To: dev@kafka.apache.org
>> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
>> > > Transactional
>> > > >> > > Messaging
>> > > >> > >
>> > > >> > > Hi Jay,
>> > > >> > >
>> > > >> > > For me having an XA transaction allows for ensuring ACID
>> across my
>> > > >> > > application.
>> > > >> > >
>> > > >> > > I believe it is part of the JMS api, and obviously JMS still
>> is in
>> > > >> > > enterprise very widely adopted for Messaging transport , so
>> > > obviously
>> > > >> to
>> > > >> > > say it isn't widely used i think is ignoring a whole range of
>> > users.
>> > > >> Like
>> > > >> > > wise I believe frameworks like spring etc fully support it more
>> > > >> evidence
>> > > >> > of
>> > > >> > > its wide adoption.
>> > > >> > >
>> > > >> > > On this note personally we try to avoid transactions entirely
>> in
>> > our
>> > > >> > flows
>> > > >> > > for performance and simplicity. but we do alas unfortunately
>> have
>> > > one
>> > > >> or
>> > > >> > > two places we cannot ignore it.
>> > > >> > >
>> > > >> > > Cheers
>> > > >> > > Mike
>> > > >> > >
>> > > >> > > Sent using OWA for iPhone
>> > > >> > > ________________________________________
>> > > >> > > From: Jay Kreps <ja...@confluent.io>
>> > > >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
>> > > >> > > To: dev@kafka.apache.org
>> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
>> > > Transactional
>> > > >> > > Messaging
>> > > >> > >
>> > > >> > > Hey Edoardo,
>> > > >> > >
>> > > >> > > For (3) can you outline what you think the benefit and use
>> cases
>> > > for a
>> > > >> > more
>> > > >> > > general cross-system XA feature would be an what changes to the
>> > > >> proposal
>> > > >> > > would be required to enable it? When I have asked people who
>> > wanted
>> > > >> > > cross-system XA in the past what they wanted it for, I haven't
>> > > really
>> > > >> > > gotten any answers that made sense. Every person really wanted
>> > > >> something
>> > > >> > > that would be better solved by a transactional (or idempotent)
>> > write
>> > > >> to
>> > > >> > > Kafka followed by an independent transactional (or idempotent)
>> > > >> > consumption
>> > > >> > > (which this proposal enables). For the use cases they described
>> > > tying
>> > > >> > these
>> > > >> > > two things together had no advantage and many disadvantages.
>> > > >> > >
>> > > >> > > I have one use case which would be accomplished by
>> cross-system XA
>> > > >> which
>> > > >> > is
>> > > >> > > allowing the producer to block on the synchronous processing of
>> > the
>> > > >> > message
>> > > >> > > by (all? some?) consumers. However I'm not convinced that
>> > > >> cross-system XA
>> > > >> > > is the best solution to this problem, and I'm also not
>> convinced
>> > > this
>> > > >> is
>> > > >> > an
>> > > >> > > important problem to solve. But maybe you have something in
>> mind
>> > > here.
>> > > >> > >
>> > > >> > > -Jay
>> > > >> > >
>> > > >> > >
>> > > >> > >
>> > > >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <
>> ECOMAR@uk.ibm.com>
>> > > >> wrote:
>> > > >> > >
>> > > >> > > > Hi,
>> > > >> > > > thanks, very interesting KIP ... I haven't fully digested it
>> > yet.
>> > > >> > > >
>> > > >> > > > We have many users who choose not to use the Java client,
>> so I
>> > > have
>> > > >> > > > concerns about the added complexity in developing the
>> clients.
>> > > >> > > > A few questions.
>> > > >> > > >
>> > > >> > > > 1 - is mixing transactional and non transactional messages on
>> > the
>> > > >> *same
>> > > >> > > > topic-partition* really a requirement ?
>> > > >> > > > What use case does it satisfy?
>> > > >> > > >
>> > > >> > > > 2 - I guess some clients may only be interested to implement
>> the
>> > > >> > producer
>> > > >> > > > idempotency.
>> > > >> > > > It's not clear how they could be implemented without having
>> to
>> > add
>> > > >> the
>> > > >> > > > transaction capabilities.
>> > > >> > > > As others on this list have said, I too would like to see
>> > > >> idempotency
>> > > >> > as
>> > > >> > > a
>> > > >> > > > more basic feature, on top which txns can be built.
>> > > >> > > >
>> > > >> > > > 3 - The KIP seems focused on a use case where consumption
>> from a
>> > > >> topic
>> > > >> > > and
>> > > >> > > > subsequent production are part of the producer transaction.
>> > > >> > > >
>> > > >> > > > It'd be great to see a way to extend the producer
>> transaction to
>> > > >> > include
>> > > >> > > > additional transactional resources,
>> > > >> > > > so that the consumption from another topic just becomes a
>> > special
>> > > >> case
>> > > >> > of
>> > > >> > > > a more general "distributed" txn.
>> > > >> > > >
>> > > >> > > > Edo
>> > > >> > > > --------------------------------------------------
>> > > >> > > > Edoardo Comar
>> > > >> > > > IBM MessageHub
>> > > >> > > > ecomar@uk.ibm.com
>> > > >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
>> > > >> > > >
>> > > >> > > > IBM United Kingdom Limited Registered in England and Wales
>> with
>> > > >> number
>> > > >> > > > 741598 Registered office: PO Box 41, North Harbour,
>> Portsmouth,
>> > > >> Hants.
>> > > >> > > PO6
>> > > >> > > > 3AU
>> > > >> > > >
>> > > >> > > >
>> > > >> > > >
>> > > >> > > > From:   Guozhang Wang <wa...@gmail.com>
>> > > >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
>> > > >> > > > Date:   30/11/2016 22:20
>> > > >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
>> > > >> > Transactional
>> > > >> > > > Messaging
>> > > >> > > >
>> > > >> > > >
>> > > >> > > >
>> > > >> > > > Hi all,
>> > > >> > > >
>> > > >> > > > I have just created KIP-98 to enhance Kafka with exactly once
>> > > >> delivery
>> > > >> > > > semantics:
>> > > >> > > >
>> > > >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > >> > > > <
>> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > >> > > > >*
>> > > >> > > >
>> > > >> > > > This KIP adds a transactional messaging mechanism along with
>> an
>> > > >> > > idempotent
>> > > >> > > > producer implementation to make sure that 1) duplicated
>> messages
>> > > >> sent
>> > > >> > > from
>> > > >> > > > the same identified producer can be detected on the broker
>> side,
>> > > and
>> > > >> > 2) a
>> > > >> > > > group of messages sent within a transaction will atomically
>> be
>> > > >> either
>> > > >> > > > reflected and fetchable to consumers or not as a whole.
>> > > >> > > >
>> > > >> > > > The above wiki page provides a high-level view of the
>> proposed
>> > > >> changes
>> > > >> > as
>> > > >> > > > well as summarized guarantees. Initial draft of the detailed
>> > > >> > > > implementation
>> > > >> > > > design is described in this Google doc:
>> > > >> > > >
>> > > >> > > > https://docs.google.com/document/d/11Jqy_
>> > > >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> > > >> > > > 0wSw9ra8
>> > > >> > > >
>> > > >> > > >
>> > > >> > > > We would love to hear your comments and suggestions.
>> > > >> > > >
>> > > >> > > > Thanks,
>> > > >> > > >
>> > > >> > > > -- Guozhang
>> > > >> > > >
>> > > >> > > >
>> > > >> > > >
>> > > >> > > > Unless stated otherwise above:
>> > > >> > > > IBM United Kingdom Limited - Registered in England and Wales
>> > with
>> > > >> > number
>> > > >> > > > 741598.
>> > > >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
>> > Hampshire
>> > > >> PO6
>> > > >> > > 3AU
>> > > >> > > >
>> > > >> > > The information contained in this email is strictly
>> confidential
>> > and
>> > > >> for
>> > > >> > > the use of the addressee only, unless otherwise indicated. If
>> you
>> > > are
>> > > >> not
>> > > >> > > the intended recipient, please do not read, copy, use or
>> disclose
>> > to
>> > > >> > others
>> > > >> > > this message or any attachment. Please also notify the sender
>> by
>> > > >> replying
>> > > >> > > to this email or by telephone (+44(020 7896 0011) and then
>> delete
>> > > the
>> > > >> > email
>> > > >> > > and any copies of it. Opinions, conclusion (etc) that do not
>> > relate
>> > > to
>> > > >> > the
>> > > >> > > official business of this company shall be understood as
>> neither
>> > > given
>> > > >> > nor
>> > > >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
>> > > company
>> > > >> > > registered in England and Wales, company number 04008957) and
>> IG
>> > > Index
>> > > >> > > Limited (a company registered in England and Wales, company
>> number
>> > > >> > > 01190902). Registered address at Cannon Bridge House, 25
>> Dowgate
>> > > Hill,
>> > > >> > > London EC4R 2YA. Both IG Markets Limited (register number
>> 195355)
>> > > and
>> > > >> IG
>> > > >> > > Index Limited (register number 114059) are authorised and
>> > regulated
>> > > by
>> > > >> > the
>> > > >> > > Financial Conduct Authority.
>> > > >> > >
>> > > >> > The information contained in this email is strictly confidential
>> and
>> > > for
>> > > >> > the use of the addressee only, unless otherwise indicated. If you
>> > are
>> > > >> not
>> > > >> > the intended recipient, please do not read, copy, use or
>> disclose to
>> > > >> others
>> > > >> > this message or any attachment. Please also notify the sender by
>> > > >> replying
>> > > >> > to this email or by telephone (+44(020 7896 0011) and then delete
>> > the
>> > > >> email
>> > > >> > and any copies of it. Opinions, conclusion (etc) that do not
>> relate
>> > to
>> > > >> the
>> > > >> > official business of this company shall be understood as neither
>> > given
>> > > >> nor
>> > > >> > endorsed by it. IG is a trading name of IG Markets Limited (a
>> > company
>> > > >> > registered in England and Wales, company number 04008957) and IG
>> > Index
>> > > >> > Limited (a company registered in England and Wales, company
>> number
>> > > >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
>> > Hill,
>> > > >> > London EC4R 2YA. Both IG Markets Limited (register number 195355)
>> > and
>> > > IG
>> > > >> > Index Limited (register number 114059) are authorised and
>> regulated
>> > by
>> > > >> the
>> > > >> > Financial Conduct Authority.
>> > > >> >
>> > > >> The information contained in this email is strictly confidential
>> and
>> > for
>> > > >> the use of the addressee only, unless otherwise indicated. If you
>> are
>> > > not
>> > > >> the intended recipient, please do not read, copy, use or disclose
>> to
>> > > others
>> > > >> this message or any attachment. Please also notify the sender by
>> > > replying
>> > > >> to this email or by telephone (+44(020 7896 0011) and then delete
>> the
>> > > email
>> > > >> and any copies of it. Opinions, conclusion (etc) that do not
>> relate to
>> > > the
>> > > >> official business of this company shall be understood as neither
>> given
>> > > nor
>> > > >> endorsed by it. IG is a trading name of IG Markets Limited (a
>> company
>> > > >> registered in England and Wales, company number 04008957) and IG
>> Index
>> > > >> Limited (a company registered in England and Wales, company number
>> > > >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate
>> Hill,
>> > > >> London EC4R 2YA. Both IG Markets Limited (register number 195355)
>> and
>> > IG
>> > > >> Index Limited (register number 114059) are authorised and
>> regulated by
>> > > the
>> > > >> Financial Conduct Authority.
>> > > >>
>> > > >
>> > > >
>> > >
>> >
>> >
>> >
>> > --
>> > -- Guozhang
>> >
>> > --
>> > Thanks,
>> > Neha
>> >
>>
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <ra...@googlemail.com>.
Thank you for the really well thought out KIP. This is a very neat approach
to solve a rather complex problem.

I haven't yet digested the doc. But I have a few comments and questions.

11 a). It feels like memory usage in the consumer is unbounded. If an
application crashed mid-transaction, lots of messages from other
transactions could be written before the transaction timeout eventually
aborted the first transaction. Any consumer reading the log would need to
wait and buffer all the messages before actually consuming any. You could
end up with a consumer that can never consume from the log even if
restarted since it doesn't have sufficient heap space.

It feels like at the very least, consumers should discard buffered records
beyond a certain memory limit and continue to consume and discard until
commit markers. And then refetch committed records. In most cases, this
would fallback to current behavior  providing low latency, but it would
ensure that memory usage is limited and all logs are consumable.


11 b) How will the new buffer.memory being introduced in KIP-81 to control
consumer memory usage be used in the transactional scenario? Would it be
completely ignored? We need to ensure that we don't stop reading from
sockets while waiting for a commit to complete.


12. How will normal log deletion work? Presumably we can have log segments
which contain partial transactions. What happens when that log segment was
deleted (expired)? New consumers replaying logs start consuming partial
transactions?


13. From the discussion thread, I gather that the KIP is based on real use
cases. The KIP does feel very useful. Even so, for some of us who come from
a more traditional messaging background, it will be very useful to
understand the use cases. Can a few be added to the KIP?


14. I think this was mentioned in the doc, but not clear from the KIP.

14a) To use transactions, you need unclean.leader.election=false and
min.insync.replicas > 1 in the broker and producer acks=all - perhaps any
other value with transaction.app.id ought to to be an invalid config for
the producer?

14b) Javadoc in the KIP says producer.app.id - should be transaction.app.id?

14c) To understand the data flow in the KIP, I had to go to the doc to
figure out which flows wait for replication to complete. I did find the
information in the docs, but it will be good to include this in the KIP.


15) As others have mentioned earlier, it may be good to consider a
higher level of durability for transactions. Perhaps for a later KIP, but
it looks like only the messages from the transaction coordinator need to be
flushed to disk to provide that guarantee?





On Mon, Dec 12, 2016 at 6:33 AM, Becket Qin <be...@gmail.com> wrote:

> Thanks for the KIP. It looks I am late to the party :)
>
> I just went through the KIP and doc for one pass. Some quick feedback:
>
>
>    1.
>
>    UnrecognizedMessageException is a synchronous exception? It is good for
>    it to catch the attention, but seems not intuitive because the
> exception is
>    not caused by the message users are currently sending.
>    2.
>
>    We probably want to have the new protocols and message formats to be
>    added to the wiki, at least a separate link to a dedicated google doc.
> The
>    description of some of the new fields seem missing.
>    3.
>
>    It has been a pain in many cases that we do not know the number of
>    messages in a message set, not sure if the OffsetDelta field in the
> wrapper
>    message will address this.
>    4. Maybe it worth clarifying the targeting topology, does the
>    transaction persists after mirrored? Do the consumer and producer of the
>    consume-process-produce loop have to talk to the same cluster?
>    5.
>
>    It would be good to give one concrete producing example involves a
>    transaction
>    6.
>
>    It seems that if there is one long transaction (in terms of the offsets
>    span, but not number of messages), the consumer side memory pressure
> would
>    be big because we do not allow LCO to go beyond LSO.
>    7.
>
>    What happens if a consumer starts up and seeks to the middle of a
>    transaction?
>
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Sun, Dec 11, 2016 at 5:15 PM, Neha Narkhede <ne...@confluent.io> wrote:
>
> > Apurva and Jason -- appreciate the detailed explanation, thank you!
> >
> > Jason -
> >
> > > In many applications, there is already a
> > natural identifier which is already used to divide the workload. For
> > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > applications where there is no natural ID, the user can generate a UUID
> and
> > persist it locally, which is as good as having the server generate it
> >
> > I think for stateful applications that aren't Streams or Connect, it is
> > more work to leverage the strongest guarantees that this proposal offers.
> > But as you say, stateless applications, on the other hand, can still get
> a
> > lot of it by getting deduplication as well as transactional semantics for
> > the lifetime of the producer instance. I think this is fine since I think
> > that the vast majority of producer apps are stateless and will benefit
> from
> > the stronger guarantees. And for the minority of apps that need to
> > carefully use the application id can just fall back to using Streams
> > instead and expressing the send as a simple Streams topology, if needed.
> >
> > I have a few comments on the naming --
> >
> > 1. I think we should consider renaming initTransactions to just init()
> and
> > moving the metadata initialization there. Let's make sure we don't add
> APIs
> > that are relevant to this proposal only. Instead, try to think what we'd
> > propose if we were writing the producer from scratch today. I suspect we
> > would end up with an init() API that would do the metadata initialization
> > as well as the transaction stuff lazily. If so, let's make that change
> now.
> > 2. Along the same lines, let's think about the role of each id that the
> > producer will have and see if everything still makes sense. For instance,
> > we have quite a few per-producer-instance notions -- client.id, a
> producer
> > id and a transaction.app.id, some set via config and some generated
> > on-the-fly. What role does each play, how do they relate to each other
> and
> > is there an opportunity to get rid of any.
> > 3. I think we should definitely consider renaming transaction.app.id to
> > something else. Given that we already have a notion of application.id
> and
> > it represents the entire Streams application, having transaction.app.id
> > that represents a producer instance is confusing. I do understand that,
> for
> > Streams, the user doesn't have to set transaction.app.id as it will
> likely
> > be application.id+taskId (am I understanding that correctly?)
> >
> > On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wa...@gmail.com> wrote:
> >
> > Onur,
> >
> > I understand your question now. So it is indeed possible that after
> > commitTxn() returned the messages could still be lost permanently if all
> > replicas failed before the data was flushed to disk. This is the virtue
> of
> > Kafka's design to reply on replication (probably in memory) for high
> > availability, hence async flushing. This scenario already exist today and
> > KIP-98 did not intend to change this factor in any ways.
> >
> > Guozhang
> >
> >
> > On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <
> > onurkaraman.apache@gmail.com>
> > wrote:
> >
> > > In other words, we can see inconsistency when the transaction log
> reports
> > > the transaction as COMMITTED while the markers and data corresponding
> to
> > > the transaction itself on the user partitions may have been partially
> > lost
> > > after-the-fact because of kafka's durability guarantees.
> > >
> > > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> > > onurkaraman.apache@gmail.com>
> > > wrote:
> > >
> > > > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> > > > markers (and a subset of the user's data produced in the transaction
> > for
> > > > that matter) getting lost from the delta between the write and flush.
> > > >
> > > > KIP-98 relies on replicas writing to logs, so transaction durability
> is
> > > > effectively limited by kafka's definition of a "write success"
> meaning
> > > > written but not flushed to disk.
> > > >
> > > > I mentioned RF=1 not because of availability but actually to
> highlight
> > a
> > > > corner-case durability scenario where the single replica
> participating
> > in
> > > > the transaction experiences a hard failure after the write but before
> > the
> > > > flush, causing the transaction to have partial data loss.
> > > >
> > > > Is this level of durability okay or do we want stronger guarantees
> for
> > > the
> > > > transaction? Basically what I'm wondering is if KIP-98 necessitates
> > > kafka'a
> > > > definition of a "write success" to be extended from "written" to an
> > > > optional "written and flushed to disk".
> > > >
> > > > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <
> Michael.Pearce@ig.com
> > >
> > > > wrote:
> > > >
> > > >> Apologies on the spelling.
> > > >>
> > > >> *Hi Jay,
> > > >> ________________________________________
> > > >> From: Michael Pearce <Mi...@ig.com>
> > > >> Sent: Friday, December 9, 2016 7:52:25 PM
> > > >> To: dev@kafka.apache.org
> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > >> Messaging
> > > >>
> > > >> Hi Jey
> > > >>
> > > >> 1) I agree, these should be used to add this in a future kip if ever
> > was
> > > >> enough of a case. As stated for us I think for these systems we will
> > > keep
> > > >> our JMS solutions there.  I think maybe in the docs when this
> feature
> > is
> > > >> written up, one should redirect users to alternative options such as
> > jms
> > > >> brokers, for these use cases.
> > > >>
> > > >> 2) I think this kip needs to be mindful and actually own to make
> sure
> > > >> things are implemented in a way to make future enchancement easy/or
> at
> > > >> least extensible. Having to in future rework things and correct
> > historic
> > > >> decisions is expensive as already finding.
> > > >>
> > > >> Sent using OWA for iPhone
> > > >> ________________________________________
> > > >> From: Jay Kreps <ja...@confluent.io>
> > > >> Sent: Friday, December 9, 2016 7:19:59 PM
> > > >> To: dev@kafka.apache.org
> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > >> Messaging
> > > >>
> > > >> Hey Michael,
> > > >>
> > > >> Yeah, I don't think you need to go into the details of whatever you
> > guys
> > > >> have. I think several people in the thread said "let's do XA
> > > transactions
> > > >> too!" Obviously in a world where features were free and always
> worked
> > > >> perfectly we would! I've probably talked to about 100 people about
> > their
> > > >> use of XA transactions in different systems and my observation has
> > been
> > > >> (a)
> > > >> they are a bit of an operational nightmare, (b) the use cases i've
> > > >> understood don't actually require full XA transactions they actually
> > > >> require a much weaker and easier to guarantee property. The result
> is
> > > you
> > > >> pay a big complexity cost for a guarantee much stronger than what
> you
> > > >> wanted. My sense is that this opinion is broadly shared by the
> > > distributed
> > > >> systems community at large and by Kafka folks in particular.
> > > >>
> > > >> I'm a contrarian so I think it is great not to be too swayed by
> > "common
> > > >> wisdom" though. Five years ago there was a consensus that
> distributed
> > > >> transactions were too hard to implement in an operationally sound
> way,
> > > >> which i think was not correct, so the bad reputation for
> cross-system
> > > >> transactions may be equally wrong!
> > > >>
> > > >> To build a compelling case this is wrong I think two things need to
> be
> > > >> done:
> > > >>
> > > >>    1. Build a case that there are a large/important set of use cases
> > > that
> > > >>    cannot be solved with two independent transactions (as i
> > described),
> > > >> and
> > > >>    that these use cases are things Kafka should be able to do.
> > > >>    2. Come up with the concrete extensions to the KIP-98 proposal
> that
> > > >>    would enable an operationally sound implementation for pluggable
> > > >>    multi-system XA.
> > > >>
> > > >> -Jay
> > > >>
> > > >>
> > > >>
> > > >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <
> > Michael.Pearce@ig.com>
> > > >> wrote:
> > > >>
> > > >> > Hi Jay,
> > > >> >
> > > >> > I can't go too deep into exact implantation due to no NDA. So
> > > apologies
> > > >> > here.
> > > >> >
> > > >> > Essentially we have multiple processes each owning selection of
> > > accounts
> > > >> > so on general flows an action for an account just needs to be
> > managed
> > > >> local
> > > >> > to the owning node, happy days ever change is handled as a tick
> tock
> > > >> change.
> > > >> >
> > > >> > Unfortunately when a transfer occurs we need the two processes to
> > > >> > co-ordinate their transaction, we also need to ensure both don't
> > > >> continue
> > > >> > other actions/changesl, we do this using a data grid technology.
> > This
> > > >> grid
> > > >> > technology supports transaction manager that we couple into
> > currently
> > > >> our
> > > >> > jms provider which supports full XA transactions as such we can
> > manage
> > > >> the
> > > >> > production of the change messages out the system transactionally
> as
> > > >> well as
> > > >> > the in grid state.
> > > >> >
> > > >> > The obvious arguement here is should we even look to move this
> flow
> > > off
> > > >> > JMS then. We prob shouldn't nor will do this.
> > > >> >
> > > >> > The point is that I think saying Kafka supports transactions but
> > then
> > > >> not
> > > >> > supporting it as per the traditional sense leads to developers
> > > expecting
> > > >> > similar behaviour and will cause issues in prod when they find it
> > > >> doesn't
> > > >> > work as they're used to.
> > > >> >
> > > >> > As my other response earlier, is there a better name to describe
> > this
> > > >> > feature, if we're not implementing transactions to the traditional
> > > >> > transaction expected, to avoid this confusion?
> > > >> >
> > > >> >
> > > >> > Sent using OWA for iPhone
> > > >> > ________________________________________
> > > >> > From: Jay Kreps <ja...@confluent.io>
> > > >> > Sent: Friday, December 9, 2016 6:08:07 PM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > > >> > Messaging
> > > >> >
> > > >> > Hey Michael,
> > > >> >
> > > >> > Doesn't that example have more to do with applying the update
> > against
> > > >> two
> > > >> > rows in a single transaction? That is, clearly the write to Kafka
> > > needs
> > > >> to
> > > >> > be "transactional" and the write to the destination needs to be
> > > >> > transactional, but it's not clear to me that you need isolation
> that
> > > >> spans
> > > >> > both operations. Can you dive into the system architecture a bit
> > more
> > > >> and
> > > >> > explain why Kafka needs to participate in the same transaction as
> > the
> > > >> > destination system?
> > > >> >
> > > >> > -Jay
> > > >> >
> > > >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> > > Michael.Pearce@ig.com>
> > > >> > wrote:
> > > >> >
> > > >> > > Usecase in IG:
> > > >> > >
> > > >> > > Fund transfer between accounts. When we debit one account and
> fund
> > > >> > another
> > > >> > > we must ensure the records to both occur as an acid action, and
> as
> > a
> > > >> > single
> > > >> > > transaction.
> > > >> > >
> > > >> > > Today we achieve this because we have jms, as such we can do the
> > > >> actions
> > > >> > > needed in an xa transaction across both the accounts. To move
> this
> > > >> flow
> > > >> > to
> > > >> > > Kafka we would need support of XA transaction.
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > > Sent using OWA for iPhone
> > > >> > > ________________________________________
> > > >> > > From: Michael Pearce <Mi...@ig.com>
> > > >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > > Messaging
> > > >> > >
> > > >> > > Hi Jay,
> > > >> > >
> > > >> > > For me having an XA transaction allows for ensuring ACID across
> my
> > > >> > > application.
> > > >> > >
> > > >> > > I believe it is part of the JMS api, and obviously JMS still is
> in
> > > >> > > enterprise very widely adopted for Messaging transport , so
> > > obviously
> > > >> to
> > > >> > > say it isn't widely used i think is ignoring a whole range of
> > users.
> > > >> Like
> > > >> > > wise I believe frameworks like spring etc fully support it more
> > > >> evidence
> > > >> > of
> > > >> > > its wide adoption.
> > > >> > >
> > > >> > > On this note personally we try to avoid transactions entirely in
> > our
> > > >> > flows
> > > >> > > for performance and simplicity. but we do alas unfortunately
> have
> > > one
> > > >> or
> > > >> > > two places we cannot ignore it.
> > > >> > >
> > > >> > > Cheers
> > > >> > > Mike
> > > >> > >
> > > >> > > Sent using OWA for iPhone
> > > >> > > ________________________________________
> > > >> > > From: Jay Kreps <ja...@confluent.io>
> > > >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > > Messaging
> > > >> > >
> > > >> > > Hey Edoardo,
> > > >> > >
> > > >> > > For (3) can you outline what you think the benefit and use cases
> > > for a
> > > >> > more
> > > >> > > general cross-system XA feature would be an what changes to the
> > > >> proposal
> > > >> > > would be required to enable it? When I have asked people who
> > wanted
> > > >> > > cross-system XA in the past what they wanted it for, I haven't
> > > really
> > > >> > > gotten any answers that made sense. Every person really wanted
> > > >> something
> > > >> > > that would be better solved by a transactional (or idempotent)
> > write
> > > >> to
> > > >> > > Kafka followed by an independent transactional (or idempotent)
> > > >> > consumption
> > > >> > > (which this proposal enables). For the use cases they described
> > > tying
> > > >> > these
> > > >> > > two things together had no advantage and many disadvantages.
> > > >> > >
> > > >> > > I have one use case which would be accomplished by cross-system
> XA
> > > >> which
> > > >> > is
> > > >> > > allowing the producer to block on the synchronous processing of
> > the
> > > >> > message
> > > >> > > by (all? some?) consumers. However I'm not convinced that
> > > >> cross-system XA
> > > >> > > is the best solution to this problem, and I'm also not convinced
> > > this
> > > >> is
> > > >> > an
> > > >> > > important problem to solve. But maybe you have something in mind
> > > here.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <
> ECOMAR@uk.ibm.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Hi,
> > > >> > > > thanks, very interesting KIP ... I haven't fully digested it
> > yet.
> > > >> > > >
> > > >> > > > We have many users who choose not to use the Java client,  so
> I
> > > have
> > > >> > > > concerns about the added complexity in developing the clients.
> > > >> > > > A few questions.
> > > >> > > >
> > > >> > > > 1 - is mixing transactional and non transactional messages on
> > the
> > > >> *same
> > > >> > > > topic-partition* really a requirement ?
> > > >> > > > What use case does it satisfy?
> > > >> > > >
> > > >> > > > 2 - I guess some clients may only be interested to implement
> the
> > > >> > producer
> > > >> > > > idempotency.
> > > >> > > > It's not clear how they could be implemented without having to
> > add
> > > >> the
> > > >> > > > transaction capabilities.
> > > >> > > > As others on this list have said, I too would like to see
> > > >> idempotency
> > > >> > as
> > > >> > > a
> > > >> > > > more basic feature, on top which txns can be built.
> > > >> > > >
> > > >> > > > 3 - The KIP seems focused on a use case where consumption
> from a
> > > >> topic
> > > >> > > and
> > > >> > > > subsequent production are part of the producer transaction.
> > > >> > > >
> > > >> > > > It'd be great to see a way to extend the producer transaction
> to
> > > >> > include
> > > >> > > > additional transactional resources,
> > > >> > > > so that the consumption from another topic just becomes a
> > special
> > > >> case
> > > >> > of
> > > >> > > > a more general "distributed" txn.
> > > >> > > >
> > > >> > > > Edo
> > > >> > > > --------------------------------------------------
> > > >> > > > Edoardo Comar
> > > >> > > > IBM MessageHub
> > > >> > > > ecomar@uk.ibm.com
> > > >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > > >> > > >
> > > >> > > > IBM United Kingdom Limited Registered in England and Wales
> with
> > > >> number
> > > >> > > > 741598 Registered office: PO Box 41, North Harbour,
> Portsmouth,
> > > >> Hants.
> > > >> > > PO6
> > > >> > > > 3AU
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > From:   Guozhang Wang <wa...@gmail.com>
> > > >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > >> > > > Date:   30/11/2016 22:20
> > > >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> > > >> > Transactional
> > > >> > > > Messaging
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > Hi all,
> > > >> > > >
> > > >> > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > >> delivery
> > > >> > > > semantics:
> > > >> > > >
> > > >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > <
> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > >*
> > > >> > > >
> > > >> > > > This KIP adds a transactional messaging mechanism along with
> an
> > > >> > > idempotent
> > > >> > > > producer implementation to make sure that 1) duplicated
> messages
> > > >> sent
> > > >> > > from
> > > >> > > > the same identified producer can be detected on the broker
> side,
> > > and
> > > >> > 2) a
> > > >> > > > group of messages sent within a transaction will atomically be
> > > >> either
> > > >> > > > reflected and fetchable to consumers or not as a whole.
> > > >> > > >
> > > >> > > > The above wiki page provides a high-level view of the proposed
> > > >> changes
> > > >> > as
> > > >> > > > well as summarized guarantees. Initial draft of the detailed
> > > >> > > > implementation
> > > >> > > > design is described in this Google doc:
> > > >> > > >
> > > >> > > > https://docs.google.com/document/d/11Jqy_
> > > >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > >> > > > 0wSw9ra8
> > > >> > > >
> > > >> > > >
> > > >> > > > We would love to hear your comments and suggestions.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > -- Guozhang
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > Unless stated otherwise above:
> > > >> > > > IBM United Kingdom Limited - Registered in England and Wales
> > with
> > > >> > number
> > > >> > > > 741598.
> > > >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
> > Hampshire
> > > >> PO6
> > > >> > > 3AU
> > > >> > > >
> > > >> > > The information contained in this email is strictly confidential
> > and
> > > >> for
> > > >> > > the use of the addressee only, unless otherwise indicated. If
> you
> > > are
> > > >> not
> > > >> > > the intended recipient, please do not read, copy, use or
> disclose
> > to
> > > >> > others
> > > >> > > this message or any attachment. Please also notify the sender by
> > > >> replying
> > > >> > > to this email or by telephone (+44(020 7896 0011) and then
> delete
> > > the
> > > >> > email
> > > >> > > and any copies of it. Opinions, conclusion (etc) that do not
> > relate
> > > to
> > > >> > the
> > > >> > > official business of this company shall be understood as neither
> > > given
> > > >> > nor
> > > >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> > > company
> > > >> > > registered in England and Wales, company number 04008957) and IG
> > > Index
> > > >> > > Limited (a company registered in England and Wales, company
> number
> > > >> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > > Hill,
> > > >> > > London EC4R 2YA. Both IG Markets Limited (register number
> 195355)
> > > and
> > > >> IG
> > > >> > > Index Limited (register number 114059) are authorised and
> > regulated
> > > by
> > > >> > the
> > > >> > > Financial Conduct Authority.
> > > >> > >
> > > >> > The information contained in this email is strictly confidential
> and
> > > for
> > > >> > the use of the addressee only, unless otherwise indicated. If you
> > are
> > > >> not
> > > >> > the intended recipient, please do not read, copy, use or disclose
> to
> > > >> others
> > > >> > this message or any attachment. Please also notify the sender by
> > > >> replying
> > > >> > to this email or by telephone (+44(020 7896 0011) and then delete
> > the
> > > >> email
> > > >> > and any copies of it. Opinions, conclusion (etc) that do not
> relate
> > to
> > > >> the
> > > >> > official business of this company shall be understood as neither
> > given
> > > >> nor
> > > >> > endorsed by it. IG is a trading name of IG Markets Limited (a
> > company
> > > >> > registered in England and Wales, company number 04008957) and IG
> > Index
> > > >> > Limited (a company registered in England and Wales, company number
> > > >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > Hill,
> > > >> > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> > and
> > > IG
> > > >> > Index Limited (register number 114059) are authorised and
> regulated
> > by
> > > >> the
> > > >> > Financial Conduct Authority.
> > > >> >
> > > >> The information contained in this email is strictly confidential and
> > for
> > > >> the use of the addressee only, unless otherwise indicated. If you
> are
> > > not
> > > >> the intended recipient, please do not read, copy, use or disclose to
> > > others
> > > >> this message or any attachment. Please also notify the sender by
> > > replying
> > > >> to this email or by telephone (+44(020 7896 0011) and then delete
> the
> > > email
> > > >> and any copies of it. Opinions, conclusion (etc) that do not relate
> to
> > > the
> > > >> official business of this company shall be understood as neither
> given
> > > nor
> > > >> endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> > > >> registered in England and Wales, company number 04008957) and IG
> Index
> > > >> Limited (a company registered in England and Wales, company number
> > > >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
> > > >> London EC4R 2YA. Both IG Markets Limited (register number 195355)
> and
> > IG
> > > >> Index Limited (register number 114059) are authorised and regulated
> by
> > > the
> > > >> Financial Conduct Authority.
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
> > --
> > Thanks,
> > Neha
> >
>



-- 
Regards,

Rajini

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Becket Qin <be...@gmail.com>.
Thanks for the KIP. It looks I am late to the party :)

I just went through the KIP and doc for one pass. Some quick feedback:


   1.

   UnrecognizedMessageException is a synchronous exception? It is good for
   it to catch the attention, but seems not intuitive because the exception is
   not caused by the message users are currently sending.
   2.

   We probably want to have the new protocols and message formats to be
   added to the wiki, at least a separate link to a dedicated google doc. The
   description of some of the new fields seem missing.
   3.

   It has been a pain in many cases that we do not know the number of
   messages in a message set, not sure if the OffsetDelta field in the wrapper
   message will address this.
   4. Maybe it worth clarifying the targeting topology, does the
   transaction persists after mirrored? Do the consumer and producer of the
   consume-process-produce loop have to talk to the same cluster?
   5.

   It would be good to give one concrete producing example involves a
   transaction
   6.

   It seems that if there is one long transaction (in terms of the offsets
   span, but not number of messages), the consumer side memory pressure would
   be big because we do not allow LCO to go beyond LSO.
   7.

   What happens if a consumer starts up and seeks to the middle of a
   transaction?


Thanks,

Jiangjie (Becket) Qin

On Sun, Dec 11, 2016 at 5:15 PM, Neha Narkhede <ne...@confluent.io> wrote:

> Apurva and Jason -- appreciate the detailed explanation, thank you!
>
> Jason -
>
> > In many applications, there is already a
> natural identifier which is already used to divide the workload. For
> example, in Kafka Streams and Kafka Connect, we have a taskId. For
> applications where there is no natural ID, the user can generate a UUID and
> persist it locally, which is as good as having the server generate it
>
> I think for stateful applications that aren't Streams or Connect, it is
> more work to leverage the strongest guarantees that this proposal offers.
> But as you say, stateless applications, on the other hand, can still get a
> lot of it by getting deduplication as well as transactional semantics for
> the lifetime of the producer instance. I think this is fine since I think
> that the vast majority of producer apps are stateless and will benefit from
> the stronger guarantees. And for the minority of apps that need to
> carefully use the application id can just fall back to using Streams
> instead and expressing the send as a simple Streams topology, if needed.
>
> I have a few comments on the naming --
>
> 1. I think we should consider renaming initTransactions to just init() and
> moving the metadata initialization there. Let's make sure we don't add APIs
> that are relevant to this proposal only. Instead, try to think what we'd
> propose if we were writing the producer from scratch today. I suspect we
> would end up with an init() API that would do the metadata initialization
> as well as the transaction stuff lazily. If so, let's make that change now.
> 2. Along the same lines, let's think about the role of each id that the
> producer will have and see if everything still makes sense. For instance,
> we have quite a few per-producer-instance notions -- client.id, a producer
> id and a transaction.app.id, some set via config and some generated
> on-the-fly. What role does each play, how do they relate to each other and
> is there an opportunity to get rid of any.
> 3. I think we should definitely consider renaming transaction.app.id to
> something else. Given that we already have a notion of application.id and
> it represents the entire Streams application, having transaction.app.id
> that represents a producer instance is confusing. I do understand that, for
> Streams, the user doesn't have to set transaction.app.id as it will likely
> be application.id+taskId (am I understanding that correctly?)
>
> On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> Onur,
>
> I understand your question now. So it is indeed possible that after
> commitTxn() returned the messages could still be lost permanently if all
> replicas failed before the data was flushed to disk. This is the virtue of
> Kafka's design to reply on replication (probably in memory) for high
> availability, hence async flushing. This scenario already exist today and
> KIP-98 did not intend to change this factor in any ways.
>
> Guozhang
>
>
> On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > In other words, we can see inconsistency when the transaction log reports
> > the transaction as COMMITTED while the markers and data corresponding to
> > the transaction itself on the user partitions may have been partially
> lost
> > after-the-fact because of kafka's durability guarantees.
> >
> > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> > onurkaraman.apache@gmail.com>
> > wrote:
> >
> > > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> > > markers (and a subset of the user's data produced in the transaction
> for
> > > that matter) getting lost from the delta between the write and flush.
> > >
> > > KIP-98 relies on replicas writing to logs, so transaction durability is
> > > effectively limited by kafka's definition of a "write success" meaning
> > > written but not flushed to disk.
> > >
> > > I mentioned RF=1 not because of availability but actually to highlight
> a
> > > corner-case durability scenario where the single replica participating
> in
> > > the transaction experiences a hard failure after the write but before
> the
> > > flush, causing the transaction to have partial data loss.
> > >
> > > Is this level of durability okay or do we want stronger guarantees for
> > the
> > > transaction? Basically what I'm wondering is if KIP-98 necessitates
> > kafka'a
> > > definition of a "write success" to be extended from "written" to an
> > > optional "written and flushed to disk".
> > >
> > > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <Michael.Pearce@ig.com
> >
> > > wrote:
> > >
> > >> Apologies on the spelling.
> > >>
> > >> *Hi Jay,
> > >> ________________________________________
> > >> From: Michael Pearce <Mi...@ig.com>
> > >> Sent: Friday, December 9, 2016 7:52:25 PM
> > >> To: dev@kafka.apache.org
> > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > >> Messaging
> > >>
> > >> Hi Jey
> > >>
> > >> 1) I agree, these should be used to add this in a future kip if ever
> was
> > >> enough of a case. As stated for us I think for these systems we will
> > keep
> > >> our JMS solutions there.  I think maybe in the docs when this feature
> is
> > >> written up, one should redirect users to alternative options such as
> jms
> > >> brokers, for these use cases.
> > >>
> > >> 2) I think this kip needs to be mindful and actually own to make sure
> > >> things are implemented in a way to make future enchancement easy/or at
> > >> least extensible. Having to in future rework things and correct
> historic
> > >> decisions is expensive as already finding.
> > >>
> > >> Sent using OWA for iPhone
> > >> ________________________________________
> > >> From: Jay Kreps <ja...@confluent.io>
> > >> Sent: Friday, December 9, 2016 7:19:59 PM
> > >> To: dev@kafka.apache.org
> > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > >> Messaging
> > >>
> > >> Hey Michael,
> > >>
> > >> Yeah, I don't think you need to go into the details of whatever you
> guys
> > >> have. I think several people in the thread said "let's do XA
> > transactions
> > >> too!" Obviously in a world where features were free and always worked
> > >> perfectly we would! I've probably talked to about 100 people about
> their
> > >> use of XA transactions in different systems and my observation has
> been
> > >> (a)
> > >> they are a bit of an operational nightmare, (b) the use cases i've
> > >> understood don't actually require full XA transactions they actually
> > >> require a much weaker and easier to guarantee property. The result is
> > you
> > >> pay a big complexity cost for a guarantee much stronger than what you
> > >> wanted. My sense is that this opinion is broadly shared by the
> > distributed
> > >> systems community at large and by Kafka folks in particular.
> > >>
> > >> I'm a contrarian so I think it is great not to be too swayed by
> "common
> > >> wisdom" though. Five years ago there was a consensus that distributed
> > >> transactions were too hard to implement in an operationally sound way,
> > >> which i think was not correct, so the bad reputation for cross-system
> > >> transactions may be equally wrong!
> > >>
> > >> To build a compelling case this is wrong I think two things need to be
> > >> done:
> > >>
> > >>    1. Build a case that there are a large/important set of use cases
> > that
> > >>    cannot be solved with two independent transactions (as i
> described),
> > >> and
> > >>    that these use cases are things Kafka should be able to do.
> > >>    2. Come up with the concrete extensions to the KIP-98 proposal that
> > >>    would enable an operationally sound implementation for pluggable
> > >>    multi-system XA.
> > >>
> > >> -Jay
> > >>
> > >>
> > >>
> > >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <
> Michael.Pearce@ig.com>
> > >> wrote:
> > >>
> > >> > Hi Jay,
> > >> >
> > >> > I can't go too deep into exact implantation due to no NDA. So
> > apologies
> > >> > here.
> > >> >
> > >> > Essentially we have multiple processes each owning selection of
> > accounts
> > >> > so on general flows an action for an account just needs to be
> managed
> > >> local
> > >> > to the owning node, happy days ever change is handled as a tick tock
> > >> change.
> > >> >
> > >> > Unfortunately when a transfer occurs we need the two processes to
> > >> > co-ordinate their transaction, we also need to ensure both don't
> > >> continue
> > >> > other actions/changesl, we do this using a data grid technology.
> This
> > >> grid
> > >> > technology supports transaction manager that we couple into
> currently
> > >> our
> > >> > jms provider which supports full XA transactions as such we can
> manage
> > >> the
> > >> > production of the change messages out the system transactionally as
> > >> well as
> > >> > the in grid state.
> > >> >
> > >> > The obvious arguement here is should we even look to move this flow
> > off
> > >> > JMS then. We prob shouldn't nor will do this.
> > >> >
> > >> > The point is that I think saying Kafka supports transactions but
> then
> > >> not
> > >> > supporting it as per the traditional sense leads to developers
> > expecting
> > >> > similar behaviour and will cause issues in prod when they find it
> > >> doesn't
> > >> > work as they're used to.
> > >> >
> > >> > As my other response earlier, is there a better name to describe
> this
> > >> > feature, if we're not implementing transactions to the traditional
> > >> > transaction expected, to avoid this confusion?
> > >> >
> > >> >
> > >> > Sent using OWA for iPhone
> > >> > ________________________________________
> > >> > From: Jay Kreps <ja...@confluent.io>
> > >> > Sent: Friday, December 9, 2016 6:08:07 PM
> > >> > To: dev@kafka.apache.org
> > >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > >> > Messaging
> > >> >
> > >> > Hey Michael,
> > >> >
> > >> > Doesn't that example have more to do with applying the update
> against
> > >> two
> > >> > rows in a single transaction? That is, clearly the write to Kafka
> > needs
> > >> to
> > >> > be "transactional" and the write to the destination needs to be
> > >> > transactional, but it's not clear to me that you need isolation that
> > >> spans
> > >> > both operations. Can you dive into the system architecture a bit
> more
> > >> and
> > >> > explain why Kafka needs to participate in the same transaction as
> the
> > >> > destination system?
> > >> >
> > >> > -Jay
> > >> >
> > >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> > Michael.Pearce@ig.com>
> > >> > wrote:
> > >> >
> > >> > > Usecase in IG:
> > >> > >
> > >> > > Fund transfer between accounts. When we debit one account and fund
> > >> > another
> > >> > > we must ensure the records to both occur as an acid action, and as
> a
> > >> > single
> > >> > > transaction.
> > >> > >
> > >> > > Today we achieve this because we have jms, as such we can do the
> > >> actions
> > >> > > needed in an xa transaction across both the accounts. To move this
> > >> flow
> > >> > to
> > >> > > Kafka we would need support of XA transaction.
> > >> > >
> > >> > >
> > >> > >
> > >> > > Sent using OWA for iPhone
> > >> > > ________________________________________
> > >> > > From: Michael Pearce <Mi...@ig.com>
> > >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> > >> > > To: dev@kafka.apache.org
> > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > >> > > Messaging
> > >> > >
> > >> > > Hi Jay,
> > >> > >
> > >> > > For me having an XA transaction allows for ensuring ACID across my
> > >> > > application.
> > >> > >
> > >> > > I believe it is part of the JMS api, and obviously JMS still is in
> > >> > > enterprise very widely adopted for Messaging transport , so
> > obviously
> > >> to
> > >> > > say it isn't widely used i think is ignoring a whole range of
> users.
> > >> Like
> > >> > > wise I believe frameworks like spring etc fully support it more
> > >> evidence
> > >> > of
> > >> > > its wide adoption.
> > >> > >
> > >> > > On this note personally we try to avoid transactions entirely in
> our
> > >> > flows
> > >> > > for performance and simplicity. but we do alas unfortunately have
> > one
> > >> or
> > >> > > two places we cannot ignore it.
> > >> > >
> > >> > > Cheers
> > >> > > Mike
> > >> > >
> > >> > > Sent using OWA for iPhone
> > >> > > ________________________________________
> > >> > > From: Jay Kreps <ja...@confluent.io>
> > >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> > >> > > To: dev@kafka.apache.org
> > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > >> > > Messaging
> > >> > >
> > >> > > Hey Edoardo,
> > >> > >
> > >> > > For (3) can you outline what you think the benefit and use cases
> > for a
> > >> > more
> > >> > > general cross-system XA feature would be an what changes to the
> > >> proposal
> > >> > > would be required to enable it? When I have asked people who
> wanted
> > >> > > cross-system XA in the past what they wanted it for, I haven't
> > really
> > >> > > gotten any answers that made sense. Every person really wanted
> > >> something
> > >> > > that would be better solved by a transactional (or idempotent)
> write
> > >> to
> > >> > > Kafka followed by an independent transactional (or idempotent)
> > >> > consumption
> > >> > > (which this proposal enables). For the use cases they described
> > tying
> > >> > these
> > >> > > two things together had no advantage and many disadvantages.
> > >> > >
> > >> > > I have one use case which would be accomplished by cross-system XA
> > >> which
> > >> > is
> > >> > > allowing the producer to block on the synchronous processing of
> the
> > >> > message
> > >> > > by (all? some?) consumers. However I'm not convinced that
> > >> cross-system XA
> > >> > > is the best solution to this problem, and I'm also not convinced
> > this
> > >> is
> > >> > an
> > >> > > important problem to solve. But maybe you have something in mind
> > here.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > >
> > >> > >
> > >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
> > >> wrote:
> > >> > >
> > >> > > > Hi,
> > >> > > > thanks, very interesting KIP ... I haven't fully digested it
> yet.
> > >> > > >
> > >> > > > We have many users who choose not to use the Java client,  so I
> > have
> > >> > > > concerns about the added complexity in developing the clients.
> > >> > > > A few questions.
> > >> > > >
> > >> > > > 1 - is mixing transactional and non transactional messages on
> the
> > >> *same
> > >> > > > topic-partition* really a requirement ?
> > >> > > > What use case does it satisfy?
> > >> > > >
> > >> > > > 2 - I guess some clients may only be interested to implement the
> > >> > producer
> > >> > > > idempotency.
> > >> > > > It's not clear how they could be implemented without having to
> add
> > >> the
> > >> > > > transaction capabilities.
> > >> > > > As others on this list have said, I too would like to see
> > >> idempotency
> > >> > as
> > >> > > a
> > >> > > > more basic feature, on top which txns can be built.
> > >> > > >
> > >> > > > 3 - The KIP seems focused on a use case where consumption from a
> > >> topic
> > >> > > and
> > >> > > > subsequent production are part of the producer transaction.
> > >> > > >
> > >> > > > It'd be great to see a way to extend the producer transaction to
> > >> > include
> > >> > > > additional transactional resources,
> > >> > > > so that the consumption from another topic just becomes a
> special
> > >> case
> > >> > of
> > >> > > > a more general "distributed" txn.
> > >> > > >
> > >> > > > Edo
> > >> > > > --------------------------------------------------
> > >> > > > Edoardo Comar
> > >> > > > IBM MessageHub
> > >> > > > ecomar@uk.ibm.com
> > >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >> > > >
> > >> > > > IBM United Kingdom Limited Registered in England and Wales with
> > >> number
> > >> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
> > >> Hants.
> > >> > > PO6
> > >> > > > 3AU
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > From:   Guozhang Wang <wa...@gmail.com>
> > >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > >> > > > Date:   30/11/2016 22:20
> > >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> > >> > Transactional
> > >> > > > Messaging
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > Hi all,
> > >> > > >
> > >> > > > I have just created KIP-98 to enhance Kafka with exactly once
> > >> delivery
> > >> > > > semantics:
> > >> > > >
> > >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >> > > > <
> > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >> > > > >*
> > >> > > >
> > >> > > > This KIP adds a transactional messaging mechanism along with an
> > >> > > idempotent
> > >> > > > producer implementation to make sure that 1) duplicated messages
> > >> sent
> > >> > > from
> > >> > > > the same identified producer can be detected on the broker side,
> > and
> > >> > 2) a
> > >> > > > group of messages sent within a transaction will atomically be
> > >> either
> > >> > > > reflected and fetchable to consumers or not as a whole.
> > >> > > >
> > >> > > > The above wiki page provides a high-level view of the proposed
> > >> changes
> > >> > as
> > >> > > > well as summarized guarantees. Initial draft of the detailed
> > >> > > > implementation
> > >> > > > design is described in this Google doc:
> > >> > > >
> > >> > > > https://docs.google.com/document/d/11Jqy_
> > >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > >> > > > 0wSw9ra8
> > >> > > >
> > >> > > >
> > >> > > > We would love to hear your comments and suggestions.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > -- Guozhang
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > Unless stated otherwise above:
> > >> > > > IBM United Kingdom Limited - Registered in England and Wales
> with
> > >> > number
> > >> > > > 741598.
> > >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
> Hampshire
> > >> PO6
> > >> > > 3AU
> > >> > > >
> > >> > > The information contained in this email is strictly confidential
> and
> > >> for
> > >> > > the use of the addressee only, unless otherwise indicated. If you
> > are
> > >> not
> > >> > > the intended recipient, please do not read, copy, use or disclose
> to
> > >> > others
> > >> > > this message or any attachment. Please also notify the sender by
> > >> replying
> > >> > > to this email or by telephone (+44(020 7896 0011) and then delete
> > the
> > >> > email
> > >> > > and any copies of it. Opinions, conclusion (etc) that do not
> relate
> > to
> > >> > the
> > >> > > official business of this company shall be understood as neither
> > given
> > >> > nor
> > >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> > company
> > >> > > registered in England and Wales, company number 04008957) and IG
> > Index
> > >> > > Limited (a company registered in England and Wales, company number
> > >> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > Hill,
> > >> > > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> > and
> > >> IG
> > >> > > Index Limited (register number 114059) are authorised and
> regulated
> > by
> > >> > the
> > >> > > Financial Conduct Authority.
> > >> > >
> > >> > The information contained in this email is strictly confidential and
> > for
> > >> > the use of the addressee only, unless otherwise indicated. If you
> are
> > >> not
> > >> > the intended recipient, please do not read, copy, use or disclose to
> > >> others
> > >> > this message or any attachment. Please also notify the sender by
> > >> replying
> > >> > to this email or by telephone (+44(020 7896 0011) and then delete
> the
> > >> email
> > >> > and any copies of it. Opinions, conclusion (etc) that do not relate
> to
> > >> the
> > >> > official business of this company shall be understood as neither
> given
> > >> nor
> > >> > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> > >> > registered in England and Wales, company number 04008957) and IG
> Index
> > >> > Limited (a company registered in England and Wales, company number
> > >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
> > >> > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> and
> > IG
> > >> > Index Limited (register number 114059) are authorised and regulated
> by
> > >> the
> > >> > Financial Conduct Authority.
> > >> >
> > >> The information contained in this email is strictly confidential and
> for
> > >> the use of the addressee only, unless otherwise indicated. If you are
> > not
> > >> the intended recipient, please do not read, copy, use or disclose to
> > others
> > >> this message or any attachment. Please also notify the sender by
> > replying
> > >> to this email or by telephone (+44(020 7896 0011) and then delete the
> > email
> > >> and any copies of it. Opinions, conclusion (etc) that do not relate to
> > the
> > >> official business of this company shall be understood as neither given
> > nor
> > >> endorsed by it. IG is a trading name of IG Markets Limited (a company
> > >> registered in England and Wales, company number 04008957) and IG Index
> > >> Limited (a company registered in England and Wales, company number
> > >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > >> London EC4R 2YA. Both IG Markets Limited (register number 195355) and
> IG
> > >> Index Limited (register number 114059) are authorised and regulated by
> > the
> > >> Financial Conduct Authority.
> > >>
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>
> --
> Thanks,
> Neha
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Neha Narkhede <ne...@confluent.io>.
Apurva and Jason -- appreciate the detailed explanation, thank you!

Jason -

> In many applications, there is already a
natural identifier which is already used to divide the workload. For
example, in Kafka Streams and Kafka Connect, we have a taskId. For
applications where there is no natural ID, the user can generate a UUID and
persist it locally, which is as good as having the server generate it

I think for stateful applications that aren't Streams or Connect, it is
more work to leverage the strongest guarantees that this proposal offers.
But as you say, stateless applications, on the other hand, can still get a
lot of it by getting deduplication as well as transactional semantics for
the lifetime of the producer instance. I think this is fine since I think
that the vast majority of producer apps are stateless and will benefit from
the stronger guarantees. And for the minority of apps that need to
carefully use the application id can just fall back to using Streams
instead and expressing the send as a simple Streams topology, if needed.

I have a few comments on the naming --

1. I think we should consider renaming initTransactions to just init() and
moving the metadata initialization there. Let's make sure we don't add APIs
that are relevant to this proposal only. Instead, try to think what we'd
propose if we were writing the producer from scratch today. I suspect we
would end up with an init() API that would do the metadata initialization
as well as the transaction stuff lazily. If so, let's make that change now.
2. Along the same lines, let's think about the role of each id that the
producer will have and see if everything still makes sense. For instance,
we have quite a few per-producer-instance notions -- client.id, a producer
id and a transaction.app.id, some set via config and some generated
on-the-fly. What role does each play, how do they relate to each other and
is there an opportunity to get rid of any.
3. I think we should definitely consider renaming transaction.app.id to
something else. Given that we already have a notion of application.id and
it represents the entire Streams application, having transaction.app.id
that represents a producer instance is confusing. I do understand that, for
Streams, the user doesn't have to set transaction.app.id as it will likely
be application.id+taskId (am I understanding that correctly?)

On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wa...@gmail.com> wrote:

Onur,

I understand your question now. So it is indeed possible that after
commitTxn() returned the messages could still be lost permanently if all
replicas failed before the data was flushed to disk. This is the virtue of
Kafka's design to reply on replication (probably in memory) for high
availability, hence async flushing. This scenario already exist today and
KIP-98 did not intend to change this factor in any ways.

Guozhang


On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <on...@gmail.com>
wrote:

> In other words, we can see inconsistency when the transaction log reports
> the transaction as COMMITTED while the markers and data corresponding to
> the transaction itself on the user partitions may have been partially lost
> after-the-fact because of kafka's durability guarantees.
>
> On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> > markers (and a subset of the user's data produced in the transaction for
> > that matter) getting lost from the delta between the write and flush.
> >
> > KIP-98 relies on replicas writing to logs, so transaction durability is
> > effectively limited by kafka's definition of a "write success" meaning
> > written but not flushed to disk.
> >
> > I mentioned RF=1 not because of availability but actually to highlight a
> > corner-case durability scenario where the single replica participating
in
> > the transaction experiences a hard failure after the write but before
the
> > flush, causing the transaction to have partial data loss.
> >
> > Is this level of durability okay or do we want stronger guarantees for
> the
> > transaction? Basically what I'm wondering is if KIP-98 necessitates
> kafka'a
> > definition of a "write success" to be extended from "written" to an
> > optional "written and flushed to disk".
> >
> > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <Mi...@ig.com>
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >> ________________________________________
> >> From: Michael Pearce <Mi...@ig.com>
> >> Sent: Friday, December 9, 2016 7:52:25 PM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> >> Messaging
> >>
> >> Hi Jey
> >>
> >> 1) I agree, these should be used to add this in a future kip if ever
was
> >> enough of a case. As stated for us I think for these systems we will
> keep
> >> our JMS solutions there.  I think maybe in the docs when this feature
is
> >> written up, one should redirect users to alternative options such as
jms
> >> brokers, for these use cases.
> >>
> >> 2) I think this kip needs to be mindful and actually own to make sure
> >> things are implemented in a way to make future enchancement easy/or at
> >> least extensible. Having to in future rework things and correct
historic
> >> decisions is expensive as already finding.
> >>
> >> Sent using OWA for iPhone
> >> ________________________________________
> >> From: Jay Kreps <ja...@confluent.io>
> >> Sent: Friday, December 9, 2016 7:19:59 PM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> >> Messaging
> >>
> >> Hey Michael,
> >>
> >> Yeah, I don't think you need to go into the details of whatever you
guys
> >> have. I think several people in the thread said "let's do XA
> transactions
> >> too!" Obviously in a world where features were free and always worked
> >> perfectly we would! I've probably talked to about 100 people about
their
> >> use of XA transactions in different systems and my observation has been
> >> (a)
> >> they are a bit of an operational nightmare, (b) the use cases i've
> >> understood don't actually require full XA transactions they actually
> >> require a much weaker and easier to guarantee property. The result is
> you
> >> pay a big complexity cost for a guarantee much stronger than what you
> >> wanted. My sense is that this opinion is broadly shared by the
> distributed
> >> systems community at large and by Kafka folks in particular.
> >>
> >> I'm a contrarian so I think it is great not to be too swayed by "common
> >> wisdom" though. Five years ago there was a consensus that distributed
> >> transactions were too hard to implement in an operationally sound way,
> >> which i think was not correct, so the bad reputation for cross-system
> >> transactions may be equally wrong!
> >>
> >> To build a compelling case this is wrong I think two things need to be
> >> done:
> >>
> >>    1. Build a case that there are a large/important set of use cases
> that
> >>    cannot be solved with two independent transactions (as i described),
> >> and
> >>    that these use cases are things Kafka should be able to do.
> >>    2. Come up with the concrete extensions to the KIP-98 proposal that
> >>    would enable an operationally sound implementation for pluggable
> >>    multi-system XA.
> >>
> >> -Jay
> >>
> >>
> >>
> >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
> >> wrote:
> >>
> >> > Hi Jay,
> >> >
> >> > I can't go too deep into exact implantation due to no NDA. So
> apologies
> >> > here.
> >> >
> >> > Essentially we have multiple processes each owning selection of
> accounts
> >> > so on general flows an action for an account just needs to be managed
> >> local
> >> > to the owning node, happy days ever change is handled as a tick tock
> >> change.
> >> >
> >> > Unfortunately when a transfer occurs we need the two processes to
> >> > co-ordinate their transaction, we also need to ensure both don't
> >> continue
> >> > other actions/changesl, we do this using a data grid technology. This
> >> grid
> >> > technology supports transaction manager that we couple into currently
> >> our
> >> > jms provider which supports full XA transactions as such we can
manage
> >> the
> >> > production of the change messages out the system transactionally as
> >> well as
> >> > the in grid state.
> >> >
> >> > The obvious arguement here is should we even look to move this flow
> off
> >> > JMS then. We prob shouldn't nor will do this.
> >> >
> >> > The point is that I think saying Kafka supports transactions but then
> >> not
> >> > supporting it as per the traditional sense leads to developers
> expecting
> >> > similar behaviour and will cause issues in prod when they find it
> >> doesn't
> >> > work as they're used to.
> >> >
> >> > As my other response earlier, is there a better name to describe this
> >> > feature, if we're not implementing transactions to the traditional
> >> > transaction expected, to avoid this confusion?
> >> >
> >> >
> >> > Sent using OWA for iPhone
> >> > ________________________________________
> >> > From: Jay Kreps <ja...@confluent.io>
> >> > Sent: Friday, December 9, 2016 6:08:07 PM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
Transactional
> >> > Messaging
> >> >
> >> > Hey Michael,
> >> >
> >> > Doesn't that example have more to do with applying the update against
> >> two
> >> > rows in a single transaction? That is, clearly the write to Kafka
> needs
> >> to
> >> > be "transactional" and the write to the destination needs to be
> >> > transactional, but it's not clear to me that you need isolation that
> >> spans
> >> > both operations. Can you dive into the system architecture a bit more
> >> and
> >> > explain why Kafka needs to participate in the same transaction as the
> >> > destination system?
> >> >
> >> > -Jay
> >> >
> >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> Michael.Pearce@ig.com>
> >> > wrote:
> >> >
> >> > > Usecase in IG:
> >> > >
> >> > > Fund transfer between accounts. When we debit one account and fund
> >> > another
> >> > > we must ensure the records to both occur as an acid action, and as
a
> >> > single
> >> > > transaction.
> >> > >
> >> > > Today we achieve this because we have jms, as such we can do the
> >> actions
> >> > > needed in an xa transaction across both the accounts. To move this
> >> flow
> >> > to
> >> > > Kafka we would need support of XA transaction.
> >> > >
> >> > >
> >> > >
> >> > > Sent using OWA for iPhone
> >> > > ________________________________________
> >> > > From: Michael Pearce <Mi...@ig.com>
> >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> >> > > To: dev@kafka.apache.org
> >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> >> > > Messaging
> >> > >
> >> > > Hi Jay,
> >> > >
> >> > > For me having an XA transaction allows for ensuring ACID across my
> >> > > application.
> >> > >
> >> > > I believe it is part of the JMS api, and obviously JMS still is in
> >> > > enterprise very widely adopted for Messaging transport , so
> obviously
> >> to
> >> > > say it isn't widely used i think is ignoring a whole range of
users.
> >> Like
> >> > > wise I believe frameworks like spring etc fully support it more
> >> evidence
> >> > of
> >> > > its wide adoption.
> >> > >
> >> > > On this note personally we try to avoid transactions entirely in
our
> >> > flows
> >> > > for performance and simplicity. but we do alas unfortunately have
> one
> >> or
> >> > > two places we cannot ignore it.
> >> > >
> >> > > Cheers
> >> > > Mike
> >> > >
> >> > > Sent using OWA for iPhone
> >> > > ________________________________________
> >> > > From: Jay Kreps <ja...@confluent.io>
> >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> >> > > To: dev@kafka.apache.org
> >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> >> > > Messaging
> >> > >
> >> > > Hey Edoardo,
> >> > >
> >> > > For (3) can you outline what you think the benefit and use cases
> for a
> >> > more
> >> > > general cross-system XA feature would be an what changes to the
> >> proposal
> >> > > would be required to enable it? When I have asked people who wanted
> >> > > cross-system XA in the past what they wanted it for, I haven't
> really
> >> > > gotten any answers that made sense. Every person really wanted
> >> something
> >> > > that would be better solved by a transactional (or idempotent)
write
> >> to
> >> > > Kafka followed by an independent transactional (or idempotent)
> >> > consumption
> >> > > (which this proposal enables). For the use cases they described
> tying
> >> > these
> >> > > two things together had no advantage and many disadvantages.
> >> > >
> >> > > I have one use case which would be accomplished by cross-system XA
> >> which
> >> > is
> >> > > allowing the producer to block on the synchronous processing of the
> >> > message
> >> > > by (all? some?) consumers. However I'm not convinced that
> >> cross-system XA
> >> > > is the best solution to this problem, and I'm also not convinced
> this
> >> is
> >> > an
> >> > > important problem to solve. But maybe you have something in mind
> here.
> >> > >
> >> > > -Jay
> >> > >
> >> > >
> >> > >
> >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
> >> wrote:
> >> > >
> >> > > > Hi,
> >> > > > thanks, very interesting KIP ... I haven't fully digested it yet.
> >> > > >
> >> > > > We have many users who choose not to use the Java client,  so I
> have
> >> > > > concerns about the added complexity in developing the clients.
> >> > > > A few questions.
> >> > > >
> >> > > > 1 - is mixing transactional and non transactional messages on the
> >> *same
> >> > > > topic-partition* really a requirement ?
> >> > > > What use case does it satisfy?
> >> > > >
> >> > > > 2 - I guess some clients may only be interested to implement the
> >> > producer
> >> > > > idempotency.
> >> > > > It's not clear how they could be implemented without having to
add
> >> the
> >> > > > transaction capabilities.
> >> > > > As others on this list have said, I too would like to see
> >> idempotency
> >> > as
> >> > > a
> >> > > > more basic feature, on top which txns can be built.
> >> > > >
> >> > > > 3 - The KIP seems focused on a use case where consumption from a
> >> topic
> >> > > and
> >> > > > subsequent production are part of the producer transaction.
> >> > > >
> >> > > > It'd be great to see a way to extend the producer transaction to
> >> > include
> >> > > > additional transactional resources,
> >> > > > so that the consumption from another topic just becomes a special
> >> case
> >> > of
> >> > > > a more general "distributed" txn.
> >> > > >
> >> > > > Edo
> >> > > > --------------------------------------------------
> >> > > > Edoardo Comar
> >> > > > IBM MessageHub
> >> > > > ecomar@uk.ibm.com
> >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> >> > > >
> >> > > > IBM United Kingdom Limited Registered in England and Wales with
> >> number
> >> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
> >> Hants.
> >> > > PO6
> >> > > > 3AU
> >> > > >
> >> > > >
> >> > > >
> >> > > > From:   Guozhang Wang <wa...@gmail.com>
> >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> >> > > > Date:   30/11/2016 22:20
> >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > Transactional
> >> > > > Messaging
> >> > > >
> >> > > >
> >> > > >
> >> > > > Hi all,
> >> > > >
> >> > > > I have just created KIP-98 to enhance Kafka with exactly once
> >> delivery
> >> > > > semantics:
> >> > > >
> >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > <
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > >*
> >> > > >
> >> > > > This KIP adds a transactional messaging mechanism along with an
> >> > > idempotent
> >> > > > producer implementation to make sure that 1) duplicated messages
> >> sent
> >> > > from
> >> > > > the same identified producer can be detected on the broker side,
> and
> >> > 2) a
> >> > > > group of messages sent within a transaction will atomically be
> >> either
> >> > > > reflected and fetchable to consumers or not as a whole.
> >> > > >
> >> > > > The above wiki page provides a high-level view of the proposed
> >> changes
> >> > as
> >> > > > well as summarized guarantees. Initial draft of the detailed
> >> > > > implementation
> >> > > > design is described in this Google doc:
> >> > > >
> >> > > > https://docs.google.com/document/d/11Jqy_
> >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> >> > > > 0wSw9ra8
> >> > > >
> >> > > >
> >> > > > We would love to hear your comments and suggestions.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > -- Guozhang
> >> > > >
> >> > > >
> >> > > >
> >> > > > Unless stated otherwise above:
> >> > > > IBM United Kingdom Limited - Registered in England and Wales with
> >> > number
> >> > > > 741598.
> >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
Hampshire
> >> PO6
> >> > > 3AU
> >> > > >
> >> > > The information contained in this email is strictly confidential
and
> >> for
> >> > > the use of the addressee only, unless otherwise indicated. If you
> are
> >> not
> >> > > the intended recipient, please do not read, copy, use or disclose
to
> >> > others
> >> > > this message or any attachment. Please also notify the sender by
> >> replying
> >> > > to this email or by telephone (+44(020 7896 0011) and then delete
> the
> >> > email
> >> > > and any copies of it. Opinions, conclusion (etc) that do not relate
> to
> >> > the
> >> > > official business of this company shall be understood as neither
> given
> >> > nor
> >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> >> > > registered in England and Wales, company number 04008957) and IG
> Index
> >> > > Limited (a company registered in England and Wales, company number
> >> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
> >> > > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> and
> >> IG
> >> > > Index Limited (register number 114059) are authorised and regulated
> by
> >> > the
> >> > > Financial Conduct Authority.
> >> > >
> >> > The information contained in this email is strictly confidential and
> for
> >> > the use of the addressee only, unless otherwise indicated. If you are
> >> not
> >> > the intended recipient, please do not read, copy, use or disclose to
> >> others
> >> > this message or any attachment. Please also notify the sender by
> >> replying
> >> > to this email or by telephone (+44(020 7896 0011) and then delete the
> >> email
> >> > and any copies of it. Opinions, conclusion (etc) that do not relate
to
> >> the
> >> > official business of this company shall be understood as neither
given
> >> nor
> >> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> >> > registered in England and Wales, company number 04008957) and IG
Index
> >> > Limited (a company registered in England and Wales, company number
> >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
Hill,
> >> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and
> IG
> >> > Index Limited (register number 114059) are authorised and regulated
by
> >> the
> >> > Financial Conduct Authority.
> >> >
> >> The information contained in this email is strictly confidential and
for
> >> the use of the addressee only, unless otherwise indicated. If you are
> not
> >> the intended recipient, please do not read, copy, use or disclose to
> others
> >> this message or any attachment. Please also notify the sender by
> replying
> >> to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> >> and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> >> official business of this company shall be understood as neither given
> nor
> >> endorsed by it. IG is a trading name of IG Markets Limited (a company
> >> registered in England and Wales, company number 04008957) and IG Index
> >> Limited (a company registered in England and Wales, company number
> >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> >> London EC4R 2YA. Both IG Markets Limited (register number 195355) and
IG
> >> Index Limited (register number 114059) are authorised and regulated by
> the
> >> Financial Conduct Authority.
> >>
> >
> >
>



--
-- Guozhang

-- 
Thanks,
Neha

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Onur,

I understand your question now. So it is indeed possible that after
commitTxn() returned the messages could still be lost permanently if all
replicas failed before the data was flushed to disk. This is the virtue of
Kafka's design to reply on replication (probably in memory) for high
availability, hence async flushing. This scenario already exist today and
KIP-98 did not intend to change this factor in any ways.

Guozhang


On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <on...@gmail.com>
wrote:

> In other words, we can see inconsistency when the transaction log reports
> the transaction as COMMITTED while the markers and data corresponding to
> the transaction itself on the user partitions may have been partially lost
> after-the-fact because of kafka's durability guarantees.
>
> On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> onurkaraman.apache@gmail.com>
> wrote:
>
> > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> > markers (and a subset of the user's data produced in the transaction for
> > that matter) getting lost from the delta between the write and flush.
> >
> > KIP-98 relies on replicas writing to logs, so transaction durability is
> > effectively limited by kafka's definition of a "write success" meaning
> > written but not flushed to disk.
> >
> > I mentioned RF=1 not because of availability but actually to highlight a
> > corner-case durability scenario where the single replica participating in
> > the transaction experiences a hard failure after the write but before the
> > flush, causing the transaction to have partial data loss.
> >
> > Is this level of durability okay or do we want stronger guarantees for
> the
> > transaction? Basically what I'm wondering is if KIP-98 necessitates
> kafka'a
> > definition of a "write success" to be extended from "written" to an
> > optional "written and flushed to disk".
> >
> > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <Mi...@ig.com>
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >> ________________________________________
> >> From: Michael Pearce <Mi...@ig.com>
> >> Sent: Friday, December 9, 2016 7:52:25 PM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> >> Messaging
> >>
> >> Hi Jey
> >>
> >> 1) I agree, these should be used to add this in a future kip if ever was
> >> enough of a case. As stated for us I think for these systems we will
> keep
> >> our JMS solutions there.  I think maybe in the docs when this feature is
> >> written up, one should redirect users to alternative options such as jms
> >> brokers, for these use cases.
> >>
> >> 2) I think this kip needs to be mindful and actually own to make sure
> >> things are implemented in a way to make future enchancement easy/or at
> >> least extensible. Having to in future rework things and correct historic
> >> decisions is expensive as already finding.
> >>
> >> Sent using OWA for iPhone
> >> ________________________________________
> >> From: Jay Kreps <ja...@confluent.io>
> >> Sent: Friday, December 9, 2016 7:19:59 PM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> >> Messaging
> >>
> >> Hey Michael,
> >>
> >> Yeah, I don't think you need to go into the details of whatever you guys
> >> have. I think several people in the thread said "let's do XA
> transactions
> >> too!" Obviously in a world where features were free and always worked
> >> perfectly we would! I've probably talked to about 100 people about their
> >> use of XA transactions in different systems and my observation has been
> >> (a)
> >> they are a bit of an operational nightmare, (b) the use cases i've
> >> understood don't actually require full XA transactions they actually
> >> require a much weaker and easier to guarantee property. The result is
> you
> >> pay a big complexity cost for a guarantee much stronger than what you
> >> wanted. My sense is that this opinion is broadly shared by the
> distributed
> >> systems community at large and by Kafka folks in particular.
> >>
> >> I'm a contrarian so I think it is great not to be too swayed by "common
> >> wisdom" though. Five years ago there was a consensus that distributed
> >> transactions were too hard to implement in an operationally sound way,
> >> which i think was not correct, so the bad reputation for cross-system
> >> transactions may be equally wrong!
> >>
> >> To build a compelling case this is wrong I think two things need to be
> >> done:
> >>
> >>    1. Build a case that there are a large/important set of use cases
> that
> >>    cannot be solved with two independent transactions (as i described),
> >> and
> >>    that these use cases are things Kafka should be able to do.
> >>    2. Come up with the concrete extensions to the KIP-98 proposal that
> >>    would enable an operationally sound implementation for pluggable
> >>    multi-system XA.
> >>
> >> -Jay
> >>
> >>
> >>
> >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
> >> wrote:
> >>
> >> > Hi Jay,
> >> >
> >> > I can't go too deep into exact implantation due to no NDA. So
> apologies
> >> > here.
> >> >
> >> > Essentially we have multiple processes each owning selection of
> accounts
> >> > so on general flows an action for an account just needs to be managed
> >> local
> >> > to the owning node, happy days ever change is handled as a tick tock
> >> change.
> >> >
> >> > Unfortunately when a transfer occurs we need the two processes to
> >> > co-ordinate their transaction, we also need to ensure both don't
> >> continue
> >> > other actions/changesl, we do this using a data grid technology. This
> >> grid
> >> > technology supports transaction manager that we couple into currently
> >> our
> >> > jms provider which supports full XA transactions as such we can manage
> >> the
> >> > production of the change messages out the system transactionally as
> >> well as
> >> > the in grid state.
> >> >
> >> > The obvious arguement here is should we even look to move this flow
> off
> >> > JMS then. We prob shouldn't nor will do this.
> >> >
> >> > The point is that I think saying Kafka supports transactions but then
> >> not
> >> > supporting it as per the traditional sense leads to developers
> expecting
> >> > similar behaviour and will cause issues in prod when they find it
> >> doesn't
> >> > work as they're used to.
> >> >
> >> > As my other response earlier, is there a better name to describe this
> >> > feature, if we're not implementing transactions to the traditional
> >> > transaction expected, to avoid this confusion?
> >> >
> >> >
> >> > Sent using OWA for iPhone
> >> > ________________________________________
> >> > From: Jay Kreps <ja...@confluent.io>
> >> > Sent: Friday, December 9, 2016 6:08:07 PM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> >> > Messaging
> >> >
> >> > Hey Michael,
> >> >
> >> > Doesn't that example have more to do with applying the update against
> >> two
> >> > rows in a single transaction? That is, clearly the write to Kafka
> needs
> >> to
> >> > be "transactional" and the write to the destination needs to be
> >> > transactional, but it's not clear to me that you need isolation that
> >> spans
> >> > both operations. Can you dive into the system architecture a bit more
> >> and
> >> > explain why Kafka needs to participate in the same transaction as the
> >> > destination system?
> >> >
> >> > -Jay
> >> >
> >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> Michael.Pearce@ig.com>
> >> > wrote:
> >> >
> >> > > Usecase in IG:
> >> > >
> >> > > Fund transfer between accounts. When we debit one account and fund
> >> > another
> >> > > we must ensure the records to both occur as an acid action, and as a
> >> > single
> >> > > transaction.
> >> > >
> >> > > Today we achieve this because we have jms, as such we can do the
> >> actions
> >> > > needed in an xa transaction across both the accounts. To move this
> >> flow
> >> > to
> >> > > Kafka we would need support of XA transaction.
> >> > >
> >> > >
> >> > >
> >> > > Sent using OWA for iPhone
> >> > > ________________________________________
> >> > > From: Michael Pearce <Mi...@ig.com>
> >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> >> > > To: dev@kafka.apache.org
> >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> >> > > Messaging
> >> > >
> >> > > Hi Jay,
> >> > >
> >> > > For me having an XA transaction allows for ensuring ACID across my
> >> > > application.
> >> > >
> >> > > I believe it is part of the JMS api, and obviously JMS still is in
> >> > > enterprise very widely adopted for Messaging transport , so
> obviously
> >> to
> >> > > say it isn't widely used i think is ignoring a whole range of users.
> >> Like
> >> > > wise I believe frameworks like spring etc fully support it more
> >> evidence
> >> > of
> >> > > its wide adoption.
> >> > >
> >> > > On this note personally we try to avoid transactions entirely in our
> >> > flows
> >> > > for performance and simplicity. but we do alas unfortunately have
> one
> >> or
> >> > > two places we cannot ignore it.
> >> > >
> >> > > Cheers
> >> > > Mike
> >> > >
> >> > > Sent using OWA for iPhone
> >> > > ________________________________________
> >> > > From: Jay Kreps <ja...@confluent.io>
> >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> >> > > To: dev@kafka.apache.org
> >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> >> > > Messaging
> >> > >
> >> > > Hey Edoardo,
> >> > >
> >> > > For (3) can you outline what you think the benefit and use cases
> for a
> >> > more
> >> > > general cross-system XA feature would be an what changes to the
> >> proposal
> >> > > would be required to enable it? When I have asked people who wanted
> >> > > cross-system XA in the past what they wanted it for, I haven't
> really
> >> > > gotten any answers that made sense. Every person really wanted
> >> something
> >> > > that would be better solved by a transactional (or idempotent) write
> >> to
> >> > > Kafka followed by an independent transactional (or idempotent)
> >> > consumption
> >> > > (which this proposal enables). For the use cases they described
> tying
> >> > these
> >> > > two things together had no advantage and many disadvantages.
> >> > >
> >> > > I have one use case which would be accomplished by cross-system XA
> >> which
> >> > is
> >> > > allowing the producer to block on the synchronous processing of the
> >> > message
> >> > > by (all? some?) consumers. However I'm not convinced that
> >> cross-system XA
> >> > > is the best solution to this problem, and I'm also not convinced
> this
> >> is
> >> > an
> >> > > important problem to solve. But maybe you have something in mind
> here.
> >> > >
> >> > > -Jay
> >> > >
> >> > >
> >> > >
> >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
> >> wrote:
> >> > >
> >> > > > Hi,
> >> > > > thanks, very interesting KIP ... I haven't fully digested it yet.
> >> > > >
> >> > > > We have many users who choose not to use the Java client,  so I
> have
> >> > > > concerns about the added complexity in developing the clients.
> >> > > > A few questions.
> >> > > >
> >> > > > 1 - is mixing transactional and non transactional messages on the
> >> *same
> >> > > > topic-partition* really a requirement ?
> >> > > > What use case does it satisfy?
> >> > > >
> >> > > > 2 - I guess some clients may only be interested to implement the
> >> > producer
> >> > > > idempotency.
> >> > > > It's not clear how they could be implemented without having to add
> >> the
> >> > > > transaction capabilities.
> >> > > > As others on this list have said, I too would like to see
> >> idempotency
> >> > as
> >> > > a
> >> > > > more basic feature, on top which txns can be built.
> >> > > >
> >> > > > 3 - The KIP seems focused on a use case where consumption from a
> >> topic
> >> > > and
> >> > > > subsequent production are part of the producer transaction.
> >> > > >
> >> > > > It'd be great to see a way to extend the producer transaction to
> >> > include
> >> > > > additional transactional resources,
> >> > > > so that the consumption from another topic just becomes a special
> >> case
> >> > of
> >> > > > a more general "distributed" txn.
> >> > > >
> >> > > > Edo
> >> > > > --------------------------------------------------
> >> > > > Edoardo Comar
> >> > > > IBM MessageHub
> >> > > > ecomar@uk.ibm.com
> >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> >> > > >
> >> > > > IBM United Kingdom Limited Registered in England and Wales with
> >> number
> >> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
> >> Hants.
> >> > > PO6
> >> > > > 3AU
> >> > > >
> >> > > >
> >> > > >
> >> > > > From:   Guozhang Wang <wa...@gmail.com>
> >> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> >> > > > Date:   30/11/2016 22:20
> >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> >> > Transactional
> >> > > > Messaging
> >> > > >
> >> > > >
> >> > > >
> >> > > > Hi all,
> >> > > >
> >> > > > I have just created KIP-98 to enhance Kafka with exactly once
> >> delivery
> >> > > > semantics:
> >> > > >
> >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > <
> >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > >*
> >> > > >
> >> > > > This KIP adds a transactional messaging mechanism along with an
> >> > > idempotent
> >> > > > producer implementation to make sure that 1) duplicated messages
> >> sent
> >> > > from
> >> > > > the same identified producer can be detected on the broker side,
> and
> >> > 2) a
> >> > > > group of messages sent within a transaction will atomically be
> >> either
> >> > > > reflected and fetchable to consumers or not as a whole.
> >> > > >
> >> > > > The above wiki page provides a high-level view of the proposed
> >> changes
> >> > as
> >> > > > well as summarized guarantees. Initial draft of the detailed
> >> > > > implementation
> >> > > > design is described in this Google doc:
> >> > > >
> >> > > > https://docs.google.com/document/d/11Jqy_
> >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> >> > > > 0wSw9ra8
> >> > > >
> >> > > >
> >> > > > We would love to hear your comments and suggestions.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > -- Guozhang
> >> > > >
> >> > > >
> >> > > >
> >> > > > Unless stated otherwise above:
> >> > > > IBM United Kingdom Limited - Registered in England and Wales with
> >> > number
> >> > > > 741598.
> >> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
> >> PO6
> >> > > 3AU
> >> > > >
> >> > > The information contained in this email is strictly confidential and
> >> for
> >> > > the use of the addressee only, unless otherwise indicated. If you
> are
> >> not
> >> > > the intended recipient, please do not read, copy, use or disclose to
> >> > others
> >> > > this message or any attachment. Please also notify the sender by
> >> replying
> >> > > to this email or by telephone (+44(020 7896 0011) and then delete
> the
> >> > email
> >> > > and any copies of it. Opinions, conclusion (etc) that do not relate
> to
> >> > the
> >> > > official business of this company shall be understood as neither
> given
> >> > nor
> >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> >> > > registered in England and Wales, company number 04008957) and IG
> Index
> >> > > Limited (a company registered in England and Wales, company number
> >> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
> >> > > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> and
> >> IG
> >> > > Index Limited (register number 114059) are authorised and regulated
> by
> >> > the
> >> > > Financial Conduct Authority.
> >> > >
> >> > The information contained in this email is strictly confidential and
> for
> >> > the use of the addressee only, unless otherwise indicated. If you are
> >> not
> >> > the intended recipient, please do not read, copy, use or disclose to
> >> others
> >> > this message or any attachment. Please also notify the sender by
> >> replying
> >> > to this email or by telephone (+44(020 7896 0011) and then delete the
> >> email
> >> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> >> the
> >> > official business of this company shall be understood as neither given
> >> nor
> >> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> >> > registered in England and Wales, company number 04008957) and IG Index
> >> > Limited (a company registered in England and Wales, company number
> >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> >> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and
> IG
> >> > Index Limited (register number 114059) are authorised and regulated by
> >> the
> >> > Financial Conduct Authority.
> >> >
> >> The information contained in this email is strictly confidential and for
> >> the use of the addressee only, unless otherwise indicated. If you are
> not
> >> the intended recipient, please do not read, copy, use or disclose to
> others
> >> this message or any attachment. Please also notify the sender by
> replying
> >> to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> >> and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> >> official business of this company shall be understood as neither given
> nor
> >> endorsed by it. IG is a trading name of IG Markets Limited (a company
> >> registered in England and Wales, company number 04008957) and IG Index
> >> Limited (a company registered in England and Wales, company number
> >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> >> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> >> Index Limited (register number 114059) are authorised and regulated by
> the
> >> Financial Conduct Authority.
> >>
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Onur Karaman <on...@gmail.com>.
In other words, we can see inconsistency when the transaction log reports
the transaction as COMMITTED while the markers and data corresponding to
the transaction itself on the user partitions may have been partially lost
after-the-fact because of kafka's durability guarantees.

On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <on...@gmail.com>
wrote:

> @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> markers (and a subset of the user's data produced in the transaction for
> that matter) getting lost from the delta between the write and flush.
>
> KIP-98 relies on replicas writing to logs, so transaction durability is
> effectively limited by kafka's definition of a "write success" meaning
> written but not flushed to disk.
>
> I mentioned RF=1 not because of availability but actually to highlight a
> corner-case durability scenario where the single replica participating in
> the transaction experiences a hard failure after the write but before the
> flush, causing the transaction to have partial data loss.
>
> Is this level of durability okay or do we want stronger guarantees for the
> transaction? Basically what I'm wondering is if KIP-98 necessitates kafka'a
> definition of a "write success" to be extended from "written" to an
> optional "written and flushed to disk".
>
> On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <Mi...@ig.com>
> wrote:
>
>> Apologies on the spelling.
>>
>> *Hi Jay,
>> ________________________________________
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: Friday, December 9, 2016 7:52:25 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> Messaging
>>
>> Hi Jey
>>
>> 1) I agree, these should be used to add this in a future kip if ever was
>> enough of a case. As stated for us I think for these systems we will keep
>> our JMS solutions there.  I think maybe in the docs when this feature is
>> written up, one should redirect users to alternative options such as jms
>> brokers, for these use cases.
>>
>> 2) I think this kip needs to be mindful and actually own to make sure
>> things are implemented in a way to make future enchancement easy/or at
>> least extensible. Having to in future rework things and correct historic
>> decisions is expensive as already finding.
>>
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Jay Kreps <ja...@confluent.io>
>> Sent: Friday, December 9, 2016 7:19:59 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> Messaging
>>
>> Hey Michael,
>>
>> Yeah, I don't think you need to go into the details of whatever you guys
>> have. I think several people in the thread said "let's do XA transactions
>> too!" Obviously in a world where features were free and always worked
>> perfectly we would! I've probably talked to about 100 people about their
>> use of XA transactions in different systems and my observation has been
>> (a)
>> they are a bit of an operational nightmare, (b) the use cases i've
>> understood don't actually require full XA transactions they actually
>> require a much weaker and easier to guarantee property. The result is you
>> pay a big complexity cost for a guarantee much stronger than what you
>> wanted. My sense is that this opinion is broadly shared by the distributed
>> systems community at large and by Kafka folks in particular.
>>
>> I'm a contrarian so I think it is great not to be too swayed by "common
>> wisdom" though. Five years ago there was a consensus that distributed
>> transactions were too hard to implement in an operationally sound way,
>> which i think was not correct, so the bad reputation for cross-system
>> transactions may be equally wrong!
>>
>> To build a compelling case this is wrong I think two things need to be
>> done:
>>
>>    1. Build a case that there are a large/important set of use cases that
>>    cannot be solved with two independent transactions (as i described),
>> and
>>    that these use cases are things Kafka should be able to do.
>>    2. Come up with the concrete extensions to the KIP-98 proposal that
>>    would enable an operationally sound implementation for pluggable
>>    multi-system XA.
>>
>> -Jay
>>
>>
>>
>> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
>> wrote:
>>
>> > Hi Jay,
>> >
>> > I can't go too deep into exact implantation due to no NDA. So apologies
>> > here.
>> >
>> > Essentially we have multiple processes each owning selection of accounts
>> > so on general flows an action for an account just needs to be managed
>> local
>> > to the owning node, happy days ever change is handled as a tick tock
>> change.
>> >
>> > Unfortunately when a transfer occurs we need the two processes to
>> > co-ordinate their transaction, we also need to ensure both don't
>> continue
>> > other actions/changesl, we do this using a data grid technology. This
>> grid
>> > technology supports transaction manager that we couple into currently
>> our
>> > jms provider which supports full XA transactions as such we can manage
>> the
>> > production of the change messages out the system transactionally as
>> well as
>> > the in grid state.
>> >
>> > The obvious arguement here is should we even look to move this flow off
>> > JMS then. We prob shouldn't nor will do this.
>> >
>> > The point is that I think saying Kafka supports transactions but then
>> not
>> > supporting it as per the traditional sense leads to developers expecting
>> > similar behaviour and will cause issues in prod when they find it
>> doesn't
>> > work as they're used to.
>> >
>> > As my other response earlier, is there a better name to describe this
>> > feature, if we're not implementing transactions to the traditional
>> > transaction expected, to avoid this confusion?
>> >
>> >
>> > Sent using OWA for iPhone
>> > ________________________________________
>> > From: Jay Kreps <ja...@confluent.io>
>> > Sent: Friday, December 9, 2016 6:08:07 PM
>> > To: dev@kafka.apache.org
>> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> > Messaging
>> >
>> > Hey Michael,
>> >
>> > Doesn't that example have more to do with applying the update against
>> two
>> > rows in a single transaction? That is, clearly the write to Kafka needs
>> to
>> > be "transactional" and the write to the destination needs to be
>> > transactional, but it's not clear to me that you need isolation that
>> spans
>> > both operations. Can you dive into the system architecture a bit more
>> and
>> > explain why Kafka needs to participate in the same transaction as the
>> > destination system?
>> >
>> > -Jay
>> >
>> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
>> > wrote:
>> >
>> > > Usecase in IG:
>> > >
>> > > Fund transfer between accounts. When we debit one account and fund
>> > another
>> > > we must ensure the records to both occur as an acid action, and as a
>> > single
>> > > transaction.
>> > >
>> > > Today we achieve this because we have jms, as such we can do the
>> actions
>> > > needed in an xa transaction across both the accounts. To move this
>> flow
>> > to
>> > > Kafka we would need support of XA transaction.
>> > >
>> > >
>> > >
>> > > Sent using OWA for iPhone
>> > > ________________________________________
>> > > From: Michael Pearce <Mi...@ig.com>
>> > > Sent: Friday, December 9, 2016 6:09:06 AM
>> > > To: dev@kafka.apache.org
>> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> > > Messaging
>> > >
>> > > Hi Jay,
>> > >
>> > > For me having an XA transaction allows for ensuring ACID across my
>> > > application.
>> > >
>> > > I believe it is part of the JMS api, and obviously JMS still is in
>> > > enterprise very widely adopted for Messaging transport , so obviously
>> to
>> > > say it isn't widely used i think is ignoring a whole range of users.
>> Like
>> > > wise I believe frameworks like spring etc fully support it more
>> evidence
>> > of
>> > > its wide adoption.
>> > >
>> > > On this note personally we try to avoid transactions entirely in our
>> > flows
>> > > for performance and simplicity. but we do alas unfortunately have one
>> or
>> > > two places we cannot ignore it.
>> > >
>> > > Cheers
>> > > Mike
>> > >
>> > > Sent using OWA for iPhone
>> > > ________________________________________
>> > > From: Jay Kreps <ja...@confluent.io>
>> > > Sent: Thursday, December 8, 2016 11:25:53 PM
>> > > To: dev@kafka.apache.org
>> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> > > Messaging
>> > >
>> > > Hey Edoardo,
>> > >
>> > > For (3) can you outline what you think the benefit and use cases for a
>> > more
>> > > general cross-system XA feature would be an what changes to the
>> proposal
>> > > would be required to enable it? When I have asked people who wanted
>> > > cross-system XA in the past what they wanted it for, I haven't really
>> > > gotten any answers that made sense. Every person really wanted
>> something
>> > > that would be better solved by a transactional (or idempotent) write
>> to
>> > > Kafka followed by an independent transactional (or idempotent)
>> > consumption
>> > > (which this proposal enables). For the use cases they described tying
>> > these
>> > > two things together had no advantage and many disadvantages.
>> > >
>> > > I have one use case which would be accomplished by cross-system XA
>> which
>> > is
>> > > allowing the producer to block on the synchronous processing of the
>> > message
>> > > by (all? some?) consumers. However I'm not convinced that
>> cross-system XA
>> > > is the best solution to this problem, and I'm also not convinced this
>> is
>> > an
>> > > important problem to solve. But maybe you have something in mind here.
>> > >
>> > > -Jay
>> > >
>> > >
>> > >
>> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
>> wrote:
>> > >
>> > > > Hi,
>> > > > thanks, very interesting KIP ... I haven't fully digested it yet.
>> > > >
>> > > > We have many users who choose not to use the Java client,  so I have
>> > > > concerns about the added complexity in developing the clients.
>> > > > A few questions.
>> > > >
>> > > > 1 - is mixing transactional and non transactional messages on the
>> *same
>> > > > topic-partition* really a requirement ?
>> > > > What use case does it satisfy?
>> > > >
>> > > > 2 - I guess some clients may only be interested to implement the
>> > producer
>> > > > idempotency.
>> > > > It's not clear how they could be implemented without having to add
>> the
>> > > > transaction capabilities.
>> > > > As others on this list have said, I too would like to see
>> idempotency
>> > as
>> > > a
>> > > > more basic feature, on top which txns can be built.
>> > > >
>> > > > 3 - The KIP seems focused on a use case where consumption from a
>> topic
>> > > and
>> > > > subsequent production are part of the producer transaction.
>> > > >
>> > > > It'd be great to see a way to extend the producer transaction to
>> > include
>> > > > additional transactional resources,
>> > > > so that the consumption from another topic just becomes a special
>> case
>> > of
>> > > > a more general "distributed" txn.
>> > > >
>> > > > Edo
>> > > > --------------------------------------------------
>> > > > Edoardo Comar
>> > > > IBM MessageHub
>> > > > ecomar@uk.ibm.com
>> > > > IBM UK Ltd, Hursley Park, SO21 2JN
>> > > >
>> > > > IBM United Kingdom Limited Registered in England and Wales with
>> number
>> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
>> Hants.
>> > > PO6
>> > > > 3AU
>> > > >
>> > > >
>> > > >
>> > > > From:   Guozhang Wang <wa...@gmail.com>
>> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
>> > > > Date:   30/11/2016 22:20
>> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
>> > Transactional
>> > > > Messaging
>> > > >
>> > > >
>> > > >
>> > > > Hi all,
>> > > >
>> > > > I have just created KIP-98 to enhance Kafka with exactly once
>> delivery
>> > > > semantics:
>> > > >
>> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > > <
>> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > > >*
>> > > >
>> > > > This KIP adds a transactional messaging mechanism along with an
>> > > idempotent
>> > > > producer implementation to make sure that 1) duplicated messages
>> sent
>> > > from
>> > > > the same identified producer can be detected on the broker side, and
>> > 2) a
>> > > > group of messages sent within a transaction will atomically be
>> either
>> > > > reflected and fetchable to consumers or not as a whole.
>> > > >
>> > > > The above wiki page provides a high-level view of the proposed
>> changes
>> > as
>> > > > well as summarized guarantees. Initial draft of the detailed
>> > > > implementation
>> > > > design is described in this Google doc:
>> > > >
>> > > > https://docs.google.com/document/d/11Jqy_
>> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> > > > 0wSw9ra8
>> > > >
>> > > >
>> > > > We would love to hear your comments and suggestions.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > -- Guozhang
>> > > >
>> > > >
>> > > >
>> > > > Unless stated otherwise above:
>> > > > IBM United Kingdom Limited - Registered in England and Wales with
>> > number
>> > > > 741598.
>> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
>> PO6
>> > > 3AU
>> > > >
>> > > The information contained in this email is strictly confidential and
>> for
>> > > the use of the addressee only, unless otherwise indicated. If you are
>> not
>> > > the intended recipient, please do not read, copy, use or disclose to
>> > others
>> > > this message or any attachment. Please also notify the sender by
>> replying
>> > > to this email or by telephone (+44(020 7896 0011) and then delete the
>> > email
>> > > and any copies of it. Opinions, conclusion (etc) that do not relate to
>> > the
>> > > official business of this company shall be understood as neither given
>> > nor
>> > > endorsed by it. IG is a trading name of IG Markets Limited (a company
>> > > registered in England and Wales, company number 04008957) and IG Index
>> > > Limited (a company registered in England and Wales, company number
>> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
>> > > London EC4R 2YA. Both IG Markets Limited (register number 195355) and
>> IG
>> > > Index Limited (register number 114059) are authorised and regulated by
>> > the
>> > > Financial Conduct Authority.
>> > >
>> > The information contained in this email is strictly confidential and for
>> > the use of the addressee only, unless otherwise indicated. If you are
>> not
>> > the intended recipient, please do not read, copy, use or disclose to
>> others
>> > this message or any attachment. Please also notify the sender by
>> replying
>> > to this email or by telephone (+44(020 7896 0011) and then delete the
>> email
>> > and any copies of it. Opinions, conclusion (etc) that do not relate to
>> the
>> > official business of this company shall be understood as neither given
>> nor
>> > endorsed by it. IG is a trading name of IG Markets Limited (a company
>> > registered in England and Wales, company number 04008957) and IG Index
>> > Limited (a company registered in England and Wales, company number
>> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
>> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
>> > Index Limited (register number 114059) are authorised and regulated by
>> the
>> > Financial Conduct Authority.
>> >
>> The information contained in this email is strictly confidential and for
>> the use of the addressee only, unless otherwise indicated. If you are not
>> the intended recipient, please do not read, copy, use or disclose to others
>> this message or any attachment. Please also notify the sender by replying
>> to this email or by telephone (+44(020 7896 0011) and then delete the email
>> and any copies of it. Opinions, conclusion (etc) that do not relate to the
>> official business of this company shall be understood as neither given nor
>> endorsed by it. IG is a trading name of IG Markets Limited (a company
>> registered in England and Wales, company number 04008957) and IG Index
>> Limited (a company registered in England and Wales, company number
>> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
>> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
>> Index Limited (register number 114059) are authorised and regulated by the
>> Financial Conduct Authority.
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Onur Karaman <on...@gmail.com>.
@Guozhang no I actually meant durability concerns over COMMIT/ABORT markers
(and a subset of the user's data produced in the transaction for that
matter) getting lost from the delta between the write and flush.

KIP-98 relies on replicas writing to logs, so transaction durability is
effectively limited by kafka's definition of a "write success" meaning
written but not flushed to disk.

I mentioned RF=1 not because of availability but actually to highlight a
corner-case durability scenario where the single replica participating in
the transaction experiences a hard failure after the write but before the
flush, causing the transaction to have partial data loss.

Is this level of durability okay or do we want stronger guarantees for the
transaction? Basically what I'm wondering is if KIP-98 necessitates kafka'a
definition of a "write success" to be extended from "written" to an
optional "written and flushed to disk".

On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Apologies on the spelling.
>
> *Hi Jay,
> ________________________________________
> From: Michael Pearce <Mi...@ig.com>
> Sent: Friday, December 9, 2016 7:52:25 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hi Jey
>
> 1) I agree, these should be used to add this in a future kip if ever was
> enough of a case. As stated for us I think for these systems we will keep
> our JMS solutions there.  I think maybe in the docs when this feature is
> written up, one should redirect users to alternative options such as jms
> brokers, for these use cases.
>
> 2) I think this kip needs to be mindful and actually own to make sure
> things are implemented in a way to make future enchancement easy/or at
> least extensible. Having to in future rework things and correct historic
> decisions is expensive as already finding.
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 7:19:59 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Yeah, I don't think you need to go into the details of whatever you guys
> have. I think several people in the thread said "let's do XA transactions
> too!" Obviously in a world where features were free and always worked
> perfectly we would! I've probably talked to about 100 people about their
> use of XA transactions in different systems and my observation has been (a)
> they are a bit of an operational nightmare, (b) the use cases i've
> understood don't actually require full XA transactions they actually
> require a much weaker and easier to guarantee property. The result is you
> pay a big complexity cost for a guarantee much stronger than what you
> wanted. My sense is that this opinion is broadly shared by the distributed
> systems community at large and by Kafka folks in particular.
>
> I'm a contrarian so I think it is great not to be too swayed by "common
> wisdom" though. Five years ago there was a consensus that distributed
> transactions were too hard to implement in an operationally sound way,
> which i think was not correct, so the bad reputation for cross-system
> transactions may be equally wrong!
>
> To build a compelling case this is wrong I think two things need to be
> done:
>
>    1. Build a case that there are a large/important set of use cases that
>    cannot be solved with two independent transactions (as i described), and
>    that these use cases are things Kafka should be able to do.
>    2. Come up with the concrete extensions to the KIP-98 proposal that
>    would enable an operationally sound implementation for pluggable
>    multi-system XA.
>
> -Jay
>
>
>
> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Hi Jay,
> >
> > I can't go too deep into exact implantation due to no NDA. So apologies
> > here.
> >
> > Essentially we have multiple processes each owning selection of accounts
> > so on general flows an action for an account just needs to be managed
> local
> > to the owning node, happy days ever change is handled as a tick tock
> change.
> >
> > Unfortunately when a transfer occurs we need the two processes to
> > co-ordinate their transaction, we also need to ensure both don't continue
> > other actions/changesl, we do this using a data grid technology. This
> grid
> > technology supports transaction manager that we couple into currently our
> > jms provider which supports full XA transactions as such we can manage
> the
> > production of the change messages out the system transactionally as well
> as
> > the in grid state.
> >
> > The obvious arguement here is should we even look to move this flow off
> > JMS then. We prob shouldn't nor will do this.
> >
> > The point is that I think saying Kafka supports transactions but then not
> > supporting it as per the traditional sense leads to developers expecting
> > similar behaviour and will cause issues in prod when they find it doesn't
> > work as they're used to.
> >
> > As my other response earlier, is there a better name to describe this
> > feature, if we're not implementing transactions to the traditional
> > transaction expected, to avoid this confusion?
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Friday, December 9, 2016 6:08:07 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Michael,
> >
> > Doesn't that example have more to do with applying the update against two
> > rows in a single transaction? That is, clearly the write to Kafka needs
> to
> > be "transactional" and the write to the destination needs to be
> > transactional, but it's not clear to me that you need isolation that
> spans
> > both operations. Can you dive into the system architecture a bit more and
> > explain why Kafka needs to participate in the same transaction as the
> > destination system?
> >
> > -Jay
> >
> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> > wrote:
> >
> > > Usecase in IG:
> > >
> > > Fund transfer between accounts. When we debit one account and fund
> > another
> > > we must ensure the records to both occur as an acid action, and as a
> > single
> > > transaction.
> > >
> > > Today we achieve this because we have jms, as such we can do the
> actions
> > > needed in an xa transaction across both the accounts. To move this flow
> > to
> > > Kafka we would need support of XA transaction.
> > >
> > >
> > >
> > > Sent using OWA for iPhone
> > > ________________________________________
> > > From: Michael Pearce <Mi...@ig.com>
> > > Sent: Friday, December 9, 2016 6:09:06 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > > Messaging
> > >
> > > Hi Jay,
> > >
> > > For me having an XA transaction allows for ensuring ACID across my
> > > application.
> > >
> > > I believe it is part of the JMS api, and obviously JMS still is in
> > > enterprise very widely adopted for Messaging transport , so obviously
> to
> > > say it isn't widely used i think is ignoring a whole range of users.
> Like
> > > wise I believe frameworks like spring etc fully support it more
> evidence
> > of
> > > its wide adoption.
> > >
> > > On this note personally we try to avoid transactions entirely in our
> > flows
> > > for performance and simplicity. but we do alas unfortunately have one
> or
> > > two places we cannot ignore it.
> > >
> > > Cheers
> > > Mike
> > >
> > > Sent using OWA for iPhone
> > > ________________________________________
> > > From: Jay Kreps <ja...@confluent.io>
> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > > Messaging
> > >
> > > Hey Edoardo,
> > >
> > > For (3) can you outline what you think the benefit and use cases for a
> > more
> > > general cross-system XA feature would be an what changes to the
> proposal
> > > would be required to enable it? When I have asked people who wanted
> > > cross-system XA in the past what they wanted it for, I haven't really
> > > gotten any answers that made sense. Every person really wanted
> something
> > > that would be better solved by a transactional (or idempotent) write to
> > > Kafka followed by an independent transactional (or idempotent)
> > consumption
> > > (which this proposal enables). For the use cases they described tying
> > these
> > > two things together had no advantage and many disadvantages.
> > >
> > > I have one use case which would be accomplished by cross-system XA
> which
> > is
> > > allowing the producer to block on the synchronous processing of the
> > message
> > > by (all? some?) consumers. However I'm not convinced that cross-system
> XA
> > > is the best solution to this problem, and I'm also not convinced this
> is
> > an
> > > important problem to solve. But maybe you have something in mind here.
> > >
> > > -Jay
> > >
> > >
> > >
> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
> wrote:
> > >
> > > > Hi,
> > > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > > >
> > > > We have many users who choose not to use the Java client,  so I have
> > > > concerns about the added complexity in developing the clients.
> > > > A few questions.
> > > >
> > > > 1 - is mixing transactional and non transactional messages on the
> *same
> > > > topic-partition* really a requirement ?
> > > > What use case does it satisfy?
> > > >
> > > > 2 - I guess some clients may only be interested to implement the
> > producer
> > > > idempotency.
> > > > It's not clear how they could be implemented without having to add
> the
> > > > transaction capabilities.
> > > > As others on this list have said, I too would like to see idempotency
> > as
> > > a
> > > > more basic feature, on top which txns can be built.
> > > >
> > > > 3 - The KIP seems focused on a use case where consumption from a
> topic
> > > and
> > > > subsequent production are part of the producer transaction.
> > > >
> > > > It'd be great to see a way to extend the producer transaction to
> > include
> > > > additional transactional resources,
> > > > so that the consumption from another topic just becomes a special
> case
> > of
> > > > a more general "distributed" txn.
> > > >
> > > > Edo
> > > > --------------------------------------------------
> > > > Edoardo Comar
> > > > IBM MessageHub
> > > > ecomar@uk.ibm.com
> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > > >
> > > > IBM United Kingdom Limited Registered in England and Wales with
> number
> > > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth,
> Hants.
> > > PO6
> > > > 3AU
> > > >
> > > >
> > > >
> > > > From:   Guozhang Wang <wa...@gmail.com>
> > > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > > Date:   30/11/2016 22:20
> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > > > Messaging
> > > >
> > > >
> > > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > > >
> > > >
> > > > Unless stated otherwise above:
> > > > IBM United Kingdom Limited - Registered in England and Wales with
> > number
> > > > 741598.
> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
> PO6
> > > 3AU
> > > >
> > > The information contained in this email is strictly confidential and
> for
> > > the use of the addressee only, unless otherwise indicated. If you are
> not
> > > the intended recipient, please do not read, copy, use or disclose to
> > others
> > > this message or any attachment. Please also notify the sender by
> replying
> > > to this email or by telephone (+44(020 7896 0011) and then delete the
> > email
> > > and any copies of it. Opinions, conclusion (etc) that do not relate to
> > the
> > > official business of this company shall be understood as neither given
> > nor
> > > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > > registered in England and Wales, company number 04008957) and IG Index
> > > Limited (a company registered in England and Wales, company number
> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > > London EC4R 2YA. Both IG Markets Limited (register number 195355) and
> IG
> > > Index Limited (register number 114059) are authorised and regulated by
> > the
> > > Financial Conduct Authority.
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Apologies on the spelling.

*Hi Jay,
________________________________________
From: Michael Pearce <Mi...@ig.com>
Sent: Friday, December 9, 2016 7:52:25 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hi Jey

1) I agree, these should be used to add this in a future kip if ever was enough of a case. As stated for us I think for these systems we will keep our JMS solutions there.  I think maybe in the docs when this feature is written up, one should redirect users to alternative options such as jms brokers, for these use cases.

2) I think this kip needs to be mindful and actually own to make sure things are implemented in a way to make future enchancement easy/or at least extensible. Having to in future rework things and correct historic decisions is expensive as already finding.

Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Friday, December 9, 2016 7:19:59 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

Yeah, I don't think you need to go into the details of whatever you guys
have. I think several people in the thread said "let's do XA transactions
too!" Obviously in a world where features were free and always worked
perfectly we would! I've probably talked to about 100 people about their
use of XA transactions in different systems and my observation has been (a)
they are a bit of an operational nightmare, (b) the use cases i've
understood don't actually require full XA transactions they actually
require a much weaker and easier to guarantee property. The result is you
pay a big complexity cost for a guarantee much stronger than what you
wanted. My sense is that this opinion is broadly shared by the distributed
systems community at large and by Kafka folks in particular.

I'm a contrarian so I think it is great not to be too swayed by "common
wisdom" though. Five years ago there was a consensus that distributed
transactions were too hard to implement in an operationally sound way,
which i think was not correct, so the bad reputation for cross-system
transactions may be equally wrong!

To build a compelling case this is wrong I think two things need to be done:

   1. Build a case that there are a large/important set of use cases that
   cannot be solved with two independent transactions (as i described), and
   that these use cases are things Kafka should be able to do.
   2. Come up with the concrete extensions to the KIP-98 proposal that
   would enable an operationally sound implementation for pluggable
   multi-system XA.

-Jay



On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Hi Jay,
>
> I can't go too deep into exact implantation due to no NDA. So apologies
> here.
>
> Essentially we have multiple processes each owning selection of accounts
> so on general flows an action for an account just needs to be managed local
> to the owning node, happy days ever change is handled as a tick tock change.
>
> Unfortunately when a transfer occurs we need the two processes to
> co-ordinate their transaction, we also need to ensure both don't continue
> other actions/changesl, we do this using a data grid technology. This grid
> technology supports transaction manager that we couple into currently our
> jms provider which supports full XA transactions as such we can manage the
> production of the change messages out the system transactionally as well as
> the in grid state.
>
> The obvious arguement here is should we even look to move this flow off
> JMS then. We prob shouldn't nor will do this.
>
> The point is that I think saying Kafka supports transactions but then not
> supporting it as per the traditional sense leads to developers expecting
> similar behaviour and will cause issues in prod when they find it doesn't
> work as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 6:08:07 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Doesn't that example have more to do with applying the update against two
> rows in a single transaction? That is, clearly the write to Kafka needs to
> be "transactional" and the write to the destination needs to be
> transactional, but it's not clear to me that you need isolation that spans
> both operations. Can you dive into the system architecture a bit more and
> explain why Kafka needs to participate in the same transaction as the
> destination system?
>
> -Jay
>
> On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund transfer between accounts. When we debit one account and fund
> another
> > we must ensure the records to both occur as an acid action, and as a
> single
> > transaction.
> >
> > Today we achieve this because we have jms, as such we can do the actions
> > needed in an xa transaction across both the accounts. To move this flow
> to
> > Kafka we would need support of XA transaction.
> >
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Michael Pearce <Mi...@ig.com>
> > Sent: Friday, December 9, 2016 6:09:06 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hi Jay,
> >
> > For me having an XA transaction allows for ensuring ACID across my
> > application.
> >
> > I believe it is part of the JMS api, and obviously JMS still is in
> > enterprise very widely adopted for Messaging transport , so obviously to
> > say it isn't widely used i think is ignoring a whole range of users. Like
> > wise I believe frameworks like spring etc fully support it more evidence
> of
> > its wide adoption.
> >
> > On this note personally we try to avoid transactions entirely in our
> flows
> > for performance and simplicity. but we do alas unfortunately have one or
> > two places we cannot ignore it.
> >
> > Cheers
> > Mike
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Thursday, December 8, 2016 11:25:53 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Edoardo,
> >
> > For (3) can you outline what you think the benefit and use cases for a
> more
> > general cross-system XA feature would be an what changes to the proposal
> > would be required to enable it? When I have asked people who wanted
> > cross-system XA in the past what they wanted it for, I haven't really
> > gotten any answers that made sense. Every person really wanted something
> > that would be better solved by a transactional (or idempotent) write to
> > Kafka followed by an independent transactional (or idempotent)
> consumption
> > (which this proposal enables). For the use cases they described tying
> these
> > two things together had no advantage and many disadvantages.
> >
> > I have one use case which would be accomplished by cross-system XA which
> is
> > allowing the producer to block on the synchronous processing of the
> message
> > by (all? some?) consumers. However I'm not convinced that cross-system XA
> > is the best solution to this problem, and I'm also not convinced this is
> an
> > important problem to solve. But maybe you have something in mind here.
> >
> > -Jay
> >
> >
> >
> > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> >
> > > Hi,
> > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > >
> > > We have many users who choose not to use the Java client,  so I have
> > > concerns about the added complexity in developing the clients.
> > > A few questions.
> > >
> > > 1 - is mixing transactional and non transactional messages on the *same
> > > topic-partition* really a requirement ?
> > > What use case does it satisfy?
> > >
> > > 2 - I guess some clients may only be interested to implement the
> producer
> > > idempotency.
> > > It's not clear how they could be implemented without having to add the
> > > transaction capabilities.
> > > As others on this list have said, I too would like to see idempotency
> as
> > a
> > > more basic feature, on top which txns can be built.
> > >
> > > 3 - The KIP seems focused on a use case where consumption from a topic
> > and
> > > subsequent production are part of the producer transaction.
> > >
> > > It'd be great to see a way to extend the producer transaction to
> include
> > > additional transactional resources,
> > > so that the consumption from another topic just becomes a special case
> of
> > > a more general "distributed" txn.
> > >
> > > Edo
> > > --------------------------------------------------
> > > Edoardo Comar
> > > IBM MessageHub
> > > ecomar@uk.ibm.com
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > > IBM United Kingdom Limited Registered in England and Wales with number
> > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> > PO6
> > > 3AU
> > >
> > >
> > >
> > > From:   Guozhang Wang <wa...@gmail.com>
> > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date:   30/11/2016 22:20
> > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > Messaging
> > >
> > >
> > >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Hi Jey

1) I agree, these should be used to add this in a future kip if ever was enough of a case. As stated for us I think for these systems we will keep our JMS solutions there.  I think maybe in the docs when this feature is written up, one should redirect users to alternative options such as jms brokers, for these use cases.

2) I think this kip needs to be mindful and actually own to make sure things are implemented in a way to make future enchancement easy/or at least extensible. Having to in future rework things and correct historic decisions is expensive as already finding.

Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Friday, December 9, 2016 7:19:59 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

Yeah, I don't think you need to go into the details of whatever you guys
have. I think several people in the thread said "let's do XA transactions
too!" Obviously in a world where features were free and always worked
perfectly we would! I've probably talked to about 100 people about their
use of XA transactions in different systems and my observation has been (a)
they are a bit of an operational nightmare, (b) the use cases i've
understood don't actually require full XA transactions they actually
require a much weaker and easier to guarantee property. The result is you
pay a big complexity cost for a guarantee much stronger than what you
wanted. My sense is that this opinion is broadly shared by the distributed
systems community at large and by Kafka folks in particular.

I'm a contrarian so I think it is great not to be too swayed by "common
wisdom" though. Five years ago there was a consensus that distributed
transactions were too hard to implement in an operationally sound way,
which i think was not correct, so the bad reputation for cross-system
transactions may be equally wrong!

To build a compelling case this is wrong I think two things need to be done:

   1. Build a case that there are a large/important set of use cases that
   cannot be solved with two independent transactions (as i described), and
   that these use cases are things Kafka should be able to do.
   2. Come up with the concrete extensions to the KIP-98 proposal that
   would enable an operationally sound implementation for pluggable
   multi-system XA.

-Jay



On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Hi Jay,
>
> I can't go too deep into exact implantation due to no NDA. So apologies
> here.
>
> Essentially we have multiple processes each owning selection of accounts
> so on general flows an action for an account just needs to be managed local
> to the owning node, happy days ever change is handled as a tick tock change.
>
> Unfortunately when a transfer occurs we need the two processes to
> co-ordinate their transaction, we also need to ensure both don't continue
> other actions/changesl, we do this using a data grid technology. This grid
> technology supports transaction manager that we couple into currently our
> jms provider which supports full XA transactions as such we can manage the
> production of the change messages out the system transactionally as well as
> the in grid state.
>
> The obvious arguement here is should we even look to move this flow off
> JMS then. We prob shouldn't nor will do this.
>
> The point is that I think saying Kafka supports transactions but then not
> supporting it as per the traditional sense leads to developers expecting
> similar behaviour and will cause issues in prod when they find it doesn't
> work as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 6:08:07 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Doesn't that example have more to do with applying the update against two
> rows in a single transaction? That is, clearly the write to Kafka needs to
> be "transactional" and the write to the destination needs to be
> transactional, but it's not clear to me that you need isolation that spans
> both operations. Can you dive into the system architecture a bit more and
> explain why Kafka needs to participate in the same transaction as the
> destination system?
>
> -Jay
>
> On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund transfer between accounts. When we debit one account and fund
> another
> > we must ensure the records to both occur as an acid action, and as a
> single
> > transaction.
> >
> > Today we achieve this because we have jms, as such we can do the actions
> > needed in an xa transaction across both the accounts. To move this flow
> to
> > Kafka we would need support of XA transaction.
> >
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Michael Pearce <Mi...@ig.com>
> > Sent: Friday, December 9, 2016 6:09:06 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hi Jay,
> >
> > For me having an XA transaction allows for ensuring ACID across my
> > application.
> >
> > I believe it is part of the JMS api, and obviously JMS still is in
> > enterprise very widely adopted for Messaging transport , so obviously to
> > say it isn't widely used i think is ignoring a whole range of users. Like
> > wise I believe frameworks like spring etc fully support it more evidence
> of
> > its wide adoption.
> >
> > On this note personally we try to avoid transactions entirely in our
> flows
> > for performance and simplicity. but we do alas unfortunately have one or
> > two places we cannot ignore it.
> >
> > Cheers
> > Mike
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Thursday, December 8, 2016 11:25:53 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Edoardo,
> >
> > For (3) can you outline what you think the benefit and use cases for a
> more
> > general cross-system XA feature would be an what changes to the proposal
> > would be required to enable it? When I have asked people who wanted
> > cross-system XA in the past what they wanted it for, I haven't really
> > gotten any answers that made sense. Every person really wanted something
> > that would be better solved by a transactional (or idempotent) write to
> > Kafka followed by an independent transactional (or idempotent)
> consumption
> > (which this proposal enables). For the use cases they described tying
> these
> > two things together had no advantage and many disadvantages.
> >
> > I have one use case which would be accomplished by cross-system XA which
> is
> > allowing the producer to block on the synchronous processing of the
> message
> > by (all? some?) consumers. However I'm not convinced that cross-system XA
> > is the best solution to this problem, and I'm also not convinced this is
> an
> > important problem to solve. But maybe you have something in mind here.
> >
> > -Jay
> >
> >
> >
> > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> >
> > > Hi,
> > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > >
> > > We have many users who choose not to use the Java client,  so I have
> > > concerns about the added complexity in developing the clients.
> > > A few questions.
> > >
> > > 1 - is mixing transactional and non transactional messages on the *same
> > > topic-partition* really a requirement ?
> > > What use case does it satisfy?
> > >
> > > 2 - I guess some clients may only be interested to implement the
> producer
> > > idempotency.
> > > It's not clear how they could be implemented without having to add the
> > > transaction capabilities.
> > > As others on this list have said, I too would like to see idempotency
> as
> > a
> > > more basic feature, on top which txns can be built.
> > >
> > > 3 - The KIP seems focused on a use case where consumption from a topic
> > and
> > > subsequent production are part of the producer transaction.
> > >
> > > It'd be great to see a way to extend the producer transaction to
> include
> > > additional transactional resources,
> > > so that the consumption from another topic just becomes a special case
> of
> > > a more general "distributed" txn.
> > >
> > > Edo
> > > --------------------------------------------------
> > > Edoardo Comar
> > > IBM MessageHub
> > > ecomar@uk.ibm.com
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > > IBM United Kingdom Limited Registered in England and Wales with number
> > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> > PO6
> > > 3AU
> > >
> > >
> > >
> > > From:   Guozhang Wang <wa...@gmail.com>
> > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date:   30/11/2016 22:20
> > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > Messaging
> > >
> > >
> > >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Michael,

Yeah, I don't think you need to go into the details of whatever you guys
have. I think several people in the thread said "let's do XA transactions
too!" Obviously in a world where features were free and always worked
perfectly we would! I've probably talked to about 100 people about their
use of XA transactions in different systems and my observation has been (a)
they are a bit of an operational nightmare, (b) the use cases i've
understood don't actually require full XA transactions they actually
require a much weaker and easier to guarantee property. The result is you
pay a big complexity cost for a guarantee much stronger than what you
wanted. My sense is that this opinion is broadly shared by the distributed
systems community at large and by Kafka folks in particular.

I'm a contrarian so I think it is great not to be too swayed by "common
wisdom" though. Five years ago there was a consensus that distributed
transactions were too hard to implement in an operationally sound way,
which i think was not correct, so the bad reputation for cross-system
transactions may be equally wrong!

To build a compelling case this is wrong I think two things need to be done:

   1. Build a case that there are a large/important set of use cases that
   cannot be solved with two independent transactions (as i described), and
   that these use cases are things Kafka should be able to do.
   2. Come up with the concrete extensions to the KIP-98 proposal that
   would enable an operationally sound implementation for pluggable
   multi-system XA.

-Jay



On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Hi Jay,
>
> I can't go too deep into exact implantation due to no NDA. So apologies
> here.
>
> Essentially we have multiple processes each owning selection of accounts
> so on general flows an action for an account just needs to be managed local
> to the owning node, happy days ever change is handled as a tick tock change.
>
> Unfortunately when a transfer occurs we need the two processes to
> co-ordinate their transaction, we also need to ensure both don't continue
> other actions/changesl, we do this using a data grid technology. This grid
> technology supports transaction manager that we couple into currently our
> jms provider which supports full XA transactions as such we can manage the
> production of the change messages out the system transactionally as well as
> the in grid state.
>
> The obvious arguement here is should we even look to move this flow off
> JMS then. We prob shouldn't nor will do this.
>
> The point is that I think saying Kafka supports transactions but then not
> supporting it as per the traditional sense leads to developers expecting
> similar behaviour and will cause issues in prod when they find it doesn't
> work as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 6:08:07 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Doesn't that example have more to do with applying the update against two
> rows in a single transaction? That is, clearly the write to Kafka needs to
> be "transactional" and the write to the destination needs to be
> transactional, but it's not clear to me that you need isolation that spans
> both operations. Can you dive into the system architecture a bit more and
> explain why Kafka needs to participate in the same transaction as the
> destination system?
>
> -Jay
>
> On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund transfer between accounts. When we debit one account and fund
> another
> > we must ensure the records to both occur as an acid action, and as a
> single
> > transaction.
> >
> > Today we achieve this because we have jms, as such we can do the actions
> > needed in an xa transaction across both the accounts. To move this flow
> to
> > Kafka we would need support of XA transaction.
> >
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Michael Pearce <Mi...@ig.com>
> > Sent: Friday, December 9, 2016 6:09:06 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hi Jay,
> >
> > For me having an XA transaction allows for ensuring ACID across my
> > application.
> >
> > I believe it is part of the JMS api, and obviously JMS still is in
> > enterprise very widely adopted for Messaging transport , so obviously to
> > say it isn't widely used i think is ignoring a whole range of users. Like
> > wise I believe frameworks like spring etc fully support it more evidence
> of
> > its wide adoption.
> >
> > On this note personally we try to avoid transactions entirely in our
> flows
> > for performance and simplicity. but we do alas unfortunately have one or
> > two places we cannot ignore it.
> >
> > Cheers
> > Mike
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Thursday, December 8, 2016 11:25:53 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Edoardo,
> >
> > For (3) can you outline what you think the benefit and use cases for a
> more
> > general cross-system XA feature would be an what changes to the proposal
> > would be required to enable it? When I have asked people who wanted
> > cross-system XA in the past what they wanted it for, I haven't really
> > gotten any answers that made sense. Every person really wanted something
> > that would be better solved by a transactional (or idempotent) write to
> > Kafka followed by an independent transactional (or idempotent)
> consumption
> > (which this proposal enables). For the use cases they described tying
> these
> > two things together had no advantage and many disadvantages.
> >
> > I have one use case which would be accomplished by cross-system XA which
> is
> > allowing the producer to block on the synchronous processing of the
> message
> > by (all? some?) consumers. However I'm not convinced that cross-system XA
> > is the best solution to this problem, and I'm also not convinced this is
> an
> > important problem to solve. But maybe you have something in mind here.
> >
> > -Jay
> >
> >
> >
> > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> >
> > > Hi,
> > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > >
> > > We have many users who choose not to use the Java client,  so I have
> > > concerns about the added complexity in developing the clients.
> > > A few questions.
> > >
> > > 1 - is mixing transactional and non transactional messages on the *same
> > > topic-partition* really a requirement ?
> > > What use case does it satisfy?
> > >
> > > 2 - I guess some clients may only be interested to implement the
> producer
> > > idempotency.
> > > It's not clear how they could be implemented without having to add the
> > > transaction capabilities.
> > > As others on this list have said, I too would like to see idempotency
> as
> > a
> > > more basic feature, on top which txns can be built.
> > >
> > > 3 - The KIP seems focused on a use case where consumption from a topic
> > and
> > > subsequent production are part of the producer transaction.
> > >
> > > It'd be great to see a way to extend the producer transaction to
> include
> > > additional transactional resources,
> > > so that the consumption from another topic just becomes a special case
> of
> > > a more general "distributed" txn.
> > >
> > > Edo
> > > --------------------------------------------------
> > > Edoardo Comar
> > > IBM MessageHub
> > > ecomar@uk.ibm.com
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > > IBM United Kingdom Limited Registered in England and Wales with number
> > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> > PO6
> > > 3AU
> > >
> > >
> > >
> > > From:   Guozhang Wang <wa...@gmail.com>
> > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date:   30/11/2016 22:20
> > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > Messaging
> > >
> > >
> > >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
With respect to the naming, I think it makes sense for two reasons.

   1. The begin/commit/rollback semantics are what people expect from
   transactions, i.e. it passes the intuitive "gut check" meaning people have.
   If you make up a new name it will likely not convey the expectation.
   2. There is a strong analogy between logs and tables and this feature
   enables ACID-like usage of the log in modifying a table.

I'll dive into the later. What does ACID mean for a log? Well it is
semi-well-defined for a table, what would the implications be for the
equivalent log representation?

Here is the analogy:

   - *Atomic* - This is straight-forward, a set of updates are either all
   published or all not published.
   - *Consistent* - This one is not very well defined in ACID, it isn't
   always as simple as linearizability. It alternately means either (a) a
   transaction started now sees all past committed transactions, (b) the
   database checks various DB-specific things like foreign-key constraints, or
   (c) some undefined notion of application correctness without any particular
   invariant. I think the most sane interpretation is based on (a) and means
   that the consumer sees transactions in commit order. We don't try to
   guarantee this, but for a log the reader controls the order of processing
   so this is possible. We could add future features to do this reordering for
   people as a convenience.
   - *Isolated* - In ACID this means a reader doesn't see the results of
   uncommitted transactions. In a log this means you get complete transactions
   all at once rather than getting half a transaction. This is primarily up to
   you in how you use the data you consume.
   - *Durable* - This falls out of Kafka's replication.

I'm less worried about confusion with other messaging systems. Kafka is
genuinely different in a number of areas and it is worth people
understanding that difference.

-Jay

On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <Mi...@ig.com>
wrote:

> Hi Jay,
>
> I can't go too deep into exact implantation due to no NDA. So apologies
> here.
>
> Essentially we have multiple processes each owning selection of accounts
> so on general flows an action for an account just needs to be managed local
> to the owning node, happy days ever change is handled as a tick tock change.
>
> Unfortunately when a transfer occurs we need the two processes to
> co-ordinate their transaction, we also need to ensure both don't continue
> other actions/changesl, we do this using a data grid technology. This grid
> technology supports transaction manager that we couple into currently our
> jms provider which supports full XA transactions as such we can manage the
> production of the change messages out the system transactionally as well as
> the in grid state.
>
> The obvious arguement here is should we even look to move this flow off
> JMS then. We prob shouldn't nor will do this.
>
> The point is that I think saying Kafka supports transactions but then not
> supporting it as per the traditional sense leads to developers expecting
> similar behaviour and will cause issues in prod when they find it doesn't
> work as they're used to.
>
> As my other response earlier, is there a better name to describe this
> feature, if we're not implementing transactions to the traditional
> transaction expected, to avoid this confusion?
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Friday, December 9, 2016 6:08:07 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Michael,
>
> Doesn't that example have more to do with applying the update against two
> rows in a single transaction? That is, clearly the write to Kafka needs to
> be "transactional" and the write to the destination needs to be
> transactional, but it's not clear to me that you need isolation that spans
> both operations. Can you dive into the system architecture a bit more and
> explain why Kafka needs to participate in the same transaction as the
> destination system?
>
> -Jay
>
> On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund transfer between accounts. When we debit one account and fund
> another
> > we must ensure the records to both occur as an acid action, and as a
> single
> > transaction.
> >
> > Today we achieve this because we have jms, as such we can do the actions
> > needed in an xa transaction across both the accounts. To move this flow
> to
> > Kafka we would need support of XA transaction.
> >
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Michael Pearce <Mi...@ig.com>
> > Sent: Friday, December 9, 2016 6:09:06 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hi Jay,
> >
> > For me having an XA transaction allows for ensuring ACID across my
> > application.
> >
> > I believe it is part of the JMS api, and obviously JMS still is in
> > enterprise very widely adopted for Messaging transport , so obviously to
> > say it isn't widely used i think is ignoring a whole range of users. Like
> > wise I believe frameworks like spring etc fully support it more evidence
> of
> > its wide adoption.
> >
> > On this note personally we try to avoid transactions entirely in our
> flows
> > for performance and simplicity. but we do alas unfortunately have one or
> > two places we cannot ignore it.
> >
> > Cheers
> > Mike
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Thursday, December 8, 2016 11:25:53 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Edoardo,
> >
> > For (3) can you outline what you think the benefit and use cases for a
> more
> > general cross-system XA feature would be an what changes to the proposal
> > would be required to enable it? When I have asked people who wanted
> > cross-system XA in the past what they wanted it for, I haven't really
> > gotten any answers that made sense. Every person really wanted something
> > that would be better solved by a transactional (or idempotent) write to
> > Kafka followed by an independent transactional (or idempotent)
> consumption
> > (which this proposal enables). For the use cases they described tying
> these
> > two things together had no advantage and many disadvantages.
> >
> > I have one use case which would be accomplished by cross-system XA which
> is
> > allowing the producer to block on the synchronous processing of the
> message
> > by (all? some?) consumers. However I'm not convinced that cross-system XA
> > is the best solution to this problem, and I'm also not convinced this is
> an
> > important problem to solve. But maybe you have something in mind here.
> >
> > -Jay
> >
> >
> >
> > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> >
> > > Hi,
> > > thanks, very interesting KIP ... I haven't fully digested it yet.
> > >
> > > We have many users who choose not to use the Java client,  so I have
> > > concerns about the added complexity in developing the clients.
> > > A few questions.
> > >
> > > 1 - is mixing transactional and non transactional messages on the *same
> > > topic-partition* really a requirement ?
> > > What use case does it satisfy?
> > >
> > > 2 - I guess some clients may only be interested to implement the
> producer
> > > idempotency.
> > > It's not clear how they could be implemented without having to add the
> > > transaction capabilities.
> > > As others on this list have said, I too would like to see idempotency
> as
> > a
> > > more basic feature, on top which txns can be built.
> > >
> > > 3 - The KIP seems focused on a use case where consumption from a topic
> > and
> > > subsequent production are part of the producer transaction.
> > >
> > > It'd be great to see a way to extend the producer transaction to
> include
> > > additional transactional resources,
> > > so that the consumption from another topic just becomes a special case
> of
> > > a more general "distributed" txn.
> > >
> > > Edo
> > > --------------------------------------------------
> > > Edoardo Comar
> > > IBM MessageHub
> > > ecomar@uk.ibm.com
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > > IBM United Kingdom Limited Registered in England and Wales with number
> > > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> > PO6
> > > 3AU
> > >
> > >
> > >
> > > From:   Guozhang Wang <wa...@gmail.com>
> > > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date:   30/11/2016 22:20
> > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > Messaging
> > >
> > >
> > >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7C
> P1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Hi Jay,

I can't go too deep into exact implantation due to no NDA. So apologies here.

Essentially we have multiple processes each owning selection of accounts so on general flows an action for an account just needs to be managed local to the owning node, happy days ever change is handled as a tick tock change.

Unfortunately when a transfer occurs we need the two processes to co-ordinate their transaction, we also need to ensure both don't continue other actions/changesl, we do this using a data grid technology. This grid technology supports transaction manager that we couple into currently our jms provider which supports full XA transactions as such we can manage the production of the change messages out the system transactionally as well as the in grid state.

The obvious arguement here is should we even look to move this flow off JMS then. We prob shouldn't nor will do this.

The point is that I think saying Kafka supports transactions but then not supporting it as per the traditional sense leads to developers expecting similar behaviour and will cause issues in prod when they find it doesn't work as they're used to.

As my other response earlier, is there a better name to describe this feature, if we're not implementing transactions to the traditional transaction expected, to avoid this confusion?


Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Friday, December 9, 2016 6:08:07 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

Doesn't that example have more to do with applying the update against two
rows in a single transaction? That is, clearly the write to Kafka needs to
be "transactional" and the write to the destination needs to be
transactional, but it's not clear to me that you need isolation that spans
both operations. Can you dive into the system architecture a bit more and
explain why Kafka needs to participate in the same transaction as the
destination system?

-Jay

On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
wrote:

> Usecase in IG:
>
> Fund transfer between accounts. When we debit one account and fund another
> we must ensure the records to both occur as an acid action, and as a single
> transaction.
>
> Today we achieve this because we have jms, as such we can do the actions
> needed in an xa transaction across both the accounts. To move this flow to
> Kafka we would need support of XA transaction.
>
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Michael Pearce <Mi...@ig.com>
> Sent: Friday, December 9, 2016 6:09:06 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hi Jay,
>
> For me having an XA transaction allows for ensuring ACID across my
> application.
>
> I believe it is part of the JMS api, and obviously JMS still is in
> enterprise very widely adopted for Messaging transport , so obviously to
> say it isn't widely used i think is ignoring a whole range of users. Like
> wise I believe frameworks like spring etc fully support it more evidence of
> its wide adoption.
>
> On this note personally we try to avoid transactions entirely in our flows
> for performance and simplicity. but we do alas unfortunately have one or
> two places we cannot ignore it.
>
> Cheers
> Mike
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Thursday, December 8, 2016 11:25:53 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Edoardo,
>
> For (3) can you outline what you think the benefit and use cases for a more
> general cross-system XA feature would be an what changes to the proposal
> would be required to enable it? When I have asked people who wanted
> cross-system XA in the past what they wanted it for, I haven't really
> gotten any answers that made sense. Every person really wanted something
> that would be better solved by a transactional (or idempotent) write to
> Kafka followed by an independent transactional (or idempotent) consumption
> (which this proposal enables). For the use cases they described tying these
> two things together had no advantage and many disadvantages.
>
> I have one use case which would be accomplished by cross-system XA which is
> allowing the producer to block on the synchronous processing of the message
> by (all? some?) consumers. However I'm not convinced that cross-system XA
> is the best solution to this problem, and I'm also not convinced this is an
> important problem to solve. But maybe you have something in mind here.
>
> -Jay
>
>
>
> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
>
> > Hi,
> > thanks, very interesting KIP ... I haven't fully digested it yet.
> >
> > We have many users who choose not to use the Java client,  so I have
> > concerns about the added complexity in developing the clients.
> > A few questions.
> >
> > 1 - is mixing transactional and non transactional messages on the *same
> > topic-partition* really a requirement ?
> > What use case does it satisfy?
> >
> > 2 - I guess some clients may only be interested to implement the producer
> > idempotency.
> > It's not clear how they could be implemented without having to add the
> > transaction capabilities.
> > As others on this list have said, I too would like to see idempotency as
> a
> > more basic feature, on top which txns can be built.
> >
> > 3 - The KIP seems focused on a use case where consumption from a topic
> and
> > subsequent production are part of the producer transaction.
> >
> > It'd be great to see a way to extend the producer transaction to include
> > additional transactional resources,
> > so that the consumption from another topic just becomes a special case of
> > a more general "distributed" txn.
> >
> > Edo
> > --------------------------------------------------
> > Edoardo Comar
> > IBM MessageHub
> > ecomar@uk.ibm.com
> > IBM UK Ltd, Hursley Park, SO21 2JN
> >
> > IBM United Kingdom Limited Registered in England and Wales with number
> > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> PO6
> > 3AU
> >
> >
> >
> > From:   Guozhang Wang <wa...@gmail.com>
> > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > Date:   30/11/2016 22:20
> > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> >
> >
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
> >
> >
> > Unless stated otherwise above:
> > IBM United Kingdom Limited - Registered in England and Wales with number
> > 741598.
> > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> 3AU
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Michael,

Doesn't that example have more to do with applying the update against two
rows in a single transaction? That is, clearly the write to Kafka needs to
be "transactional" and the write to the destination needs to be
transactional, but it's not clear to me that you need isolation that spans
both operations. Can you dive into the system architecture a bit more and
explain why Kafka needs to participate in the same transaction as the
destination system?

-Jay

On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <Mi...@ig.com>
wrote:

> Usecase in IG:
>
> Fund transfer between accounts. When we debit one account and fund another
> we must ensure the records to both occur as an acid action, and as a single
> transaction.
>
> Today we achieve this because we have jms, as such we can do the actions
> needed in an xa transaction across both the accounts. To move this flow to
> Kafka we would need support of XA transaction.
>
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Michael Pearce <Mi...@ig.com>
> Sent: Friday, December 9, 2016 6:09:06 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hi Jay,
>
> For me having an XA transaction allows for ensuring ACID across my
> application.
>
> I believe it is part of the JMS api, and obviously JMS still is in
> enterprise very widely adopted for Messaging transport , so obviously to
> say it isn't widely used i think is ignoring a whole range of users. Like
> wise I believe frameworks like spring etc fully support it more evidence of
> its wide adoption.
>
> On this note personally we try to avoid transactions entirely in our flows
> for performance and simplicity. but we do alas unfortunately have one or
> two places we cannot ignore it.
>
> Cheers
> Mike
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Thursday, December 8, 2016 11:25:53 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Edoardo,
>
> For (3) can you outline what you think the benefit and use cases for a more
> general cross-system XA feature would be an what changes to the proposal
> would be required to enable it? When I have asked people who wanted
> cross-system XA in the past what they wanted it for, I haven't really
> gotten any answers that made sense. Every person really wanted something
> that would be better solved by a transactional (or idempotent) write to
> Kafka followed by an independent transactional (or idempotent) consumption
> (which this proposal enables). For the use cases they described tying these
> two things together had no advantage and many disadvantages.
>
> I have one use case which would be accomplished by cross-system XA which is
> allowing the producer to block on the synchronous processing of the message
> by (all? some?) consumers. However I'm not convinced that cross-system XA
> is the best solution to this problem, and I'm also not convinced this is an
> important problem to solve. But maybe you have something in mind here.
>
> -Jay
>
>
>
> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
>
> > Hi,
> > thanks, very interesting KIP ... I haven't fully digested it yet.
> >
> > We have many users who choose not to use the Java client,  so I have
> > concerns about the added complexity in developing the clients.
> > A few questions.
> >
> > 1 - is mixing transactional and non transactional messages on the *same
> > topic-partition* really a requirement ?
> > What use case does it satisfy?
> >
> > 2 - I guess some clients may only be interested to implement the producer
> > idempotency.
> > It's not clear how they could be implemented without having to add the
> > transaction capabilities.
> > As others on this list have said, I too would like to see idempotency as
> a
> > more basic feature, on top which txns can be built.
> >
> > 3 - The KIP seems focused on a use case where consumption from a topic
> and
> > subsequent production are part of the producer transaction.
> >
> > It'd be great to see a way to extend the producer transaction to include
> > additional transactional resources,
> > so that the consumption from another topic just becomes a special case of
> > a more general "distributed" txn.
> >
> > Edo
> > --------------------------------------------------
> > Edoardo Comar
> > IBM MessageHub
> > ecomar@uk.ibm.com
> > IBM UK Ltd, Hursley Park, SO21 2JN
> >
> > IBM United Kingdom Limited Registered in England and Wales with number
> > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> PO6
> > 3AU
> >
> >
> >
> > From:   Guozhang Wang <wa...@gmail.com>
> > To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> > Date:   30/11/2016 22:20
> > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> >
> >
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
> >
> >
> > Unless stated otherwise above:
> > IBM United Kingdom Limited - Registered in England and Wales with number
> > 741598.
> > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> 3AU
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Should the word transaction therefor be avoided here and another way to describe the feature found.

My worry is that by naming this transaction there instantly becomes a tie and expectation to traditional transaction properties.
________________________________________
From: Edoardo Comar <EC...@uk.ibm.com>
Sent: Friday, December 9, 2016 3:54:43 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hi Michael,
I don't think KIP-98 should be considered a half-way house.
It would provide transaction with the level of durability that Kafka
provides.

Which is as Andrew suggests, different to other transactional systems.
BTW, I am convinced that Andrew has answered the 3rd of my previous 3
questions below, thanks.

Edo
--------------------------------------------------
Edoardo Comar
IBM MessageHub
ecomar@uk.ibm.com
IBM UK Ltd, Hursley Park, SO21 2JN

IBM United Kingdom Limited Registered in England and Wales with number
741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
3AU



From:   Michael André Pearce <mi...@me.com>
To:     dev@kafka.apache.org
Date:   09/12/2016 12:40
Subject:        Re: [DISCUSS] KIP-98: Exactly Once Delivery and
Transactional Messaging



Agreed, I think it is better to either support transactions properly and
fully to the level or expectation normally associated with a transactional
system. Or don't provide it at all.

Having a half way house can be more dangerous.

As I said earlier the issue of message duplication can be handled entirely
without transactions.

Sent from my iPhone

> On 9 Dec 2016, at 10:49, Andrew Schofield
<an...@outlook.com> wrote:
>
> I've been pondering this question of coordinating other resource
managers with
> Kafka transactions for a while and I'm not convinced it's a good idea.
My
> reservations come down to the guarantees that it would provide in
failure
> scenarios.
>
> I don't think KIP-98 gives proper ACID semantics in the presence of all
> failures. For a transaction which contains a mixture of publishes and
offset
> updates, a bunch of topics are involved and it appears to me that an
> uncontrolled shutdown could result in some but not all of the lazy
writes
> making it to disk.
>
> Here are some of the failures that I'm worried about:
>
> * A message is published to a topic which crashes the leader Kafka node,
as
>  it's replicated across the cluster, it crashes all of the other Kafka
nodes
>  (we've really had this - SEGV, our fault and we've fixed it, but it
happened)
>  so this is a kind of rolling node crash in a cluster
> * Out of memory error in one or more Kafka nodes
> * Disk fills in one or more Kafka nodes
> * Uncontrolled power-off to all nodes in the cluster
>
> Does KIP-98 guarantee atomicity for transactions in all of these cases?
> Unless all of the topics involved in a transaction are recovered to the
> same point in time, you can't consider a transaction to be properly
atomic.
> If application code is designed expecting atomicity, there are going to
be
> tears. Perhaps only when disaster strikes, but the risk is there.
>
> I think KIP-98 is interesting, but I wouldn't equate what it provides
> with the transactions provided by resource managers with traditional
transaction
> logging. It's not better or worse, just different. If you tried to
migrate
> from a previous transactional system to Kafka transactions, I think
you'd
> better have procedures for reconciliation with the other resource
managers.
> Better still, don't build applications that are so fragile. The
principle
> of dumb pipes and smart endpoints is good in my view.
>
> If you're creating a global transaction containing two or more resource
> managers and using two-phase commit, it's very important that all of the
> resource managers maintain a coherent view of the sequence of events. If
any
> part fails due to a software or hardware failure, once the system is
> recovered, nothing must be forgotten. If you look at how presume-abort
> works, you'll see how important this is.
>
> Kafka doesn't really fit very nicely in this kind of environment because
of
> the way that it writes lazily to disk. The theory is that you must avoid
at all
> costs having an uncontrolled shutdown of an entire cluster because
you'll lose
> a little data at the end of the logs. So, if you are coordinating Kafka
and a
> relational database in a global transaction, it's theoretically possible
that
> a crashed Kafka would be a little forgetful while a crashed database
would not.
> The database would be an order of magnitude or more slower because of
the way
> its recovery logs are handled, but it would not be forgetful in the same
way.
>
> You get exactly the same kind of worries when you implement some kind of
> asynchronous replication for disaster recovery, even if all of the
resource
> managers force all of their log writes to disk eagerly. The replica at
the DR
> site is slightly behind the primary site, so if you have to recover from
an
> outage and switch to the DR site, it can be considered to be slightly
forgetful
> about the last few moments before the outage. This is why a DR plan
usually has
> some concept of compensation or reconciliation to make good any
forgotten work.
>
> In summary, I think Kafka would have to change in ways which would
negate many
> of its good points in order to support XA transactions. It would be
better to
> design applications to be resilient to message duplication and loss,
rather
> than tightly coupling resource managers and ending up with something
fragile.
>
> Don't get me wrong. This is not an anti-Kafka rant. I just work with
people
> used to traditional transactional systems, making use of Kafka for
business
> applications, and it's important to understand the concepts on both
sides
> and make sure your assumptions are valid.
>
> Andrew Schofield
> IBM Watson and Cloud Platform
>
>
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: 09 December 2016 06:19
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
Messaging
>>
>> Usecase in IG:
>>
>> Fund transfer between accounts. When we debit one account and fund
another we must ensure the records to both occur > as an acid action, and
as a single transaction.
>>
>> Today we achieve this because we have jms, as such we can do the
actions needed in an xa transaction across both the > accounts. To move
this flow to Kafka we would need support of XA transaction.
>>
>>
>>
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: Friday, December 9, 2016 6:09:06 AM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
Messaging
>>
>> Hi Jay,
>>
>> For me having an XA transaction allows for ensuring ACID across my
application.
>>
>> I believe it is part of the JMS api, and obviously JMS still is in
enterprise very widely adopted for Messaging transport , so > obviously to
say it isn't widely used i think is ignoring a whole range of users. Like
wise I believe frameworks like spring etc fully  support it more evidence
of its wide adoption.
>>
>> On this note personally we try to avoid transactions entirely in our
flows for performance and simplicity. but we do alas unfortunately have
one or two places we cannot ignore it.
>>
>> Cheers
>> Mike
>>
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Jay Kreps <ja...@confluent.io>
>> Sent: Thursday, December 8, 2016 11:25:53 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
Messaging
>>
>> Hey Edoardo,
>>
>> For (3) can you outline what you think the benefit and use cases for a
more
>> general cross-system XA feature would be an what changes to the
proposal
>> would be required to enable it? When I have asked people who wanted
>> cross-system XA in the past what they wanted it for, I haven't really
>> gotten any answers that made sense. Every person really wanted
something
>> that would be better solved by a transactional (or idempotent) write to
>> Kafka followed by an independent transactional (or idempotent)
consumption
>> (which this proposal enables). For the use cases they described tying
these
>> two things together had no advantage and many disadvantages.
>>
>> I have one use case which would be accomplished by cross-system XA
which is
>> allowing the producer to block on the synchronous processing of the
message
>> by (all? some?) consumers. However I'm not convinced that cross-system
XA
>> is the best solution to this problem, and I'm also not convinced this
is an
>> important problem to solve. But maybe you have something in mind here.
>>
>> -Jay
>>
>>
>>
>>> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com>
wrote:
>>>
>>> Hi,
>>> thanks, very interesting KIP ... I haven't fully digested it yet.
>>>
>>> We have many users who choose not to use the Java client,  so I have
>>> concerns about the added complexity in developing the clients.
>>> A few questions.
>>>
>>> 1 - is mixing transactional and non transactional messages on the
*same
>>> topic-partition* really a requirement ?
>>> What use case does it satisfy?
>>>
>>> 2 - I guess some clients may only be interested to implement the
producer
>>> idempotency.
>>> It's not clear how they could be implemented without having to add the
>>> transaction capabilities.
>>> As others on this list have said, I too would like to see idempotency
as a
>>> more basic feature, on top which txns can be built.
>>>
>>> 3 - The KIP seems focused on a use case where consumption from a topic
and
>>> subsequent production are part of the producer transaction.
>>>
>>> It'd be great to see a way to extend the producer transaction to
include
>>> additional transactional resources,
>>> so that the consumption from another topic just becomes a special case
of
>>> a more general "distributed" txn.
>>>
>>> Edo
>>> --------------------------------------------------
>>> Edoardo Comar
>>> IBM MessageHub
>>> ecomar@uk.ibm.com
>>> IBM UK Ltd, Hursley Park, SO21 2JN
>>>
>>> IBM United Kingdom Limited Registered in England and Wales with number
>>> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
PO6
>> 3AU
>>
>>
>>
>> From:   Guozhang Wang <wa...@gmail.com>
>> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
>> Date:   30/11/2016 22:20
>> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
Transactional
>> Messaging
>>
>>
>>
>> Hi all,
>>
>> I have just created KIP-98 to enhance Kafka with exactly once delivery
>> semantics:
>>
>> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>>> *
>>
>> This KIP adds a transactional messaging mechanism along with an
idempotent
>> producer implementation to make sure that 1) duplicated messages sent
from
>> the same identified producer can be detected on the broker side, and 2)
a
>> group of messages sent within a transaction will atomically be either
>> reflected and fetchable to consumers or not as a whole.
>>
>> The above wiki page provides a high-level view of the proposed changes
as
>> well as summarized guarantees. Initial draft of the detailed
>> implementation
>> design is described in this Google doc:
>>
>> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> 0wSw9ra8
>>
>>
>> We would love to hear your comments and suggestions.
>>
>> Thanks,
>>
>> -- Guozhang
>>
>>
>>
>> Unless stated otherwise above:
>> IBM United Kingdom Limited - Registered in England and Wales with
number
>> 741598.
>> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
3AU
>>
> The information contained in this email is strictly confidential and for
the use of the addressee only, unless otherwise indicated. If you are not
the intended recipient, please do not read, copy, use or disclose to
others this message or any attachment. Please  also notify the sender by
replying to this email or by telephone (+44(020 7896 0011) and then delete
the email and any copies of it. Opinions, conclusion (etc) that do not
relate to the official business of this company shall be understood as
neither given  nor endorsed by it. IG is a trading name of IG Markets
Limited (a company registered in England and Wales, company number
04008957) and IG Index Limited (a company registered in England and Wales,
company number 01190902). Registered address at Cannon Bridge  House, 25
Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number
195355) and IG Index Limited (register number 114059) are authorised and
regulated by the Financial Conduct Authority.



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number
741598.
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Edoardo Comar <EC...@uk.ibm.com>.
Hi Michael,
I don't think KIP-98 should be considered a half-way house. 
It would provide transaction with the level of durability that Kafka 
provides.

Which is as Andrew suggests, different to other transactional systems.
BTW, I am convinced that Andrew has answered the 3rd of my previous 3 
questions below, thanks.

Edo
--------------------------------------------------
Edoardo Comar
IBM MessageHub
ecomar@uk.ibm.com
IBM UK Ltd, Hursley Park, SO21 2JN

IBM United Kingdom Limited Registered in England and Wales with number 
741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 
3AU



From:   Michael André Pearce <mi...@me.com>
To:     dev@kafka.apache.org
Date:   09/12/2016 12:40
Subject:        Re: [DISCUSS] KIP-98: Exactly Once Delivery and 
Transactional Messaging



Agreed, I think it is better to either support transactions properly and 
fully to the level or expectation normally associated with a transactional 
system. Or don't provide it at all. 

Having a half way house can be more dangerous.

As I said earlier the issue of message duplication can be handled entirely 
without transactions.

Sent from my iPhone

> On 9 Dec 2016, at 10:49, Andrew Schofield 
<an...@outlook.com> wrote:
> 
> I've been pondering this question of coordinating other resource 
managers with
> Kafka transactions for a while and I'm not convinced it's a good idea. 
My
> reservations come down to the guarantees that it would provide in 
failure
> scenarios.
> 
> I don't think KIP-98 gives proper ACID semantics in the presence of all
> failures. For a transaction which contains a mixture of publishes and 
offset
> updates, a bunch of topics are involved and it appears to me that an
> uncontrolled shutdown could result in some but not all of the lazy 
writes
> making it to disk.
> 
> Here are some of the failures that I'm worried about:
> 
> * A message is published to a topic which crashes the leader Kafka node, 
as
>  it's replicated across the cluster, it crashes all of the other Kafka 
nodes
>  (we've really had this - SEGV, our fault and we've fixed it, but it 
happened)
>  so this is a kind of rolling node crash in a cluster
> * Out of memory error in one or more Kafka nodes
> * Disk fills in one or more Kafka nodes
> * Uncontrolled power-off to all nodes in the cluster
> 
> Does KIP-98 guarantee atomicity for transactions in all of these cases?
> Unless all of the topics involved in a transaction are recovered to the
> same point in time, you can't consider a transaction to be properly 
atomic.
> If application code is designed expecting atomicity, there are going to 
be
> tears. Perhaps only when disaster strikes, but the risk is there.
> 
> I think KIP-98 is interesting, but I wouldn't equate what it provides
> with the transactions provided by resource managers with traditional 
transaction
> logging. It's not better or worse, just different. If you tried to 
migrate
> from a previous transactional system to Kafka transactions, I think 
you'd
> better have procedures for reconciliation with the other resource 
managers.
> Better still, don't build applications that are so fragile. The 
principle
> of dumb pipes and smart endpoints is good in my view.
> 
> If you're creating a global transaction containing two or more resource
> managers and using two-phase commit, it's very important that all of the
> resource managers maintain a coherent view of the sequence of events. If 
any
> part fails due to a software or hardware failure, once the system is
> recovered, nothing must be forgotten. If you look at how presume-abort
> works, you'll see how important this is.
> 
> Kafka doesn't really fit very nicely in this kind of environment because 
of
> the way that it writes lazily to disk. The theory is that you must avoid 
at all
> costs having an uncontrolled shutdown of an entire cluster because 
you'll lose
> a little data at the end of the logs. So, if you are coordinating Kafka 
and a
> relational database in a global transaction, it's theoretically possible 
that
> a crashed Kafka would be a little forgetful while a crashed database 
would not.
> The database would be an order of magnitude or more slower because of 
the way
> its recovery logs are handled, but it would not be forgetful in the same 
way.
> 
> You get exactly the same kind of worries when you implement some kind of
> asynchronous replication for disaster recovery, even if all of the 
resource
> managers force all of their log writes to disk eagerly. The replica at 
the DR
> site is slightly behind the primary site, so if you have to recover from 
an
> outage and switch to the DR site, it can be considered to be slightly 
forgetful
> about the last few moments before the outage. This is why a DR plan 
usually has
> some concept of compensation or reconciliation to make good any 
forgotten work.
> 
> In summary, I think Kafka would have to change in ways which would 
negate many
> of its good points in order to support XA transactions. It would be 
better to
> design applications to be resilient to message duplication and loss, 
rather
> than tightly coupling resource managers and ending up with something 
fragile.
> 
> Don't get me wrong. This is not an anti-Kafka rant. I just work with 
people
> used to traditional transactional systems, making use of Kafka for 
business
> applications, and it's important to understand the concepts on both 
sides
> and make sure your assumptions are valid.
> 
> Andrew Schofield
> IBM Watson and Cloud Platform
> 
> 
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: 09 December 2016 06:19
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional 
Messaging
>> 
>> Usecase in IG:
>> 
>> Fund transfer between accounts. When we debit one account and fund 
another we must ensure the records to both occur > as an acid action, and 
as a single transaction.
>> 
>> Today we achieve this because we have jms, as such we can do the 
actions needed in an xa transaction across both the > accounts. To move 
this flow to Kafka we would need support of XA transaction.
>> 
>> 
>> 
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: Friday, December 9, 2016 6:09:06 AM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional 
Messaging
>> 
>> Hi Jay,
>> 
>> For me having an XA transaction allows for ensuring ACID across my 
application.
>> 
>> I believe it is part of the JMS api, and obviously JMS still is in 
enterprise very widely adopted for Messaging transport , so > obviously to 
say it isn't widely used i think is ignoring a whole range of users. Like 
wise I believe frameworks like spring etc fully  support it more evidence 
of its wide adoption.
>> 
>> On this note personally we try to avoid transactions entirely in our 
flows for performance and simplicity. but we do alas unfortunately have 
one or two places we cannot ignore it.
>> 
>> Cheers
>> Mike
>> 
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Jay Kreps <ja...@confluent.io>
>> Sent: Thursday, December 8, 2016 11:25:53 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional 
Messaging
>> 
>> Hey Edoardo,
>> 
>> For (3) can you outline what you think the benefit and use cases for a 
more
>> general cross-system XA feature would be an what changes to the 
proposal
>> would be required to enable it? When I have asked people who wanted
>> cross-system XA in the past what they wanted it for, I haven't really
>> gotten any answers that made sense. Every person really wanted 
something
>> that would be better solved by a transactional (or idempotent) write to
>> Kafka followed by an independent transactional (or idempotent) 
consumption
>> (which this proposal enables). For the use cases they described tying 
these
>> two things together had no advantage and many disadvantages.
>> 
>> I have one use case which would be accomplished by cross-system XA 
which is
>> allowing the producer to block on the synchronous processing of the 
message
>> by (all? some?) consumers. However I'm not convinced that cross-system 
XA
>> is the best solution to this problem, and I'm also not convinced this 
is an
>> important problem to solve. But maybe you have something in mind here.
>> 
>> -Jay
>> 
>> 
>> 
>>> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> 
wrote:
>>> 
>>> Hi,
>>> thanks, very interesting KIP ... I haven't fully digested it yet.
>>> 
>>> We have many users who choose not to use the Java client,  so I have
>>> concerns about the added complexity in developing the clients.
>>> A few questions.
>>> 
>>> 1 - is mixing transactional and non transactional messages on the 
*same
>>> topic-partition* really a requirement ?
>>> What use case does it satisfy?
>>> 
>>> 2 - I guess some clients may only be interested to implement the 
producer
>>> idempotency.
>>> It's not clear how they could be implemented without having to add the
>>> transaction capabilities.
>>> As others on this list have said, I too would like to see idempotency 
as a
>>> more basic feature, on top which txns can be built.
>>> 
>>> 3 - The KIP seems focused on a use case where consumption from a topic 
and
>>> subsequent production are part of the producer transaction.
>>> 
>>> It'd be great to see a way to extend the producer transaction to 
include
>>> additional transactional resources,
>>> so that the consumption from another topic just becomes a special case 
of
>>> a more general "distributed" txn.
>>> 
>>> Edo
>>> --------------------------------------------------
>>> Edoardo Comar
>>> IBM MessageHub
>>> ecomar@uk.ibm.com
>>> IBM UK Ltd, Hursley Park, SO21 2JN
>>> 
>>> IBM United Kingdom Limited Registered in England and Wales with number
>>> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. 
PO6
>> 3AU
>> 
>> 
>> 
>> From:   Guozhang Wang <wa...@gmail.com>
>> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
>> Date:   30/11/2016 22:20
>> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and 
Transactional
>> Messaging
>> 
>> 
>> 
>> Hi all,
>> 
>> I have just created KIP-98 to enhance Kafka with exactly once delivery
>> semantics:
>> 
>> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>>> *
>> 
>> This KIP adds a transactional messaging mechanism along with an 
idempotent
>> producer implementation to make sure that 1) duplicated messages sent 
from
>> the same identified producer can be detected on the broker side, and 2) 
a
>> group of messages sent within a transaction will atomically be either
>> reflected and fetchable to consumers or not as a whole.
>> 
>> The above wiki page provides a high-level view of the proposed changes 
as
>> well as summarized guarantees. Initial draft of the detailed
>> implementation
>> design is described in this Google doc:
>> 
>> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> 0wSw9ra8
>> 
>> 
>> We would love to hear your comments and suggestions.
>> 
>> Thanks,
>> 
>> -- Guozhang
>> 
>> 
>> 
>> Unless stated otherwise above:
>> IBM United Kingdom Limited - Registered in England and Wales with 
number
>> 741598.
>> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 
3AU
>> 
> The information contained in this email is strictly confidential and for 
the use of the addressee only, unless otherwise indicated. If you are not 
the intended recipient, please do not read, copy, use or disclose to 
others this message or any attachment. Please  also notify the sender by 
replying to this email or by telephone (+44(020 7896 0011) and then delete 
the email and any copies of it. Opinions, conclusion (etc) that do not 
relate to the official business of this company shall be understood as 
neither given  nor endorsed by it. IG is a trading name of IG Markets 
Limited (a company registered in England and Wales, company number 
04008957) and IG Index Limited (a company registered in England and Wales, 
company number 01190902). Registered address at Cannon Bridge  House, 25 
Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 
195355) and IG Index Limited (register number 114059) are authorised and 
regulated by the Financial Conduct Authority.



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number 
741598. 
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael André Pearce <mi...@me.com>.
Agreed, I think it is better to either support transactions properly and fully to the level or expectation normally associated with a transactional system. Or don't provide it at all.  

Having a half way house can be more dangerous.

As I said earlier the issue of message duplication can be handled entirely without transactions.

Sent from my iPhone

> On 9 Dec 2016, at 10:49, Andrew Schofield <an...@outlook.com> wrote:
> 
> I've been pondering this question of coordinating other resource managers with
> Kafka transactions for a while and I'm not convinced it's a good idea. My
> reservations come down to the guarantees that it would provide in failure
> scenarios.
> 
> I don't think KIP-98 gives proper ACID semantics in the presence of all
> failures. For a transaction which contains a mixture of publishes and offset
> updates, a bunch of topics are involved and it appears to me that an
> uncontrolled shutdown could result in some but not all of the lazy writes
> making it to disk.
> 
> Here are some of the failures that I'm worried about:
> 
> * A message is published to a topic which crashes the leader Kafka node, as
>  it's replicated across the cluster, it crashes all of the other Kafka nodes
>  (we've really had this - SEGV, our fault and we've fixed it, but it happened)
>  so this is a kind of rolling node crash in a cluster
> * Out of memory error in one or more Kafka nodes
> * Disk fills in one or more Kafka nodes
> * Uncontrolled power-off to all nodes in the cluster
> 
> Does KIP-98 guarantee atomicity for transactions in all of these cases?
> Unless all of the topics involved in a transaction are recovered to the
> same point in time, you can't consider a transaction to be properly atomic.
> If application code is designed expecting atomicity, there are going to be
> tears. Perhaps only when disaster strikes, but the risk is there.
> 
> I think KIP-98 is interesting, but I wouldn't equate what it provides
> with the transactions provided by resource managers with traditional transaction
> logging. It's not better or worse, just different. If you tried to migrate
> from a previous transactional system to Kafka transactions, I think you'd
> better have procedures for reconciliation with the other resource managers.
> Better still, don't build applications that are so fragile. The principle
> of dumb pipes and smart endpoints is good in my view.
> 
> If you're creating a global transaction containing two or more resource
> managers and using two-phase commit, it's very important that all of the
> resource managers maintain a coherent view of the sequence of events. If any
> part fails due to a software or hardware failure, once the system is
> recovered, nothing must be forgotten. If you look at how presume-abort
> works, you'll see how important this is.
> 
> Kafka doesn't really fit very nicely in this kind of environment because of
> the way that it writes lazily to disk. The theory is that you must avoid at all
> costs having an uncontrolled shutdown of an entire cluster because you'll lose
> a little data at the end of the logs. So, if you are coordinating Kafka and a
> relational database in a global transaction, it's theoretically possible that
> a crashed Kafka would be a little forgetful while a crashed database would not.
> The database would be an order of magnitude or more slower because of the way
> its recovery logs are handled, but it would not be forgetful in the same way.
> 
> You get exactly the same kind of worries when you implement some kind of
> asynchronous replication for disaster recovery, even if all of the resource
> managers force all of their log writes to disk eagerly. The replica at the DR
> site is slightly behind the primary site, so if you have to recover from an
> outage and switch to the DR site, it can be considered to be slightly forgetful
> about the last few moments before the outage. This is why a DR plan usually has
> some concept of compensation or reconciliation to make good any forgotten work.
> 
> In summary, I think Kafka would have to change in ways which would negate many
> of its good points in order to support XA transactions. It would be better to
> design applications to be resilient to message duplication and loss, rather
> than tightly coupling resource managers and ending up with something fragile.
> 
> Don't get me wrong. This is not an anti-Kafka rant. I just work with people
> used to traditional transactional systems, making use of Kafka for business
> applications, and it's important to understand the concepts on both sides
> and make sure your assumptions are valid.
> 
> Andrew Schofield
> IBM Watson and Cloud Platform
> 
> 
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: 09 December 2016 06:19
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
>>     
>> Usecase in IG:
>> 
>> Fund transfer between accounts. When we debit one account and fund another we must ensure the records to both occur > as an acid action, and as a single transaction.
>> 
>> Today we achieve this because we have jms, as such we can do the actions needed in an xa transaction across both the > accounts. To move this flow to Kafka we would need support of XA transaction.
>> 
>> 
>> 
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Michael Pearce <Mi...@ig.com>
>> Sent: Friday, December 9, 2016 6:09:06 AM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
>> 
>> Hi Jay,
>> 
>> For me having an XA transaction allows for ensuring ACID across my application.
>> 
>> I believe it is part of the JMS api, and obviously JMS still is in enterprise very widely adopted for Messaging transport , so > obviously to say it isn't widely used i think is ignoring a whole range of users. Like wise I believe frameworks like spring etc fully  support it more evidence of its wide adoption.
>> 
>> On this note personally we try to avoid transactions entirely in our flows for performance and simplicity. but we do alas unfortunately have one or two places we cannot ignore it.
>> 
>> Cheers
>> Mike
>> 
>> Sent using OWA for iPhone
>> ________________________________________
>> From: Jay Kreps <ja...@confluent.io>
>> Sent: Thursday, December 8, 2016 11:25:53 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
>> 
>> Hey Edoardo,
>> 
>> For (3) can you outline what you think the benefit and use cases for a more
>> general cross-system XA feature would be an what changes to the proposal
>> would be required to enable it? When I have asked people who wanted
>> cross-system XA in the past what they wanted it for, I haven't really
>> gotten any answers that made sense. Every person really wanted something
>> that would be better solved by a transactional (or idempotent) write to
>> Kafka followed by an independent transactional (or idempotent) consumption
>> (which this proposal enables). For the use cases they described tying these
>> two things together had no advantage and many disadvantages.
>> 
>> I have one use case which would be accomplished by cross-system XA which is
>> allowing the producer to block on the synchronous processing of the message
>> by (all? some?) consumers. However I'm not convinced that cross-system XA
>> is the best solution to this problem, and I'm also not convinced this is an
>> important problem to solve. But maybe you have something in mind here.
>> 
>> -Jay
>> 
>> 
>> 
>>> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
>>> 
>>> Hi,
>>> thanks, very interesting KIP ... I haven't fully digested it yet.
>>> 
>>> We have many users who choose not to use the Java client,  so I have
>>> concerns about the added complexity in developing the clients.
>>> A few questions.
>>> 
>>> 1 - is mixing transactional and non transactional messages on the *same
>>> topic-partition* really a requirement ?
>>> What use case does it satisfy?
>>> 
>>> 2 - I guess some clients may only be interested to implement the producer
>>> idempotency.
>>> It's not clear how they could be implemented without having to add the
>>> transaction capabilities.
>>> As others on this list have said, I too would like to see idempotency as a
>>> more basic feature, on top which txns can be built.
>>> 
>>> 3 - The KIP seems focused on a use case where consumption from a topic and
>>> subsequent production are part of the producer transaction.
>>> 
>>> It'd be great to see a way to extend the producer transaction to include
>>> additional transactional resources,
>>> so that the consumption from another topic just becomes a special case of
>>> a more general "distributed" txn.
>>> 
>>> Edo
>>> --------------------------------------------------
>>> Edoardo Comar
>>> IBM MessageHub
>>> ecomar@uk.ibm.com
>>> IBM UK Ltd, Hursley Park, SO21 2JN
>>> 
>>> IBM United Kingdom Limited Registered in England and Wales with number
>>> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
>> 3AU
>> 
>> 
>> 
>> From:   Guozhang Wang <wa...@gmail.com>
>> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
>> Date:   30/11/2016 22:20
>> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
>> Messaging
>> 
>> 
>> 
>> Hi all,
>> 
>> I have just created KIP-98 to enhance Kafka with exactly once delivery
>> semantics:
>> 
>> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>>> *
>> 
>> This KIP adds a transactional messaging mechanism along with an idempotent
>> producer implementation to make sure that 1) duplicated messages sent from
>> the same identified producer can be detected on the broker side, and 2) a
>> group of messages sent within a transaction will atomically be either
>> reflected and fetchable to consumers or not as a whole.
>> 
>> The above wiki page provides a high-level view of the proposed changes as
>> well as summarized guarantees. Initial draft of the detailed
>> implementation
>> design is described in this Google doc:
>> 
>> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> 0wSw9ra8
>> 
>> 
>> We would love to hear your comments and suggestions.
>> 
>> Thanks,
>> 
>> -- Guozhang
>> 
>> 
>> 
>> Unless stated otherwise above:
>> IBM United Kingdom Limited - Registered in England and Wales with number
>> 741598.
>> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>> 
> The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please  also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given  nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge  House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Andrew Schofield <an...@outlook.com>.
I've been pondering this question of coordinating other resource managers with
Kafka transactions for a while and I'm not convinced it's a good idea. My
reservations come down to the guarantees that it would provide in failure
scenarios.

I don't think KIP-98 gives proper ACID semantics in the presence of all
failures. For a transaction which contains a mixture of publishes and offset
updates, a bunch of topics are involved and it appears to me that an
uncontrolled shutdown could result in some but not all of the lazy writes
making it to disk.

Here are some of the failures that I'm worried about:

* A message is published to a topic which crashes the leader Kafka node, as
  it's replicated across the cluster, it crashes all of the other Kafka nodes
  (we've really had this - SEGV, our fault and we've fixed it, but it happened)
  so this is a kind of rolling node crash in a cluster
* Out of memory error in one or more Kafka nodes
* Disk fills in one or more Kafka nodes
* Uncontrolled power-off to all nodes in the cluster

Does KIP-98 guarantee atomicity for transactions in all of these cases?
Unless all of the topics involved in a transaction are recovered to the
same point in time, you can't consider a transaction to be properly atomic.
If application code is designed expecting atomicity, there are going to be
tears. Perhaps only when disaster strikes, but the risk is there.

I think KIP-98 is interesting, but I wouldn't equate what it provides
with the transactions provided by resource managers with traditional transaction
logging. It's not better or worse, just different. If you tried to migrate
from a previous transactional system to Kafka transactions, I think you'd
better have procedures for reconciliation with the other resource managers.
Better still, don't build applications that are so fragile. The principle
of dumb pipes and smart endpoints is good in my view.

If you're creating a global transaction containing two or more resource
managers and using two-phase commit, it's very important that all of the
resource managers maintain a coherent view of the sequence of events. If any
part fails due to a software or hardware failure, once the system is
recovered, nothing must be forgotten. If you look at how presume-abort
works, you'll see how important this is.

Kafka doesn't really fit very nicely in this kind of environment because of
the way that it writes lazily to disk. The theory is that you must avoid at all
costs having an uncontrolled shutdown of an entire cluster because you'll lose
a little data at the end of the logs. So, if you are coordinating Kafka and a
relational database in a global transaction, it's theoretically possible that
a crashed Kafka would be a little forgetful while a crashed database would not.
The database would be an order of magnitude or more slower because of the way
its recovery logs are handled, but it would not be forgetful in the same way.

You get exactly the same kind of worries when you implement some kind of
asynchronous replication for disaster recovery, even if all of the resource
managers force all of their log writes to disk eagerly. The replica at the DR
site is slightly behind the primary site, so if you have to recover from an
outage and switch to the DR site, it can be considered to be slightly forgetful
about the last few moments before the outage. This is why a DR plan usually has
some concept of compensation or reconciliation to make good any forgotten work.

In summary, I think Kafka would have to change in ways which would negate many
of its good points in order to support XA transactions. It would be better to
design applications to be resilient to message duplication and loss, rather
than tightly coupling resource managers and ending up with something fragile.

Don't get me wrong. This is not an anti-Kafka rant. I just work with people
used to traditional transactional systems, making use of Kafka for business
applications, and it's important to understand the concepts on both sides
and make sure your assumptions are valid.

Andrew Schofield
IBM Watson and Cloud Platform


> From: Michael Pearce <Mi...@ig.com>
> Sent: 09 December 2016 06:19
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
>     
> Usecase in IG:
> 
> Fund transfer between accounts. When we debit one account and fund another we must ensure the records to both occur > as an acid action, and as a single transaction.
> 
> Today we achieve this because we have jms, as such we can do the actions needed in an xa transaction across both the > accounts. To move this flow to Kafka we would need support of XA transaction.
> 
> 
> 
> Sent using OWA for iPhone
> ________________________________________
> From: Michael Pearce <Mi...@ig.com>
> Sent: Friday, December 9, 2016 6:09:06 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
> 
> Hi Jay,
> 
> For me having an XA transaction allows for ensuring ACID across my application.
> 
> I believe it is part of the JMS api, and obviously JMS still is in enterprise very widely adopted for Messaging transport , so > obviously to say it isn't widely used i think is ignoring a whole range of users. Like wise I believe frameworks like spring etc fully  support it more evidence of its wide adoption.
> 
> On this note personally we try to avoid transactions entirely in our flows for performance and simplicity. but we do alas unfortunately have one or two places we cannot ignore it.
> 
> Cheers
> Mike
> 
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Thursday, December 8, 2016 11:25:53 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
> 
> Hey Edoardo,
> 
> For (3) can you outline what you think the benefit and use cases for a more
> general cross-system XA feature would be an what changes to the proposal
> would be required to enable it? When I have asked people who wanted
> cross-system XA in the past what they wanted it for, I haven't really
> gotten any answers that made sense. Every person really wanted something
> that would be better solved by a transactional (or idempotent) write to
> Kafka followed by an independent transactional (or idempotent) consumption
> (which this proposal enables). For the use cases they described tying these
> two things together had no advantage and many disadvantages.
> 
> I have one use case which would be accomplished by cross-system XA which is
> allowing the producer to block on the synchronous processing of the message
> by (all? some?) consumers. However I'm not convinced that cross-system XA
> is the best solution to this problem, and I'm also not convinced this is an
> important problem to solve. But maybe you have something in mind here.
> 
> -Jay
> 
> 
> 
> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:
> 
> > Hi,
> > thanks, very interesting KIP ... I haven't fully digested it yet.
> >
> > We have many users who choose not to use the Java client,  so I have
> > concerns about the added complexity in developing the clients.
> > A few questions.
> >
> > 1 - is mixing transactional and non transactional messages on the *same
> > topic-partition* really a requirement ?
> > What use case does it satisfy?
> >
> > 2 - I guess some clients may only be interested to implement the producer
> > idempotency.
> > It's not clear how they could be implemented without having to add the
> > transaction capabilities.
> > As others on this list have said, I too would like to see idempotency as a
> > more basic feature, on top which txns can be built.
> >
> > 3 - The KIP seems focused on a use case where consumption from a topic and
> > subsequent production are part of the producer transaction.
> >
> > It'd be great to see a way to extend the producer transaction to include
> > additional transactional resources,
> > so that the consumption from another topic just becomes a special case of
> > a more general "distributed" txn.
> >
> > Edo
> > --------------------------------------------------
> > Edoardo Comar
> > IBM MessageHub
> > ecomar@uk.ibm.com
> > IBM UK Ltd, Hursley Park, SO21 2JN
> >
> > IBM United Kingdom Limited Registered in England and Wales with number
> > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
> 3AU
>
>
>
> From:   Guozhang Wang <wa...@gmail.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   30/11/2016 22:20
> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
>
>
> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed
> implementation
> design is described in this Google doc:
>
>  https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please  also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given  nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge  House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.
    

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Usecase in IG:

Fund transfer between accounts. When we debit one account and fund another we must ensure the records to both occur as an acid action, and as a single transaction.

Today we achieve this because we have jms, as such we can do the actions needed in an xa transaction across both the accounts. To move this flow to Kafka we would need support of XA transaction.



Sent using OWA for iPhone
________________________________________
From: Michael Pearce <Mi...@ig.com>
Sent: Friday, December 9, 2016 6:09:06 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hi Jay,

For me having an XA transaction allows for ensuring ACID across my application.

I believe it is part of the JMS api, and obviously JMS still is in enterprise very widely adopted for Messaging transport , so obviously to say it isn't widely used i think is ignoring a whole range of users. Like wise I believe frameworks like spring etc fully support it more evidence of its wide adoption.

On this note personally we try to avoid transactions entirely in our flows for performance and simplicity. but we do alas unfortunately have one or two places we cannot ignore it.

Cheers
Mike

Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Thursday, December 8, 2016 11:25:53 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Edoardo,

For (3) can you outline what you think the benefit and use cases for a more
general cross-system XA feature would be an what changes to the proposal
would be required to enable it? When I have asked people who wanted
cross-system XA in the past what they wanted it for, I haven't really
gotten any answers that made sense. Every person really wanted something
that would be better solved by a transactional (or idempotent) write to
Kafka followed by an independent transactional (or idempotent) consumption
(which this proposal enables). For the use cases they described tying these
two things together had no advantage and many disadvantages.

I have one use case which would be accomplished by cross-system XA which is
allowing the producer to block on the synchronous processing of the message
by (all? some?) consumers. However I'm not convinced that cross-system XA
is the best solution to this problem, and I'm also not convinced this is an
important problem to solve. But maybe you have something in mind here.

-Jay



On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:

> Hi,
> thanks, very interesting KIP ... I haven't fully digested it yet.
>
> We have many users who choose not to use the Java client,  so I have
> concerns about the added complexity in developing the clients.
> A few questions.
>
> 1 - is mixing transactional and non transactional messages on the *same
> topic-partition* really a requirement ?
> What use case does it satisfy?
>
> 2 - I guess some clients may only be interested to implement the producer
> idempotency.
> It's not clear how they could be implemented without having to add the
> transaction capabilities.
> As others on this list have said, I too would like to see idempotency as a
> more basic feature, on top which txns can be built.
>
> 3 - The KIP seems focused on a use case where consumption from a topic and
> subsequent production are part of the producer transaction.
>
> It'd be great to see a way to extend the producer transaction to include
> additional transactional resources,
> so that the consumption from another topic just becomes a special case of
> a more general "distributed" txn.
>
> Edo
> --------------------------------------------------
> Edoardo Comar
> IBM MessageHub
> ecomar@uk.ibm.com
> IBM UK Ltd, Hursley Park, SO21 2JN
>
> IBM United Kingdom Limited Registered in England and Wales with number
> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
> 3AU
>
>
>
> From:   Guozhang Wang <wa...@gmail.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   30/11/2016 22:20
> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
>
>
> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed
> implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Hi Jay,

For me having an XA transaction allows for ensuring ACID across my application.

I believe it is part of the JMS api, and obviously JMS still is in enterprise very widely adopted for Messaging transport , so obviously to say it isn't widely used i think is ignoring a whole range of users. Like wise I believe frameworks like spring etc fully support it more evidence of its wide adoption.

On this note personally we try to avoid transactions entirely in our flows for performance and simplicity. but we do alas unfortunately have one or two places we cannot ignore it.

Cheers
Mike

Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Thursday, December 8, 2016 11:25:53 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Edoardo,

For (3) can you outline what you think the benefit and use cases for a more
general cross-system XA feature would be an what changes to the proposal
would be required to enable it? When I have asked people who wanted
cross-system XA in the past what they wanted it for, I haven't really
gotten any answers that made sense. Every person really wanted something
that would be better solved by a transactional (or idempotent) write to
Kafka followed by an independent transactional (or idempotent) consumption
(which this proposal enables). For the use cases they described tying these
two things together had no advantage and many disadvantages.

I have one use case which would be accomplished by cross-system XA which is
allowing the producer to block on the synchronous processing of the message
by (all? some?) consumers. However I'm not convinced that cross-system XA
is the best solution to this problem, and I'm also not convinced this is an
important problem to solve. But maybe you have something in mind here.

-Jay



On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:

> Hi,
> thanks, very interesting KIP ... I haven't fully digested it yet.
>
> We have many users who choose not to use the Java client,  so I have
> concerns about the added complexity in developing the clients.
> A few questions.
>
> 1 - is mixing transactional and non transactional messages on the *same
> topic-partition* really a requirement ?
> What use case does it satisfy?
>
> 2 - I guess some clients may only be interested to implement the producer
> idempotency.
> It's not clear how they could be implemented without having to add the
> transaction capabilities.
> As others on this list have said, I too would like to see idempotency as a
> more basic feature, on top which txns can be built.
>
> 3 - The KIP seems focused on a use case where consumption from a topic and
> subsequent production are part of the producer transaction.
>
> It'd be great to see a way to extend the producer transaction to include
> additional transactional resources,
> so that the consumption from another topic just becomes a special case of
> a more general "distributed" txn.
>
> Edo
> --------------------------------------------------
> Edoardo Comar
> IBM MessageHub
> ecomar@uk.ibm.com
> IBM UK Ltd, Hursley Park, SO21 2JN
>
> IBM United Kingdom Limited Registered in England and Wales with number
> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
> 3AU
>
>
>
> From:   Guozhang Wang <wa...@gmail.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   30/11/2016 22:20
> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
>
>
> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed
> implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Edoardo,

For (3) can you outline what you think the benefit and use cases for a more
general cross-system XA feature would be an what changes to the proposal
would be required to enable it? When I have asked people who wanted
cross-system XA in the past what they wanted it for, I haven't really
gotten any answers that made sense. Every person really wanted something
that would be better solved by a transactional (or idempotent) write to
Kafka followed by an independent transactional (or idempotent) consumption
(which this proposal enables). For the use cases they described tying these
two things together had no advantage and many disadvantages.

I have one use case which would be accomplished by cross-system XA which is
allowing the producer to block on the synchronous processing of the message
by (all? some?) consumers. However I'm not convinced that cross-system XA
is the best solution to this problem, and I'm also not convinced this is an
important problem to solve. But maybe you have something in mind here.

-Jay



On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <EC...@uk.ibm.com> wrote:

> Hi,
> thanks, very interesting KIP ... I haven't fully digested it yet.
>
> We have many users who choose not to use the Java client,  so I have
> concerns about the added complexity in developing the clients.
> A few questions.
>
> 1 - is mixing transactional and non transactional messages on the *same
> topic-partition* really a requirement ?
> What use case does it satisfy?
>
> 2 - I guess some clients may only be interested to implement the producer
> idempotency.
> It's not clear how they could be implemented without having to add the
> transaction capabilities.
> As others on this list have said, I too would like to see idempotency as a
> more basic feature, on top which txns can be built.
>
> 3 - The KIP seems focused on a use case where consumption from a topic and
> subsequent production are part of the producer transaction.
>
> It'd be great to see a way to extend the producer transaction to include
> additional transactional resources,
> so that the consumption from another topic just becomes a special case of
> a more general "distributed" txn.
>
> Edo
> --------------------------------------------------
> Edoardo Comar
> IBM MessageHub
> ecomar@uk.ibm.com
> IBM UK Ltd, Hursley Park, SO21 2JN
>
> IBM United Kingdom Limited Registered in England and Wales with number
> 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
> 3AU
>
>
>
> From:   Guozhang Wang <wa...@gmail.com>
> To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date:   30/11/2016 22:20
> Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
>
>
> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed
> implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Edoardo Comar <EC...@uk.ibm.com>.
Hi, 
thanks, very interesting KIP ... I haven't fully digested it yet.

We have many users who choose not to use the Java client,  so I have 
concerns about the added complexity in developing the clients.
A few questions.

1 - is mixing transactional and non transactional messages on the *same 
topic-partition* really a requirement ?
What use case does it satisfy?

2 - I guess some clients may only be interested to implement the producer 
idempotency.
It's not clear how they could be implemented without having to add the 
transaction capabilities.
As others on this list have said, I too would like to see idempotency as a 
more basic feature, on top which txns can be built.

3 - The KIP seems focused on a use case where consumption from a topic and 
subsequent production are part of the producer transaction.

It'd be great to see a way to extend the producer transaction to include 
additional transactional resources, 
so that the consumption from another topic just becomes a special case of 
a more general "distributed" txn.

Edo
--------------------------------------------------
Edoardo Comar
IBM MessageHub
ecomar@uk.ibm.com
IBM UK Ltd, Hursley Park, SO21 2JN

IBM United Kingdom Limited Registered in England and Wales with number 
741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 
3AU



From:   Guozhang Wang <wa...@gmail.com>
To:     "dev@kafka.apache.org" <de...@kafka.apache.org>
Date:   30/11/2016 22:20
Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional 
Messaging



Hi all,

I have just created KIP-98 to enhance Kafka with exactly once delivery
semantics:

*https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>*

This KIP adds a transactional messaging mechanism along with an idempotent
producer implementation to make sure that 1) duplicated messages sent from
the same identified producer can be detected on the broker side, and 2) a
group of messages sent within a transaction will atomically be either
reflected and fetchable to consumers or not as a whole.

The above wiki page provides a high-level view of the proposed changes as
well as summarized guarantees. Initial draft of the detailed 
implementation
design is described in this Google doc:

https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
0wSw9ra8


We would love to hear your comments and suggestions.

Thanks,

-- Guozhang



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number 
741598. 
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Ben,

Those are both great questions. I will tackle the second one now, and
address the first one a bit later.

AppIds are prerequisite for using transactions, and must be consistent
across across application sessions. They are the mechanism by which
transaction recovery can occur across sessions, and assume that the
application has state which is persistent across sessions.

PIDs without AppIds are ephemeral and are valid for a single session. In
this context, they are only used to ensure idempotent message message
delivery.

Keeping them separate caters to those applications which don't care about
transactions but do care about idempotent message delivery guarantees,
which we assume will be the vast majority of the applications out there.

Further, from an API perspective, separating out AppId from PID means that
applications get idempotent message delivery for free once they upgrade
their kafka version on the client and server and switch to the new message
format: there are no additional code or config changes needed, since PID is
a completely internal concept.

Hope this clarifies the reason for separating these two things out.

Regards,
Apurva

On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:

> Thanks for this! I'm looking forward to going through the full proposal in
> detail soon; a few early questions:
>
> First: what happens when a consumer rebalances in the middle of a
> transaction? The full documentation suggests that such a transaction ought
> to be rejected:
>
> > [...] if a rebalance has happened and this consumer
> > instance becomes a zombie, even if this offset message is appended in the
> > offset topic, the transaction will be rejected later on when it tries to
> > commit the transaction via the EndTxnRequest.
>
> ...but it's unclear to me how we ensure that a transaction can't complete
> if a rebalance has happened. (It's quite possible I'm missing something
> obvious!)
>
> As a concrete example: suppose a process with PID 1 adds offsets for some
> partition to a transaction; a consumer rebalance happens that assigns the
> partition to a process with PID 2, which adds some offsets to its current
> transaction; both processes try and commit. Allowing both commits would
> cause the messages to be processed twice -- how is that avoided?
>
> Second: App IDs normally map to a single PID. It seems like one could do
> away with the PID concept entirely, and just use App IDs in most places
> that require a PID. This feels like it would be significantly simpler,
> though it does increase the message size. Are there other reasons why the
> App ID / PID split is necessary?
>
> On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Ben,

You are are right on both counts:

Writing apps to do consume-process-produce batching will be tricky to
program using this API directly. The expectation is that 99% of the users
would use the streams API to leverage this functionality, and that API will
take care of the details. This seems fair, since this pattern is at the
core of stream processing.

Using an internally generated PID is definitely more a performance than a
correctness thing: we could generate UUIDs in the producer if no AppId is
specified, and that would also work. However, as you may have seen
elsewhere in the thread, there are calls for the PID to be 4 bytes (vs. the
present 8 bytes). So 16 bytes will be really far out. While the cost of a
PID is amortized across the messages in a message set, we still want to
keep it as small as possible to reduce the overhead.

We are thinking about how to recover expired PIDs which would enable us to
keep it to just 4 bytes (enough to handle 4billion concurrently alive
producers). However, this will be very very tricky because a given PID
could produce to multiple topic partitions, and recovering a PID will
require _all_ instances of it across _all_ topic partitions to be expired.
This would be very hard to achieve without invasive things like reference
counting. Still we are searching for a more elegant and lightweight
solution to the problem, and will use 4 byte PID if we can find an
acceptable solution for PID recovery.

Thanks,
Apurva



On Tue, Dec 13, 2016 at 9:30 PM, Ben Kirwin <be...@kirw.in> wrote:

> Hi Apurva,
>
> Thanks for the detailed answers... and sorry for the late reply!
>
> It does sound like, if the input-partitions-to-app-id mapping never
> changes, the existing fencing mechanisms should prevent duplicates. Great!
> I'm a bit concerned the proposed API will be delicate to program against
> successfully -- even in the simple case, we need to create a new producer
> instance per input partition, and anything fancier is going to need its own
> implementation of the Streams/Samza-style 'task' idea -- but that may be
> fine for this sort of advanced feature.
>
> For the second question, I notice that Jason also elaborated on this
> downthread:
>
> > We also looked at removing the producer ID.
> > This was discussed somewhere above, but basically the idea is to store
> the
> > AppID in the message set header directly and avoid the mapping to
> producer
> > ID altogether. As long as batching isn't too bad, the impact on total
> size
> > may not be too bad, but we were ultimately more comfortable with a fixed
> > size ID.
>
> ...which suggests that the distinction is useful for performance, but not
> necessary for correctness, which makes good sense to me. (Would a 128-bid
> ID be a reasonable compromise? That's enough room for a UUID, or a
> reasonable hash of an arbitrary string, and has only a marginal increase on
> the message size.)
>
> On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Ben,
> >
> > Now, on to your first question of how deal with consumer rebalances. The
> > short answer is that the application needs to ensure that the the
> > assignment of input partitions to appId is consistent across rebalances.
> >
> > For Kafka streams, they already ensure that the mapping of input
> partitions
> > to task Id is invariant across rebalances by implementing a custom sticky
> > assignor. Other non-streams apps can trivially have one producer per
> input
> > partition and have the appId be the same as the partition number to
> achieve
> > the same effect.
> >
> > With this precondition in place, we can maintain transactions across
> > rebalances.
> >
> > Hope this answers your question.
> >
> > Thanks,
> > Apurva
> >
> > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> >
> > > Thanks for this! I'm looking forward to going through the full proposal
> > in
> > > detail soon; a few early questions:
> > >
> > > First: what happens when a consumer rebalances in the middle of a
> > > transaction? The full documentation suggests that such a transaction
> > ought
> > > to be rejected:
> > >
> > > > [...] if a rebalance has happened and this consumer
> > > > instance becomes a zombie, even if this offset message is appended in
> > the
> > > > offset topic, the transaction will be rejected later on when it tries
> > to
> > > > commit the transaction via the EndTxnRequest.
> > >
> > > ...but it's unclear to me how we ensure that a transaction can't
> complete
> > > if a rebalance has happened. (It's quite possible I'm missing something
> > > obvious!)
> > >
> > > As a concrete example: suppose a process with PID 1 adds offsets for
> some
> > > partition to a transaction; a consumer rebalance happens that assigns
> the
> > > partition to a process with PID 2, which adds some offsets to its
> current
> > > transaction; both processes try and commit. Allowing both commits would
> > > cause the messages to be processed twice -- how is that avoided?
> > >
> > > Second: App IDs normally map to a single PID. It seems like one could
> do
> > > away with the PID concept entirely, and just use App IDs in most places
> > > that require a PID. This feels like it would be significantly simpler,
> > > though it does increase the message size. Are there other reasons why
> the
> > > App ID / PID split is necessary?
> > >
> > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
a few more points:

1. "we can make that efficient by storing only the 'start offsets' of
transactions and then replaying the log once we hit the corresponding
commit/abort markers"

this is efficient in terms of memory consumption on the consumer. it is
definitely not efficient when i consider what this means in a large
datacenter with lots and lots of consumers - you've double reads on the
network (and broker disks, potentially). your other alternative is to spool
to client disk (again not efficient across the whole datacenter). this is
why i think more of TX support belongs on the broker.

2. while a commit is in progress (partition leader is still appending tx
msgs onto the partition and replicating to followers) any new msgs sent to
the partition (or committed from another TX) will be queued behind them.
there is a downside to this - watermarks will not advance until a whole TX
is appended. on the other hand, actual writes continue at full possible
speed, so this is not a real performance degradation, just a potential
latency spike (head of queue blocking on appending TXs). logically any
consumer that "opts in" under the current proposal already suffers it.

On Fri, Dec 16, 2016 at 9:24 AM, radai <ra...@gmail.com> wrote:

> Hi Apurva,
>
> here's an outline of what I had in mind:
>
> 1. TXs in progress are written "sideways" on the partition leader. this
> can be (in order of increasing reliability and decreasing performance):
>      1.1 an im-mem buffer on the partition leader (TX lost on broker
> crash, memory limits need to be in place, but disk and network efficient.
>      1.2 a file on the partition leader (still not replicated)
>      1.3 a full blown "ad-hoc partition" - so file on the leader
> replicated to followers, open TX will survive a crash.
> 2. as long as a TX is in progress its not written to the "real"/"target"
> partition at all. this means nothing downstream sees it and that any
> offsets returned to client at this point are "virtual".
> 3. when a client commits, the partition leader will append all the msgs
> accumulated in the TX store (#1 above) to the real partition as if theyve
> just been written:
>     3.1 this is the point where msgs get "real" offsets
>     3.2 since TX size may be larger than what followers pull in a single
> go, and any broker can still crash at this point, the appended TX msgs can
> only be "seen" by clients once everything has been distributed to
> followers. this would mean some marker on the 1st msg appended indicating
> how many more are expected (known at this point) so that watermarks will
> not advance until everything is appended and new leaders can resume/abort
> on crash.
>     3.3 only after commit step in #3.2 above has completed can watermarks
> advance and msgs become visible. this is also the point where the client
> can consider the TX committed.
> 4. depending on the choice made in #1, there are several alternatives
> w.r.t complexity and disk efficiency:
>     4.1 append contents of TX file to partition segment file. this means
> that any TX msg incurs 2 disk writes and 1 disk read (written to TX file,
> read from TX file, written to partition segment).
>     4.2 make the partition log a new segment file. this is efficient in
> terms of disk writes (msgs were already written to disk, just reuse the
> file and make it a segment) but implemented naively would lead to
> fragmentation. could be consolidated in the background (similar in concept
> to compaction) to amortize the cost.
>     4.3 offer lower resiliency guarantees for ongoing TXs compared to
> normal msg delivery (this is common in DBs - if your sql connection drops,
> your TX is gone. its accepted by users), spool them in memory with
> disk-overflow and so given proper sizing incur no extra disk costs at all
> (overflow to disk when out of memory).
>
> this design puts more burden server-side but presents a cleaner view
> downstream and allows for simpler, smaller, more efficient clients. given
> the kafka ecosystem includes one server implementation and multiple client
> implementations across various languages owned and maintained by different
> people i think this is a better solution in terms of overall complexity,
> overall network utilization (io/disk across the whole datacenter, not just
> the broker) and in terms of eco system adoption (less burden on client
> writers, as we want to make it easier to have lots of client impls).
>
>
> On Thu, Dec 15, 2016 at 11:59 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
>
>> Hi Radai,
>>
>> Thanks for your email. You raise some good points.
>>
>> Your larger point centers on your --correct!-- observation that persisting
>> messages belonging to aborted transactions causes complications for
>> consumers.
>>
>> You are right in observing that the current proposal does require
>> consumers
>> to buffer, though we can make that efficient by storing only the 'start
>> offsets' of transactions and then replaying the log once we hit the
>> corresponding commit/abort markers. And yes, it does mean that in some
>> consumer modes, we will deliver messages belonging to aborted
>> transactions.
>>
>> As such, I would really like to understand the details of your alternative
>> to see if we can address these issues efficiently.
>>
>> When you say that you will write to the main partition from the 'sideways'
>> transaction partition atomically, do you literally mean that a sequence of
>> individual messages from the 'sideways' partition will be written to the
>> main partition atomically?
>>
>> If the 'sideways partition' is written to disk, you have two copies of
>> data. I would like to know how you would ensure that you do not create
>> duplicates in the event of crashes. And if it is in-mem, you are now
>> buffering anyway, on all replicas.
>>
>> And, in the absence of any markers in the log, how would you ensure that
>> the batch of messages in a committed transaction is delivered to the
>> consumer all together? In the current proposal, the return of a 'poll'
>> would include all the messages of a transaction, or none of them.
>>
>> * *
>>
>> Re: 3::  you ask about the migration plan. This has been detailed in the
>> design
>> doc
>> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7
>> CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.3s6zgsaq9cep>.
>> In particular, the recommended sequence is to bump all your clients before
>> bumping up the message format. Further, since transactions are entirely
>> opt-in, it is not unreasonable to ensure that the ecosystem is ready for
>> them before starting to use them.
>>
>> * *
>>
>> Thanks,
>> Apurva
>>
>>
>> On Thu, Dec 15, 2016 at 3:07 PM, radai <ra...@gmail.com>
>> wrote:
>>
>> > some clarifications on my alternative proposal:
>> >
>> > TX msgs are written "sideways" to a transaction (ad-hoc) partition. this
>> > partition can be replicated to followers, or can be an in-mem buffer -
>> > depends on the resilience guarantees you want to provide for TXs in
>> case of
>> > broker crash.
>> > on "commit" the partition leader broker (being the single point of
>> > synchronization for the partition anyway) can atomically append the
>> > contents of this TX "partition" onto the real target partition. this is
>> the
>> > point where the msgs get "real" offsets. there's some trickiness around
>> how
>> > not to expose these offsets to any consumers until everything's been
>> > replicated to followers, but we believe its possible.
>> >
>> >
>> >
>> > On Thu, Dec 15, 2016 at 2:31 PM, radai <ra...@gmail.com>
>> wrote:
>> >
>> > > I can see several issues with the current proposal.
>> > >
>> > > messages, even if sent under a TX, are delivered directly to their
>> > > destination partitions, downstream consumers need to be TX-aware. they
>> > can
>> > > either:
>> > >    1. be oblivious to TXs. that means they will deliver "garbage" -
>> msgs
>> > > sent during eventually-aborted TXs.
>> > >    2. "opt-in" - they would have to not deliver _ANY_ msg until they
>> know
>> > > the fate of all outstanding overlapping TXs - if i see msg A1 (in a
>> TX),
>> > > followed by B, which is not under any TX, i cannot deliver B until i
>> know
>> > > if A1 was committed or not (or I violate ordering). this would require
>> > some
>> > > sort of buffering on consumers. with a naive buffering impl i could
>> DOS
>> > > everyone on a topic - just start a TX on a very busy topic and keep it
>> > open
>> > > as long as I can ....
>> > >    3. explode if youre an old consumer that sees a control msg (whats
>> > your
>> > > migration plan?)
>> > >    4. cross-cluster replication mechanisms either replicate the
>> garbage
>> > or
>> > > need to clean it up. there are >1 such different mechanism (almost one
>> > per
>> > > company really :-) ) so lots of adjustments.
>> > >
>> > > I think the end result could be better if ongoing TXs are treated as
>> > > logically separate topic partitions, and only atomically appended onto
>> > the
>> > > target partitions on commit (meaning they are written to separate
>> journal
>> > > file(s) on the broker).
>> > >
>> > > such a design would present a "clean" view to any downstream
>> consumers -
>> > > anything not committed wont even show up. old consumers wont need to
>> know
>> > > about control msgs, no issues with unbounded msg buffering, generally
>> > > cleaner overall?
>> > >
>> > > there would need to be adjustments made to watermark and follower
>> fetch
>> > > logic but some of us here have discussed this over lunch and we think
>> its
>> > > doable.
>> > >
>> > >
>> > > On Thu, Dec 15, 2016 at 1:08 AM, Rajini Sivaram <rs...@pivotal.io>
>> > > wrote:
>> > >
>> > >> Hi Apurva,
>> > >>
>> > >> Thank you, makes sense.
>> > >>
>> > >> Rajini
>> > >>
>> > >> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io>
>> > >> wrote:
>> > >>
>> > >> > Hi Rajini,
>> > >> >
>> > >> > I think my original response to your point 15 was not accurate. The
>> > >> regular
>> > >> > definition of durability is that data once committed would never be
>> > >> lost.
>> > >> > So it is not enough for only the control messages to be flushed
>> before
>> > >> > being acknowledged -- all the messages (and offset commits) which
>> are
>> > >> part
>> > >> > of the transaction would need to be flushed before being
>> acknowledged
>> > as
>> > >> > well.
>> > >> >
>> > >> > Otherwise, it is possible that if all replicas of a topic partition
>> > >> crash
>> > >> > before the transactional messages are flushed, those messages will
>> be
>> > >> lost
>> > >> > even if the commit marker exists in the log. In this case, the
>> > >> transaction
>> > >> > would be 'committed' with incomplete data.
>> > >> >
>> > >> > Right now, there isn't any config which will ensure that the flush
>> to
>> > >> disk
>> > >> > happens before the acknowledgement. We could add it in the future,
>> and
>> > >> get
>> > >> > durability guarantees for kafka transactions.
>> > >> >
>> > >> > I hope this clarifies the situation. The present KIP does not
>> intend
>> > to
>> > >> add
>> > >> > the aforementioned config, so even the control messages are
>> > susceptible
>> > >> to
>> > >> > being lost if there is a simultaneous crash across all replicas. So
>> > >> > transactions are only as durable as existing Kafka messages. We
>> don't
>> > >> > strengthen any durability guarantees as part of this KIP.
>> > >> >
>> > >> > Thanks,
>> > >> > Apurva
>> > >> >
>> > >> >
>> > >> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
>> rsivaram@pivotal.io>
>> > >> > wrote:
>> > >> >
>> > >> > > Hi Apurva,
>> > >> > >
>> > >> > > Thank you for the answers. Just one follow-on.
>> > >> > >
>> > >> > > 15. Let me rephrase my original question. If all control messages
>> > >> > (messages
>> > >> > > to transaction logs and markers on user logs) were acknowledged
>> only
>> > >> > after
>> > >> > > flushing the log segment, will transactions become durable in the
>> > >> > > traditional sense (i.e. not restricted to min.insync.replicas
>> > >> failures) ?
>> > >> > > This is not a suggestion to update the KIP. It seems to me that
>> the
>> > >> > design
>> > >> > > enables full durability if required in the future with a rather
>> > >> > > non-intrusive change. I just wanted to make sure I haven't missed
>> > >> > anything
>> > >> > > fundamental that prevents Kafka from doing this.
>> > >> > >
>> > >> > >
>> > >> > >
>> > >> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
>> > >> > >
>> > >> > > > Hi Apurva,
>> > >> > > >
>> > >> > > > Thanks for the detailed answers... and sorry for the late
>> reply!
>> > >> > > >
>> > >> > > > It does sound like, if the input-partitions-to-app-id mapping
>> > never
>> > >> > > > changes, the existing fencing mechanisms should prevent
>> > duplicates.
>> > >> > > Great!
>> > >> > > > I'm a bit concerned the proposed API will be delicate to
>> program
>> > >> > against
>> > >> > > > successfully -- even in the simple case, we need to create a
>> new
>> > >> > producer
>> > >> > > > instance per input partition, and anything fancier is going to
>> > need
>> > >> its
>> > >> > > own
>> > >> > > > implementation of the Streams/Samza-style 'task' idea -- but
>> that
>> > >> may
>> > >> > be
>> > >> > > > fine for this sort of advanced feature.
>> > >> > > >
>> > >> > > > For the second question, I notice that Jason also elaborated on
>> > this
>> > >> > > > downthread:
>> > >> > > >
>> > >> > > > > We also looked at removing the producer ID.
>> > >> > > > > This was discussed somewhere above, but basically the idea
>> is to
>> > >> > store
>> > >> > > > the
>> > >> > > > > AppID in the message set header directly and avoid the
>> mapping
>> > to
>> > >> > > > producer
>> > >> > > > > ID altogether. As long as batching isn't too bad, the impact
>> on
>> > >> total
>> > >> > > > size
>> > >> > > > > may not be too bad, but we were ultimately more comfortable
>> > with a
>> > >> > > fixed
>> > >> > > > > size ID.
>> > >> > > >
>> > >> > > > ...which suggests that the distinction is useful for
>> performance,
>> > >> but
>> > >> > not
>> > >> > > > necessary for correctness, which makes good sense to me.
>> (Would a
>> > >> > 128-bid
>> > >> > > > ID be a reasonable compromise? That's enough room for a UUID,
>> or a
>> > >> > > > reasonable hash of an arbitrary string, and has only a marginal
>> > >> > increase
>> > >> > > on
>> > >> > > > the message size.)
>> > >> > > >
>> > >> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
>> > apurva@confluent.io>
>> > >> > > wrote:
>> > >> > > >
>> > >> > > > > Hi Ben,
>> > >> > > > >
>> > >> > > > > Now, on to your first question of how deal with consumer
>> > >> rebalances.
>> > >> > > The
>> > >> > > > > short answer is that the application needs to ensure that the
>> > the
>> > >> > > > > assignment of input partitions to appId is consistent across
>> > >> > > rebalances.
>> > >> > > > >
>> > >> > > > > For Kafka streams, they already ensure that the mapping of
>> input
>> > >> > > > partitions
>> > >> > > > > to task Id is invariant across rebalances by implementing a
>> > custom
>> > >> > > sticky
>> > >> > > > > assignor. Other non-streams apps can trivially have one
>> producer
>> > >> per
>> > >> > > > input
>> > >> > > > > partition and have the appId be the same as the partition
>> number
>> > >> to
>> > >> > > > achieve
>> > >> > > > > the same effect.
>> > >> > > > >
>> > >> > > > > With this precondition in place, we can maintain transactions
>> > >> across
>> > >> > > > > rebalances.
>> > >> > > > >
>> > >> > > > > Hope this answers your question.
>> > >> > > > >
>> > >> > > > > Thanks,
>> > >> > > > > Apurva
>> > >> > > > >
>> > >> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
>> wrote:
>> > >> > > > >
>> > >> > > > > > Thanks for this! I'm looking forward to going through the
>> full
>> > >> > > proposal
>> > >> > > > > in
>> > >> > > > > > detail soon; a few early questions:
>> > >> > > > > >
>> > >> > > > > > First: what happens when a consumer rebalances in the
>> middle
>> > of
>> > >> a
>> > >> > > > > > transaction? The full documentation suggests that such a
>> > >> > transaction
>> > >> > > > > ought
>> > >> > > > > > to be rejected:
>> > >> > > > > >
>> > >> > > > > > > [...] if a rebalance has happened and this consumer
>> > >> > > > > > > instance becomes a zombie, even if this offset message is
>> > >> > appended
>> > >> > > in
>> > >> > > > > the
>> > >> > > > > > > offset topic, the transaction will be rejected later on
>> when
>> > >> it
>> > >> > > tries
>> > >> > > > > to
>> > >> > > > > > > commit the transaction via the EndTxnRequest.
>> > >> > > > > >
>> > >> > > > > > ...but it's unclear to me how we ensure that a transaction
>> > can't
>> > >> > > > complete
>> > >> > > > > > if a rebalance has happened. (It's quite possible I'm
>> missing
>> > >> > > something
>> > >> > > > > > obvious!)
>> > >> > > > > >
>> > >> > > > > > As a concrete example: suppose a process with PID 1 adds
>> > offsets
>> > >> > for
>> > >> > > > some
>> > >> > > > > > partition to a transaction; a consumer rebalance happens
>> that
>> > >> > assigns
>> > >> > > > the
>> > >> > > > > > partition to a process with PID 2, which adds some offsets
>> to
>> > >> its
>> > >> > > > current
>> > >> > > > > > transaction; both processes try and commit. Allowing both
>> > >> commits
>> > >> > > would
>> > >> > > > > > cause the messages to be processed twice -- how is that
>> > avoided?
>> > >> > > > > >
>> > >> > > > > > Second: App IDs normally map to a single PID. It seems like
>> > one
>> > >> > could
>> > >> > > > do
>> > >> > > > > > away with the PID concept entirely, and just use App IDs in
>> > most
>> > >> > > places
>> > >> > > > > > that require a PID. This feels like it would be
>> significantly
>> > >> > > simpler,
>> > >> > > > > > though it does increase the message size. Are there other
>> > >> reasons
>> > >> > why
>> > >> > > > the
>> > >> > > > > > App ID / PID split is necessary?
>> > >> > > > > >
>> > >> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
>> > >> wangguoz@gmail.com
>> > >> > >
>> > >> > > > > wrote:
>> > >> > > > > >
>> > >> > > > > > > Hi all,
>> > >> > > > > > >
>> > >> > > > > > > I have just created KIP-98 to enhance Kafka with exactly
>> > once
>> > >> > > > delivery
>> > >> > > > > > > semantics:
>> > >> > > > > > >
>> > >> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > >> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>> > >> > > > > > >
>> > >> > > > > > > This KIP adds a transactional messaging mechanism along
>> with
>> > >> an
>> > >> > > > > > idempotent
>> > >> > > > > > > producer implementation to make sure that 1) duplicated
>> > >> messages
>> > >> > > sent
>> > >> > > > > > from
>> > >> > > > > > > the same identified producer can be detected on the
>> broker
>> > >> side,
>> > >> > > and
>> > >> > > > > 2) a
>> > >> > > > > > > group of messages sent within a transaction will
>> atomically
>> > be
>> > >> > > either
>> > >> > > > > > > reflected and fetchable to consumers or not as a whole.
>> > >> > > > > > >
>> > >> > > > > > > The above wiki page provides a high-level view of the
>> > proposed
>> > >> > > > changes
>> > >> > > > > as
>> > >> > > > > > > well as summarized guarantees. Initial draft of the
>> detailed
>> > >> > > > > > implementation
>> > >> > > > > > > design is described in this Google doc:
>> > >> > > > > > >
>> > >> > > > > > > https://docs.google.com/document/d/11Jqy_
>> > >> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> > >> > > > > > > 0wSw9ra8
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > > > We would love to hear your comments and suggestions.
>> > >> > > > > > >
>> > >> > > > > > > Thanks,
>> > >> > > > > > >
>> > >> > > > > > > -- Guozhang
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
Hi Apurva,

here's an outline of what I had in mind:

1. TXs in progress are written "sideways" on the partition leader. this can
be (in order of increasing reliability and decreasing performance):
     1.1 an im-mem buffer on the partition leader (TX lost on broker crash,
memory limits need to be in place, but disk and network efficient.
     1.2 a file on the partition leader (still not replicated)
     1.3 a full blown "ad-hoc partition" - so file on the leader replicated
to followers, open TX will survive a crash.
2. as long as a TX is in progress its not written to the "real"/"target"
partition at all. this means nothing downstream sees it and that any
offsets returned to client at this point are "virtual".
3. when a client commits, the partition leader will append all the msgs
accumulated in the TX store (#1 above) to the real partition as if theyve
just been written:
    3.1 this is the point where msgs get "real" offsets
    3.2 since TX size may be larger than what followers pull in a single
go, and any broker can still crash at this point, the appended TX msgs can
only be "seen" by clients once everything has been distributed to
followers. this would mean some marker on the 1st msg appended indicating
how many more are expected (known at this point) so that watermarks will
not advance until everything is appended and new leaders can resume/abort
on crash.
    3.3 only after commit step in #3.2 above has completed can watermarks
advance and msgs become visible. this is also the point where the client
can consider the TX committed.
4. depending on the choice made in #1, there are several alternatives w.r.t
complexity and disk efficiency:
    4.1 append contents of TX file to partition segment file. this means
that any TX msg incurs 2 disk writes and 1 disk read (written to TX file,
read from TX file, written to partition segment).
    4.2 make the partition log a new segment file. this is efficient in
terms of disk writes (msgs were already written to disk, just reuse the
file and make it a segment) but implemented naively would lead to
fragmentation. could be consolidated in the background (similar in concept
to compaction) to amortize the cost.
    4.3 offer lower resiliency guarantees for ongoing TXs compared to
normal msg delivery (this is common in DBs - if your sql connection drops,
your TX is gone. its accepted by users), spool them in memory with
disk-overflow and so given proper sizing incur no extra disk costs at all
(overflow to disk when out of memory).

this design puts more burden server-side but presents a cleaner view
downstream and allows for simpler, smaller, more efficient clients. given
the kafka ecosystem includes one server implementation and multiple client
implementations across various languages owned and maintained by different
people i think this is a better solution in terms of overall complexity,
overall network utilization (io/disk across the whole datacenter, not just
the broker) and in terms of eco system adoption (less burden on client
writers, as we want to make it easier to have lots of client impls).


On Thu, Dec 15, 2016 at 11:59 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Radai,
>
> Thanks for your email. You raise some good points.
>
> Your larger point centers on your --correct!-- observation that persisting
> messages belonging to aborted transactions causes complications for
> consumers.
>
> You are right in observing that the current proposal does require consumers
> to buffer, though we can make that efficient by storing only the 'start
> offsets' of transactions and then replaying the log once we hit the
> corresponding commit/abort markers. And yes, it does mean that in some
> consumer modes, we will deliver messages belonging to aborted transactions.
>
> As such, I would really like to understand the details of your alternative
> to see if we can address these issues efficiently.
>
> When you say that you will write to the main partition from the 'sideways'
> transaction partition atomically, do you literally mean that a sequence of
> individual messages from the 'sideways' partition will be written to the
> main partition atomically?
>
> If the 'sideways partition' is written to disk, you have two copies of
> data. I would like to know how you would ensure that you do not create
> duplicates in the event of crashes. And if it is in-mem, you are now
> buffering anyway, on all replicas.
>
> And, in the absence of any markers in the log, how would you ensure that
> the batch of messages in a committed transaction is delivered to the
> consumer all together? In the current proposal, the return of a 'poll'
> would include all the messages of a transaction, or none of them.
>
> * *
>
> Re: 3::  you ask about the migration plan. This has been detailed in the
> design
> doc
> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#bookmark=id.3s6zgsaq9cep>.
> In particular, the recommended sequence is to bump all your clients before
> bumping up the message format. Further, since transactions are entirely
> opt-in, it is not unreasonable to ensure that the ecosystem is ready for
> them before starting to use them.
>
> * *
>
> Thanks,
> Apurva
>
>
> On Thu, Dec 15, 2016 at 3:07 PM, radai <ra...@gmail.com> wrote:
>
> > some clarifications on my alternative proposal:
> >
> > TX msgs are written "sideways" to a transaction (ad-hoc) partition. this
> > partition can be replicated to followers, or can be an in-mem buffer -
> > depends on the resilience guarantees you want to provide for TXs in case
> of
> > broker crash.
> > on "commit" the partition leader broker (being the single point of
> > synchronization for the partition anyway) can atomically append the
> > contents of this TX "partition" onto the real target partition. this is
> the
> > point where the msgs get "real" offsets. there's some trickiness around
> how
> > not to expose these offsets to any consumers until everything's been
> > replicated to followers, but we believe its possible.
> >
> >
> >
> > On Thu, Dec 15, 2016 at 2:31 PM, radai <ra...@gmail.com>
> wrote:
> >
> > > I can see several issues with the current proposal.
> > >
> > > messages, even if sent under a TX, are delivered directly to their
> > > destination partitions, downstream consumers need to be TX-aware. they
> > can
> > > either:
> > >    1. be oblivious to TXs. that means they will deliver "garbage" -
> msgs
> > > sent during eventually-aborted TXs.
> > >    2. "opt-in" - they would have to not deliver _ANY_ msg until they
> know
> > > the fate of all outstanding overlapping TXs - if i see msg A1 (in a
> TX),
> > > followed by B, which is not under any TX, i cannot deliver B until i
> know
> > > if A1 was committed or not (or I violate ordering). this would require
> > some
> > > sort of buffering on consumers. with a naive buffering impl i could DOS
> > > everyone on a topic - just start a TX on a very busy topic and keep it
> > open
> > > as long as I can ....
> > >    3. explode if youre an old consumer that sees a control msg (whats
> > your
> > > migration plan?)
> > >    4. cross-cluster replication mechanisms either replicate the garbage
> > or
> > > need to clean it up. there are >1 such different mechanism (almost one
> > per
> > > company really :-) ) so lots of adjustments.
> > >
> > > I think the end result could be better if ongoing TXs are treated as
> > > logically separate topic partitions, and only atomically appended onto
> > the
> > > target partitions on commit (meaning they are written to separate
> journal
> > > file(s) on the broker).
> > >
> > > such a design would present a "clean" view to any downstream consumers
> -
> > > anything not committed wont even show up. old consumers wont need to
> know
> > > about control msgs, no issues with unbounded msg buffering, generally
> > > cleaner overall?
> > >
> > > there would need to be adjustments made to watermark and follower fetch
> > > logic but some of us here have discussed this over lunch and we think
> its
> > > doable.
> > >
> > >
> > > On Thu, Dec 15, 2016 at 1:08 AM, Rajini Sivaram <rs...@pivotal.io>
> > > wrote:
> > >
> > >> Hi Apurva,
> > >>
> > >> Thank you, makes sense.
> > >>
> > >> Rajini
> > >>
> > >> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io>
> > >> wrote:
> > >>
> > >> > Hi Rajini,
> > >> >
> > >> > I think my original response to your point 15 was not accurate. The
> > >> regular
> > >> > definition of durability is that data once committed would never be
> > >> lost.
> > >> > So it is not enough for only the control messages to be flushed
> before
> > >> > being acknowledged -- all the messages (and offset commits) which
> are
> > >> part
> > >> > of the transaction would need to be flushed before being
> acknowledged
> > as
> > >> > well.
> > >> >
> > >> > Otherwise, it is possible that if all replicas of a topic partition
> > >> crash
> > >> > before the transactional messages are flushed, those messages will
> be
> > >> lost
> > >> > even if the commit marker exists in the log. In this case, the
> > >> transaction
> > >> > would be 'committed' with incomplete data.
> > >> >
> > >> > Right now, there isn't any config which will ensure that the flush
> to
> > >> disk
> > >> > happens before the acknowledgement. We could add it in the future,
> and
> > >> get
> > >> > durability guarantees for kafka transactions.
> > >> >
> > >> > I hope this clarifies the situation. The present KIP does not intend
> > to
> > >> add
> > >> > the aforementioned config, so even the control messages are
> > susceptible
> > >> to
> > >> > being lost if there is a simultaneous crash across all replicas. So
> > >> > transactions are only as durable as existing Kafka messages. We
> don't
> > >> > strengthen any durability guarantees as part of this KIP.
> > >> >
> > >> > Thanks,
> > >> > Apurva
> > >> >
> > >> >
> > >> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> rsivaram@pivotal.io>
> > >> > wrote:
> > >> >
> > >> > > Hi Apurva,
> > >> > >
> > >> > > Thank you for the answers. Just one follow-on.
> > >> > >
> > >> > > 15. Let me rephrase my original question. If all control messages
> > >> > (messages
> > >> > > to transaction logs and markers on user logs) were acknowledged
> only
> > >> > after
> > >> > > flushing the log segment, will transactions become durable in the
> > >> > > traditional sense (i.e. not restricted to min.insync.replicas
> > >> failures) ?
> > >> > > This is not a suggestion to update the KIP. It seems to me that
> the
> > >> > design
> > >> > > enables full durability if required in the future with a rather
> > >> > > non-intrusive change. I just wanted to make sure I haven't missed
> > >> > anything
> > >> > > fundamental that prevents Kafka from doing this.
> > >> > >
> > >> > >
> > >> > >
> > >> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > >> > >
> > >> > > > Hi Apurva,
> > >> > > >
> > >> > > > Thanks for the detailed answers... and sorry for the late reply!
> > >> > > >
> > >> > > > It does sound like, if the input-partitions-to-app-id mapping
> > never
> > >> > > > changes, the existing fencing mechanisms should prevent
> > duplicates.
> > >> > > Great!
> > >> > > > I'm a bit concerned the proposed API will be delicate to program
> > >> > against
> > >> > > > successfully -- even in the simple case, we need to create a new
> > >> > producer
> > >> > > > instance per input partition, and anything fancier is going to
> > need
> > >> its
> > >> > > own
> > >> > > > implementation of the Streams/Samza-style 'task' idea -- but
> that
> > >> may
> > >> > be
> > >> > > > fine for this sort of advanced feature.
> > >> > > >
> > >> > > > For the second question, I notice that Jason also elaborated on
> > this
> > >> > > > downthread:
> > >> > > >
> > >> > > > > We also looked at removing the producer ID.
> > >> > > > > This was discussed somewhere above, but basically the idea is
> to
> > >> > store
> > >> > > > the
> > >> > > > > AppID in the message set header directly and avoid the mapping
> > to
> > >> > > > producer
> > >> > > > > ID altogether. As long as batching isn't too bad, the impact
> on
> > >> total
> > >> > > > size
> > >> > > > > may not be too bad, but we were ultimately more comfortable
> > with a
> > >> > > fixed
> > >> > > > > size ID.
> > >> > > >
> > >> > > > ...which suggests that the distinction is useful for
> performance,
> > >> but
> > >> > not
> > >> > > > necessary for correctness, which makes good sense to me. (Would
> a
> > >> > 128-bid
> > >> > > > ID be a reasonable compromise? That's enough room for a UUID,
> or a
> > >> > > > reasonable hash of an arbitrary string, and has only a marginal
> > >> > increase
> > >> > > on
> > >> > > > the message size.)
> > >> > > >
> > >> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > apurva@confluent.io>
> > >> > > wrote:
> > >> > > >
> > >> > > > > Hi Ben,
> > >> > > > >
> > >> > > > > Now, on to your first question of how deal with consumer
> > >> rebalances.
> > >> > > The
> > >> > > > > short answer is that the application needs to ensure that the
> > the
> > >> > > > > assignment of input partitions to appId is consistent across
> > >> > > rebalances.
> > >> > > > >
> > >> > > > > For Kafka streams, they already ensure that the mapping of
> input
> > >> > > > partitions
> > >> > > > > to task Id is invariant across rebalances by implementing a
> > custom
> > >> > > sticky
> > >> > > > > assignor. Other non-streams apps can trivially have one
> producer
> > >> per
> > >> > > > input
> > >> > > > > partition and have the appId be the same as the partition
> number
> > >> to
> > >> > > > achieve
> > >> > > > > the same effect.
> > >> > > > >
> > >> > > > > With this precondition in place, we can maintain transactions
> > >> across
> > >> > > > > rebalances.
> > >> > > > >
> > >> > > > > Hope this answers your question.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > > Apurva
> > >> > > > >
> > >> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> wrote:
> > >> > > > >
> > >> > > > > > Thanks for this! I'm looking forward to going through the
> full
> > >> > > proposal
> > >> > > > > in
> > >> > > > > > detail soon; a few early questions:
> > >> > > > > >
> > >> > > > > > First: what happens when a consumer rebalances in the middle
> > of
> > >> a
> > >> > > > > > transaction? The full documentation suggests that such a
> > >> > transaction
> > >> > > > > ought
> > >> > > > > > to be rejected:
> > >> > > > > >
> > >> > > > > > > [...] if a rebalance has happened and this consumer
> > >> > > > > > > instance becomes a zombie, even if this offset message is
> > >> > appended
> > >> > > in
> > >> > > > > the
> > >> > > > > > > offset topic, the transaction will be rejected later on
> when
> > >> it
> > >> > > tries
> > >> > > > > to
> > >> > > > > > > commit the transaction via the EndTxnRequest.
> > >> > > > > >
> > >> > > > > > ...but it's unclear to me how we ensure that a transaction
> > can't
> > >> > > > complete
> > >> > > > > > if a rebalance has happened. (It's quite possible I'm
> missing
> > >> > > something
> > >> > > > > > obvious!)
> > >> > > > > >
> > >> > > > > > As a concrete example: suppose a process with PID 1 adds
> > offsets
> > >> > for
> > >> > > > some
> > >> > > > > > partition to a transaction; a consumer rebalance happens
> that
> > >> > assigns
> > >> > > > the
> > >> > > > > > partition to a process with PID 2, which adds some offsets
> to
> > >> its
> > >> > > > current
> > >> > > > > > transaction; both processes try and commit. Allowing both
> > >> commits
> > >> > > would
> > >> > > > > > cause the messages to be processed twice -- how is that
> > avoided?
> > >> > > > > >
> > >> > > > > > Second: App IDs normally map to a single PID. It seems like
> > one
> > >> > could
> > >> > > > do
> > >> > > > > > away with the PID concept entirely, and just use App IDs in
> > most
> > >> > > places
> > >> > > > > > that require a PID. This feels like it would be
> significantly
> > >> > > simpler,
> > >> > > > > > though it does increase the message size. Are there other
> > >> reasons
> > >> > why
> > >> > > > the
> > >> > > > > > App ID / PID split is necessary?
> > >> > > > > >
> > >> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > >> wangguoz@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Hi all,
> > >> > > > > > >
> > >> > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> > once
> > >> > > > delivery
> > >> > > > > > > semantics:
> > >> > > > > > >
> > >> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > >> > > > > > >
> > >> > > > > > > This KIP adds a transactional messaging mechanism along
> with
> > >> an
> > >> > > > > > idempotent
> > >> > > > > > > producer implementation to make sure that 1) duplicated
> > >> messages
> > >> > > sent
> > >> > > > > > from
> > >> > > > > > > the same identified producer can be detected on the broker
> > >> side,
> > >> > > and
> > >> > > > > 2) a
> > >> > > > > > > group of messages sent within a transaction will
> atomically
> > be
> > >> > > either
> > >> > > > > > > reflected and fetchable to consumers or not as a whole.
> > >> > > > > > >
> > >> > > > > > > The above wiki page provides a high-level view of the
> > proposed
> > >> > > > changes
> > >> > > > > as
> > >> > > > > > > well as summarized guarantees. Initial draft of the
> detailed
> > >> > > > > > implementation
> > >> > > > > > > design is described in this Google doc:
> > >> > > > > > >
> > >> > > > > > > https://docs.google.com/document/d/11Jqy_
> > >> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > >> > > > > > > 0wSw9ra8
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > We would love to hear your comments and suggestions.
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > >
> > >> > > > > > > -- Guozhang
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Radai,

Thanks for your email. You raise some good points.

Your larger point centers on your --correct!-- observation that persisting
messages belonging to aborted transactions causes complications for
consumers.

You are right in observing that the current proposal does require consumers
to buffer, though we can make that efficient by storing only the 'start
offsets' of transactions and then replaying the log once we hit the
corresponding commit/abort markers. And yes, it does mean that in some
consumer modes, we will deliver messages belonging to aborted transactions.

As such, I would really like to understand the details of your alternative
to see if we can address these issues efficiently.

When you say that you will write to the main partition from the 'sideways'
transaction partition atomically, do you literally mean that a sequence of
individual messages from the 'sideways' partition will be written to the
main partition atomically?

If the 'sideways partition' is written to disk, you have two copies of
data. I would like to know how you would ensure that you do not create
duplicates in the event of crashes. And if it is in-mem, you are now
buffering anyway, on all replicas.

And, in the absence of any markers in the log, how would you ensure that
the batch of messages in a committed transaction is delivered to the
consumer all together? In the current proposal, the return of a 'poll'
would include all the messages of a transaction, or none of them.

* *

Re: 3::  you ask about the migration plan. This has been detailed in the design
doc
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.3s6zgsaq9cep>.
In particular, the recommended sequence is to bump all your clients before
bumping up the message format. Further, since transactions are entirely
opt-in, it is not unreasonable to ensure that the ecosystem is ready for
them before starting to use them.

* *

Thanks,
Apurva


On Thu, Dec 15, 2016 at 3:07 PM, radai <ra...@gmail.com> wrote:

> some clarifications on my alternative proposal:
>
> TX msgs are written "sideways" to a transaction (ad-hoc) partition. this
> partition can be replicated to followers, or can be an in-mem buffer -
> depends on the resilience guarantees you want to provide for TXs in case of
> broker crash.
> on "commit" the partition leader broker (being the single point of
> synchronization for the partition anyway) can atomically append the
> contents of this TX "partition" onto the real target partition. this is the
> point where the msgs get "real" offsets. there's some trickiness around how
> not to expose these offsets to any consumers until everything's been
> replicated to followers, but we believe its possible.
>
>
>
> On Thu, Dec 15, 2016 at 2:31 PM, radai <ra...@gmail.com> wrote:
>
> > I can see several issues with the current proposal.
> >
> > messages, even if sent under a TX, are delivered directly to their
> > destination partitions, downstream consumers need to be TX-aware. they
> can
> > either:
> >    1. be oblivious to TXs. that means they will deliver "garbage" - msgs
> > sent during eventually-aborted TXs.
> >    2. "opt-in" - they would have to not deliver _ANY_ msg until they know
> > the fate of all outstanding overlapping TXs - if i see msg A1 (in a TX),
> > followed by B, which is not under any TX, i cannot deliver B until i know
> > if A1 was committed or not (or I violate ordering). this would require
> some
> > sort of buffering on consumers. with a naive buffering impl i could DOS
> > everyone on a topic - just start a TX on a very busy topic and keep it
> open
> > as long as I can ....
> >    3. explode if youre an old consumer that sees a control msg (whats
> your
> > migration plan?)
> >    4. cross-cluster replication mechanisms either replicate the garbage
> or
> > need to clean it up. there are >1 such different mechanism (almost one
> per
> > company really :-) ) so lots of adjustments.
> >
> > I think the end result could be better if ongoing TXs are treated as
> > logically separate topic partitions, and only atomically appended onto
> the
> > target partitions on commit (meaning they are written to separate journal
> > file(s) on the broker).
> >
> > such a design would present a "clean" view to any downstream consumers -
> > anything not committed wont even show up. old consumers wont need to know
> > about control msgs, no issues with unbounded msg buffering, generally
> > cleaner overall?
> >
> > there would need to be adjustments made to watermark and follower fetch
> > logic but some of us here have discussed this over lunch and we think its
> > doable.
> >
> >
> > On Thu, Dec 15, 2016 at 1:08 AM, Rajini Sivaram <rs...@pivotal.io>
> > wrote:
> >
> >> Hi Apurva,
> >>
> >> Thank you, makes sense.
> >>
> >> Rajini
> >>
> >> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io>
> >> wrote:
> >>
> >> > Hi Rajini,
> >> >
> >> > I think my original response to your point 15 was not accurate. The
> >> regular
> >> > definition of durability is that data once committed would never be
> >> lost.
> >> > So it is not enough for only the control messages to be flushed before
> >> > being acknowledged -- all the messages (and offset commits) which are
> >> part
> >> > of the transaction would need to be flushed before being acknowledged
> as
> >> > well.
> >> >
> >> > Otherwise, it is possible that if all replicas of a topic partition
> >> crash
> >> > before the transactional messages are flushed, those messages will be
> >> lost
> >> > even if the commit marker exists in the log. In this case, the
> >> transaction
> >> > would be 'committed' with incomplete data.
> >> >
> >> > Right now, there isn't any config which will ensure that the flush to
> >> disk
> >> > happens before the acknowledgement. We could add it in the future, and
> >> get
> >> > durability guarantees for kafka transactions.
> >> >
> >> > I hope this clarifies the situation. The present KIP does not intend
> to
> >> add
> >> > the aforementioned config, so even the control messages are
> susceptible
> >> to
> >> > being lost if there is a simultaneous crash across all replicas. So
> >> > transactions are only as durable as existing Kafka messages. We don't
> >> > strengthen any durability guarantees as part of this KIP.
> >> >
> >> > Thanks,
> >> > Apurva
> >> >
> >> >
> >> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> >> > wrote:
> >> >
> >> > > Hi Apurva,
> >> > >
> >> > > Thank you for the answers. Just one follow-on.
> >> > >
> >> > > 15. Let me rephrase my original question. If all control messages
> >> > (messages
> >> > > to transaction logs and markers on user logs) were acknowledged only
> >> > after
> >> > > flushing the log segment, will transactions become durable in the
> >> > > traditional sense (i.e. not restricted to min.insync.replicas
> >> failures) ?
> >> > > This is not a suggestion to update the KIP. It seems to me that the
> >> > design
> >> > > enables full durability if required in the future with a rather
> >> > > non-intrusive change. I just wanted to make sure I haven't missed
> >> > anything
> >> > > fundamental that prevents Kafka from doing this.
> >> > >
> >> > >
> >> > >
> >> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> >> > >
> >> > > > Hi Apurva,
> >> > > >
> >> > > > Thanks for the detailed answers... and sorry for the late reply!
> >> > > >
> >> > > > It does sound like, if the input-partitions-to-app-id mapping
> never
> >> > > > changes, the existing fencing mechanisms should prevent
> duplicates.
> >> > > Great!
> >> > > > I'm a bit concerned the proposed API will be delicate to program
> >> > against
> >> > > > successfully -- even in the simple case, we need to create a new
> >> > producer
> >> > > > instance per input partition, and anything fancier is going to
> need
> >> its
> >> > > own
> >> > > > implementation of the Streams/Samza-style 'task' idea -- but that
> >> may
> >> > be
> >> > > > fine for this sort of advanced feature.
> >> > > >
> >> > > > For the second question, I notice that Jason also elaborated on
> this
> >> > > > downthread:
> >> > > >
> >> > > > > We also looked at removing the producer ID.
> >> > > > > This was discussed somewhere above, but basically the idea is to
> >> > store
> >> > > > the
> >> > > > > AppID in the message set header directly and avoid the mapping
> to
> >> > > > producer
> >> > > > > ID altogether. As long as batching isn't too bad, the impact on
> >> total
> >> > > > size
> >> > > > > may not be too bad, but we were ultimately more comfortable
> with a
> >> > > fixed
> >> > > > > size ID.
> >> > > >
> >> > > > ...which suggests that the distinction is useful for performance,
> >> but
> >> > not
> >> > > > necessary for correctness, which makes good sense to me. (Would a
> >> > 128-bid
> >> > > > ID be a reasonable compromise? That's enough room for a UUID, or a
> >> > > > reasonable hash of an arbitrary string, and has only a marginal
> >> > increase
> >> > > on
> >> > > > the message size.)
> >> > > >
> >> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> apurva@confluent.io>
> >> > > wrote:
> >> > > >
> >> > > > > Hi Ben,
> >> > > > >
> >> > > > > Now, on to your first question of how deal with consumer
> >> rebalances.
> >> > > The
> >> > > > > short answer is that the application needs to ensure that the
> the
> >> > > > > assignment of input partitions to appId is consistent across
> >> > > rebalances.
> >> > > > >
> >> > > > > For Kafka streams, they already ensure that the mapping of input
> >> > > > partitions
> >> > > > > to task Id is invariant across rebalances by implementing a
> custom
> >> > > sticky
> >> > > > > assignor. Other non-streams apps can trivially have one producer
> >> per
> >> > > > input
> >> > > > > partition and have the appId be the same as the partition number
> >> to
> >> > > > achieve
> >> > > > > the same effect.
> >> > > > >
> >> > > > > With this precondition in place, we can maintain transactions
> >> across
> >> > > > > rebalances.
> >> > > > >
> >> > > > > Hope this answers your question.
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Apurva
> >> > > > >
> >> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> >> > > > >
> >> > > > > > Thanks for this! I'm looking forward to going through the full
> >> > > proposal
> >> > > > > in
> >> > > > > > detail soon; a few early questions:
> >> > > > > >
> >> > > > > > First: what happens when a consumer rebalances in the middle
> of
> >> a
> >> > > > > > transaction? The full documentation suggests that such a
> >> > transaction
> >> > > > > ought
> >> > > > > > to be rejected:
> >> > > > > >
> >> > > > > > > [...] if a rebalance has happened and this consumer
> >> > > > > > > instance becomes a zombie, even if this offset message is
> >> > appended
> >> > > in
> >> > > > > the
> >> > > > > > > offset topic, the transaction will be rejected later on when
> >> it
> >> > > tries
> >> > > > > to
> >> > > > > > > commit the transaction via the EndTxnRequest.
> >> > > > > >
> >> > > > > > ...but it's unclear to me how we ensure that a transaction
> can't
> >> > > > complete
> >> > > > > > if a rebalance has happened. (It's quite possible I'm missing
> >> > > something
> >> > > > > > obvious!)
> >> > > > > >
> >> > > > > > As a concrete example: suppose a process with PID 1 adds
> offsets
> >> > for
> >> > > > some
> >> > > > > > partition to a transaction; a consumer rebalance happens that
> >> > assigns
> >> > > > the
> >> > > > > > partition to a process with PID 2, which adds some offsets to
> >> its
> >> > > > current
> >> > > > > > transaction; both processes try and commit. Allowing both
> >> commits
> >> > > would
> >> > > > > > cause the messages to be processed twice -- how is that
> avoided?
> >> > > > > >
> >> > > > > > Second: App IDs normally map to a single PID. It seems like
> one
> >> > could
> >> > > > do
> >> > > > > > away with the PID concept entirely, and just use App IDs in
> most
> >> > > places
> >> > > > > > that require a PID. This feels like it would be significantly
> >> > > simpler,
> >> > > > > > though it does increase the message size. Are there other
> >> reasons
> >> > why
> >> > > > the
> >> > > > > > App ID / PID split is necessary?
> >> > > > > >
> >> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> >> wangguoz@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > Hi all,
> >> > > > > > >
> >> > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
> >> > > > delivery
> >> > > > > > > semantics:
> >> > > > > > >
> >> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >> > > > > > >
> >> > > > > > > This KIP adds a transactional messaging mechanism along with
> >> an
> >> > > > > > idempotent
> >> > > > > > > producer implementation to make sure that 1) duplicated
> >> messages
> >> > > sent
> >> > > > > > from
> >> > > > > > > the same identified producer can be detected on the broker
> >> side,
> >> > > and
> >> > > > > 2) a
> >> > > > > > > group of messages sent within a transaction will atomically
> be
> >> > > either
> >> > > > > > > reflected and fetchable to consumers or not as a whole.
> >> > > > > > >
> >> > > > > > > The above wiki page provides a high-level view of the
> proposed
> >> > > > changes
> >> > > > > as
> >> > > > > > > well as summarized guarantees. Initial draft of the detailed
> >> > > > > > implementation
> >> > > > > > > design is described in this Google doc:
> >> > > > > > >
> >> > > > > > > https://docs.google.com/document/d/11Jqy_
> >> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> >> > > > > > > 0wSw9ra8
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > We would love to hear your comments and suggestions.
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > >
> >> > > > > > > -- Guozhang
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
some clarifications on my alternative proposal:

TX msgs are written "sideways" to a transaction (ad-hoc) partition. this
partition can be replicated to followers, or can be an in-mem buffer -
depends on the resilience guarantees you want to provide for TXs in case of
broker crash.
on "commit" the partition leader broker (being the single point of
synchronization for the partition anyway) can atomically append the
contents of this TX "partition" onto the real target partition. this is the
point where the msgs get "real" offsets. there's some trickiness around how
not to expose these offsets to any consumers until everything's been
replicated to followers, but we believe its possible.



On Thu, Dec 15, 2016 at 2:31 PM, radai <ra...@gmail.com> wrote:

> I can see several issues with the current proposal.
>
> messages, even if sent under a TX, are delivered directly to their
> destination partitions, downstream consumers need to be TX-aware. they can
> either:
>    1. be oblivious to TXs. that means they will deliver "garbage" - msgs
> sent during eventually-aborted TXs.
>    2. "opt-in" - they would have to not deliver _ANY_ msg until they know
> the fate of all outstanding overlapping TXs - if i see msg A1 (in a TX),
> followed by B, which is not under any TX, i cannot deliver B until i know
> if A1 was committed or not (or I violate ordering). this would require some
> sort of buffering on consumers. with a naive buffering impl i could DOS
> everyone on a topic - just start a TX on a very busy topic and keep it open
> as long as I can ....
>    3. explode if youre an old consumer that sees a control msg (whats your
> migration plan?)
>    4. cross-cluster replication mechanisms either replicate the garbage or
> need to clean it up. there are >1 such different mechanism (almost one per
> company really :-) ) so lots of adjustments.
>
> I think the end result could be better if ongoing TXs are treated as
> logically separate topic partitions, and only atomically appended onto the
> target partitions on commit (meaning they are written to separate journal
> file(s) on the broker).
>
> such a design would present a "clean" view to any downstream consumers -
> anything not committed wont even show up. old consumers wont need to know
> about control msgs, no issues with unbounded msg buffering, generally
> cleaner overall?
>
> there would need to be adjustments made to watermark and follower fetch
> logic but some of us here have discussed this over lunch and we think its
> doable.
>
>
> On Thu, Dec 15, 2016 at 1:08 AM, Rajini Sivaram <rs...@pivotal.io>
> wrote:
>
>> Hi Apurva,
>>
>> Thank you, makes sense.
>>
>> Rajini
>>
>> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io>
>> wrote:
>>
>> > Hi Rajini,
>> >
>> > I think my original response to your point 15 was not accurate. The
>> regular
>> > definition of durability is that data once committed would never be
>> lost.
>> > So it is not enough for only the control messages to be flushed before
>> > being acknowledged -- all the messages (and offset commits) which are
>> part
>> > of the transaction would need to be flushed before being acknowledged as
>> > well.
>> >
>> > Otherwise, it is possible that if all replicas of a topic partition
>> crash
>> > before the transactional messages are flushed, those messages will be
>> lost
>> > even if the commit marker exists in the log. In this case, the
>> transaction
>> > would be 'committed' with incomplete data.
>> >
>> > Right now, there isn't any config which will ensure that the flush to
>> disk
>> > happens before the acknowledgement. We could add it in the future, and
>> get
>> > durability guarantees for kafka transactions.
>> >
>> > I hope this clarifies the situation. The present KIP does not intend to
>> add
>> > the aforementioned config, so even the control messages are susceptible
>> to
>> > being lost if there is a simultaneous crash across all replicas. So
>> > transactions are only as durable as existing Kafka messages. We don't
>> > strengthen any durability guarantees as part of this KIP.
>> >
>> > Thanks,
>> > Apurva
>> >
>> >
>> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
>> > wrote:
>> >
>> > > Hi Apurva,
>> > >
>> > > Thank you for the answers. Just one follow-on.
>> > >
>> > > 15. Let me rephrase my original question. If all control messages
>> > (messages
>> > > to transaction logs and markers on user logs) were acknowledged only
>> > after
>> > > flushing the log segment, will transactions become durable in the
>> > > traditional sense (i.e. not restricted to min.insync.replicas
>> failures) ?
>> > > This is not a suggestion to update the KIP. It seems to me that the
>> > design
>> > > enables full durability if required in the future with a rather
>> > > non-intrusive change. I just wanted to make sure I haven't missed
>> > anything
>> > > fundamental that prevents Kafka from doing this.
>> > >
>> > >
>> > >
>> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
>> > >
>> > > > Hi Apurva,
>> > > >
>> > > > Thanks for the detailed answers... and sorry for the late reply!
>> > > >
>> > > > It does sound like, if the input-partitions-to-app-id mapping never
>> > > > changes, the existing fencing mechanisms should prevent duplicates.
>> > > Great!
>> > > > I'm a bit concerned the proposed API will be delicate to program
>> > against
>> > > > successfully -- even in the simple case, we need to create a new
>> > producer
>> > > > instance per input partition, and anything fancier is going to need
>> its
>> > > own
>> > > > implementation of the Streams/Samza-style 'task' idea -- but that
>> may
>> > be
>> > > > fine for this sort of advanced feature.
>> > > >
>> > > > For the second question, I notice that Jason also elaborated on this
>> > > > downthread:
>> > > >
>> > > > > We also looked at removing the producer ID.
>> > > > > This was discussed somewhere above, but basically the idea is to
>> > store
>> > > > the
>> > > > > AppID in the message set header directly and avoid the mapping to
>> > > > producer
>> > > > > ID altogether. As long as batching isn't too bad, the impact on
>> total
>> > > > size
>> > > > > may not be too bad, but we were ultimately more comfortable with a
>> > > fixed
>> > > > > size ID.
>> > > >
>> > > > ...which suggests that the distinction is useful for performance,
>> but
>> > not
>> > > > necessary for correctness, which makes good sense to me. (Would a
>> > 128-bid
>> > > > ID be a reasonable compromise? That's enough room for a UUID, or a
>> > > > reasonable hash of an arbitrary string, and has only a marginal
>> > increase
>> > > on
>> > > > the message size.)
>> > > >
>> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
>> > > wrote:
>> > > >
>> > > > > Hi Ben,
>> > > > >
>> > > > > Now, on to your first question of how deal with consumer
>> rebalances.
>> > > The
>> > > > > short answer is that the application needs to ensure that the the
>> > > > > assignment of input partitions to appId is consistent across
>> > > rebalances.
>> > > > >
>> > > > > For Kafka streams, they already ensure that the mapping of input
>> > > > partitions
>> > > > > to task Id is invariant across rebalances by implementing a custom
>> > > sticky
>> > > > > assignor. Other non-streams apps can trivially have one producer
>> per
>> > > > input
>> > > > > partition and have the appId be the same as the partition number
>> to
>> > > > achieve
>> > > > > the same effect.
>> > > > >
>> > > > > With this precondition in place, we can maintain transactions
>> across
>> > > > > rebalances.
>> > > > >
>> > > > > Hope this answers your question.
>> > > > >
>> > > > > Thanks,
>> > > > > Apurva
>> > > > >
>> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
>> > > > >
>> > > > > > Thanks for this! I'm looking forward to going through the full
>> > > proposal
>> > > > > in
>> > > > > > detail soon; a few early questions:
>> > > > > >
>> > > > > > First: what happens when a consumer rebalances in the middle of
>> a
>> > > > > > transaction? The full documentation suggests that such a
>> > transaction
>> > > > > ought
>> > > > > > to be rejected:
>> > > > > >
>> > > > > > > [...] if a rebalance has happened and this consumer
>> > > > > > > instance becomes a zombie, even if this offset message is
>> > appended
>> > > in
>> > > > > the
>> > > > > > > offset topic, the transaction will be rejected later on when
>> it
>> > > tries
>> > > > > to
>> > > > > > > commit the transaction via the EndTxnRequest.
>> > > > > >
>> > > > > > ...but it's unclear to me how we ensure that a transaction can't
>> > > > complete
>> > > > > > if a rebalance has happened. (It's quite possible I'm missing
>> > > something
>> > > > > > obvious!)
>> > > > > >
>> > > > > > As a concrete example: suppose a process with PID 1 adds offsets
>> > for
>> > > > some
>> > > > > > partition to a transaction; a consumer rebalance happens that
>> > assigns
>> > > > the
>> > > > > > partition to a process with PID 2, which adds some offsets to
>> its
>> > > > current
>> > > > > > transaction; both processes try and commit. Allowing both
>> commits
>> > > would
>> > > > > > cause the messages to be processed twice -- how is that avoided?
>> > > > > >
>> > > > > > Second: App IDs normally map to a single PID. It seems like one
>> > could
>> > > > do
>> > > > > > away with the PID concept entirely, and just use App IDs in most
>> > > places
>> > > > > > that require a PID. This feels like it would be significantly
>> > > simpler,
>> > > > > > though it does increase the message size. Are there other
>> reasons
>> > why
>> > > > the
>> > > > > > App ID / PID split is necessary?
>> > > > > >
>> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
>> wangguoz@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >
>> > > > > > > Hi all,
>> > > > > > >
>> > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
>> > > > delivery
>> > > > > > > semantics:
>> > > > > > >
>> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>> > > > > > >
>> > > > > > > This KIP adds a transactional messaging mechanism along with
>> an
>> > > > > > idempotent
>> > > > > > > producer implementation to make sure that 1) duplicated
>> messages
>> > > sent
>> > > > > > from
>> > > > > > > the same identified producer can be detected on the broker
>> side,
>> > > and
>> > > > > 2) a
>> > > > > > > group of messages sent within a transaction will atomically be
>> > > either
>> > > > > > > reflected and fetchable to consumers or not as a whole.
>> > > > > > >
>> > > > > > > The above wiki page provides a high-level view of the proposed
>> > > > changes
>> > > > > as
>> > > > > > > well as summarized guarantees. Initial draft of the detailed
>> > > > > > implementation
>> > > > > > > design is described in this Google doc:
>> > > > > > >
>> > > > > > > https://docs.google.com/document/d/11Jqy_
>> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> > > > > > > 0wSw9ra8
>> > > > > > >
>> > > > > > >
>> > > > > > > We would love to hear your comments and suggestions.
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > >
>> > > > > > > -- Guozhang
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
I can see several issues with the current proposal.

messages, even if sent under a TX, are delivered directly to their
destination partitions, downstream consumers need to be TX-aware. they can
either:
   1. be oblivious to TXs. that means they will deliver "garbage" - msgs
sent during eventually-aborted TXs.
   2. "opt-in" - they would have to not deliver _ANY_ msg until they know
the fate of all outstanding overlapping TXs - if i see msg A1 (in a TX),
followed by B, which is not under any TX, i cannot deliver B until i know
if A1 was committed or not (or I violate ordering). this would require some
sort of buffering on consumers. with a naive buffering impl i could DOS
everyone on a topic - just start a TX on a very busy topic and keep it open
as long as I can ....
   3. explode if youre an old consumer that sees a control msg (whats your
migration plan?)
   4. cross-cluster replication mechanisms either replicate the garbage or
need to clean it up. there are >1 such different mechanism (almost one per
company really :-) ) so lots of adjustments.

I think the end result could be better if ongoing TXs are treated as
logically separate topic partitions, and only atomically appended onto the
target partitions on commit (meaning they are written to separate journal
file(s) on the broker).

such a design would present a "clean" view to any downstream consumers -
anything not committed wont even show up. old consumers wont need to know
about control msgs, no issues with unbounded msg buffering, generally
cleaner overall?

there would need to be adjustments made to watermark and follower fetch
logic but some of us here have discussed this over lunch and we think its
doable.


On Thu, Dec 15, 2016 at 1:08 AM, Rajini Sivaram <rs...@pivotal.io> wrote:

> Hi Apurva,
>
> Thank you, makes sense.
>
> Rajini
>
> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Rajini,
> >
> > I think my original response to your point 15 was not accurate. The
> regular
> > definition of durability is that data once committed would never be lost.
> > So it is not enough for only the control messages to be flushed before
> > being acknowledged -- all the messages (and offset commits) which are
> part
> > of the transaction would need to be flushed before being acknowledged as
> > well.
> >
> > Otherwise, it is possible that if all replicas of a topic partition crash
> > before the transactional messages are flushed, those messages will be
> lost
> > even if the commit marker exists in the log. In this case, the
> transaction
> > would be 'committed' with incomplete data.
> >
> > Right now, there isn't any config which will ensure that the flush to
> disk
> > happens before the acknowledgement. We could add it in the future, and
> get
> > durability guarantees for kafka transactions.
> >
> > I hope this clarifies the situation. The present KIP does not intend to
> add
> > the aforementioned config, so even the control messages are susceptible
> to
> > being lost if there is a simultaneous crash across all replicas. So
> > transactions are only as durable as existing Kafka messages. We don't
> > strengthen any durability guarantees as part of this KIP.
> >
> > Thanks,
> > Apurva
> >
> >
> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> > wrote:
> >
> > > Hi Apurva,
> > >
> > > Thank you for the answers. Just one follow-on.
> > >
> > > 15. Let me rephrase my original question. If all control messages
> > (messages
> > > to transaction logs and markers on user logs) were acknowledged only
> > after
> > > flushing the log segment, will transactions become durable in the
> > > traditional sense (i.e. not restricted to min.insync.replicas
> failures) ?
> > > This is not a suggestion to update the KIP. It seems to me that the
> > design
> > > enables full durability if required in the future with a rather
> > > non-intrusive change. I just wanted to make sure I haven't missed
> > anything
> > > fundamental that prevents Kafka from doing this.
> > >
> > >
> > >
> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > >
> > > > Hi Apurva,
> > > >
> > > > Thanks for the detailed answers... and sorry for the late reply!
> > > >
> > > > It does sound like, if the input-partitions-to-app-id mapping never
> > > > changes, the existing fencing mechanisms should prevent duplicates.
> > > Great!
> > > > I'm a bit concerned the proposed API will be delicate to program
> > against
> > > > successfully -- even in the simple case, we need to create a new
> > producer
> > > > instance per input partition, and anything fancier is going to need
> its
> > > own
> > > > implementation of the Streams/Samza-style 'task' idea -- but that may
> > be
> > > > fine for this sort of advanced feature.
> > > >
> > > > For the second question, I notice that Jason also elaborated on this
> > > > downthread:
> > > >
> > > > > We also looked at removing the producer ID.
> > > > > This was discussed somewhere above, but basically the idea is to
> > store
> > > > the
> > > > > AppID in the message set header directly and avoid the mapping to
> > > > producer
> > > > > ID altogether. As long as batching isn't too bad, the impact on
> total
> > > > size
> > > > > may not be too bad, but we were ultimately more comfortable with a
> > > fixed
> > > > > size ID.
> > > >
> > > > ...which suggests that the distinction is useful for performance, but
> > not
> > > > necessary for correctness, which makes good sense to me. (Would a
> > 128-bid
> > > > ID be a reasonable compromise? That's enough room for a UUID, or a
> > > > reasonable hash of an arbitrary string, and has only a marginal
> > increase
> > > on
> > > > the message size.)
> > > >
> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > > >
> > > > > Hi Ben,
> > > > >
> > > > > Now, on to your first question of how deal with consumer
> rebalances.
> > > The
> > > > > short answer is that the application needs to ensure that the the
> > > > > assignment of input partitions to appId is consistent across
> > > rebalances.
> > > > >
> > > > > For Kafka streams, they already ensure that the mapping of input
> > > > partitions
> > > > > to task Id is invariant across rebalances by implementing a custom
> > > sticky
> > > > > assignor. Other non-streams apps can trivially have one producer
> per
> > > > input
> > > > > partition and have the appId be the same as the partition number to
> > > > achieve
> > > > > the same effect.
> > > > >
> > > > > With this precondition in place, we can maintain transactions
> across
> > > > > rebalances.
> > > > >
> > > > > Hope this answers your question.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> > > > >
> > > > > > Thanks for this! I'm looking forward to going through the full
> > > proposal
> > > > > in
> > > > > > detail soon; a few early questions:
> > > > > >
> > > > > > First: what happens when a consumer rebalances in the middle of a
> > > > > > transaction? The full documentation suggests that such a
> > transaction
> > > > > ought
> > > > > > to be rejected:
> > > > > >
> > > > > > > [...] if a rebalance has happened and this consumer
> > > > > > > instance becomes a zombie, even if this offset message is
> > appended
> > > in
> > > > > the
> > > > > > > offset topic, the transaction will be rejected later on when it
> > > tries
> > > > > to
> > > > > > > commit the transaction via the EndTxnRequest.
> > > > > >
> > > > > > ...but it's unclear to me how we ensure that a transaction can't
> > > > complete
> > > > > > if a rebalance has happened. (It's quite possible I'm missing
> > > something
> > > > > > obvious!)
> > > > > >
> > > > > > As a concrete example: suppose a process with PID 1 adds offsets
> > for
> > > > some
> > > > > > partition to a transaction; a consumer rebalance happens that
> > assigns
> > > > the
> > > > > > partition to a process with PID 2, which adds some offsets to its
> > > > current
> > > > > > transaction; both processes try and commit. Allowing both commits
> > > would
> > > > > > cause the messages to be processed twice -- how is that avoided?
> > > > > >
> > > > > > Second: App IDs normally map to a single PID. It seems like one
> > could
> > > > do
> > > > > > away with the PID concept entirely, and just use App IDs in most
> > > places
> > > > > > that require a PID. This feels like it would be significantly
> > > simpler,
> > > > > > though it does increase the message size. Are there other reasons
> > why
> > > > the
> > > > > > App ID / PID split is necessary?
> > > > > >
> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > > delivery
> > > > > > > semantics:
> > > > > > >
> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > > > >
> > > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > > idempotent
> > > > > > > producer implementation to make sure that 1) duplicated
> messages
> > > sent
> > > > > > from
> > > > > > > the same identified producer can be detected on the broker
> side,
> > > and
> > > > > 2) a
> > > > > > > group of messages sent within a transaction will atomically be
> > > either
> > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > > >
> > > > > > > The above wiki page provides a high-level view of the proposed
> > > > changes
> > > > > as
> > > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > > implementation
> > > > > > > design is described in this Google doc:
> > > > > > >
> > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > 0wSw9ra8
> > > > > > >
> > > > > > >
> > > > > > > We would love to hear your comments and suggestions.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <rs...@pivotal.io>.
Hi Apurva,

Thank you, makes sense.

Rajini

On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Rajini,
>
> I think my original response to your point 15 was not accurate. The regular
> definition of durability is that data once committed would never be lost.
> So it is not enough for only the control messages to be flushed before
> being acknowledged -- all the messages (and offset commits) which are part
> of the transaction would need to be flushed before being acknowledged as
> well.
>
> Otherwise, it is possible that if all replicas of a topic partition crash
> before the transactional messages are flushed, those messages will be lost
> even if the commit marker exists in the log. In this case, the transaction
> would be 'committed' with incomplete data.
>
> Right now, there isn't any config which will ensure that the flush to disk
> happens before the acknowledgement. We could add it in the future, and get
> durability guarantees for kafka transactions.
>
> I hope this clarifies the situation. The present KIP does not intend to add
> the aforementioned config, so even the control messages are susceptible to
> being lost if there is a simultaneous crash across all replicas. So
> transactions are only as durable as existing Kafka messages. We don't
> strengthen any durability guarantees as part of this KIP.
>
> Thanks,
> Apurva
>
>
> On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> wrote:
>
> > Hi Apurva,
> >
> > Thank you for the answers. Just one follow-on.
> >
> > 15. Let me rephrase my original question. If all control messages
> (messages
> > to transaction logs and markers on user logs) were acknowledged only
> after
> > flushing the log segment, will transactions become durable in the
> > traditional sense (i.e. not restricted to min.insync.replicas failures) ?
> > This is not a suggestion to update the KIP. It seems to me that the
> design
> > enables full durability if required in the future with a rather
> > non-intrusive change. I just wanted to make sure I haven't missed
> anything
> > fundamental that prevents Kafka from doing this.
> >
> >
> >
> > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> >
> > > Hi Apurva,
> > >
> > > Thanks for the detailed answers... and sorry for the late reply!
> > >
> > > It does sound like, if the input-partitions-to-app-id mapping never
> > > changes, the existing fencing mechanisms should prevent duplicates.
> > Great!
> > > I'm a bit concerned the proposed API will be delicate to program
> against
> > > successfully -- even in the simple case, we need to create a new
> producer
> > > instance per input partition, and anything fancier is going to need its
> > own
> > > implementation of the Streams/Samza-style 'task' idea -- but that may
> be
> > > fine for this sort of advanced feature.
> > >
> > > For the second question, I notice that Jason also elaborated on this
> > > downthread:
> > >
> > > > We also looked at removing the producer ID.
> > > > This was discussed somewhere above, but basically the idea is to
> store
> > > the
> > > > AppID in the message set header directly and avoid the mapping to
> > > producer
> > > > ID altogether. As long as batching isn't too bad, the impact on total
> > > size
> > > > may not be too bad, but we were ultimately more comfortable with a
> > fixed
> > > > size ID.
> > >
> > > ...which suggests that the distinction is useful for performance, but
> not
> > > necessary for correctness, which makes good sense to me. (Would a
> 128-bid
> > > ID be a reasonable compromise? That's enough room for a UUID, or a
> > > reasonable hash of an arbitrary string, and has only a marginal
> increase
> > on
> > > the message size.)
> > >
> > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> > >
> > > > Hi Ben,
> > > >
> > > > Now, on to your first question of how deal with consumer rebalances.
> > The
> > > > short answer is that the application needs to ensure that the the
> > > > assignment of input partitions to appId is consistent across
> > rebalances.
> > > >
> > > > For Kafka streams, they already ensure that the mapping of input
> > > partitions
> > > > to task Id is invariant across rebalances by implementing a custom
> > sticky
> > > > assignor. Other non-streams apps can trivially have one producer per
> > > input
> > > > partition and have the appId be the same as the partition number to
> > > achieve
> > > > the same effect.
> > > >
> > > > With this precondition in place, we can maintain transactions across
> > > > rebalances.
> > > >
> > > > Hope this answers your question.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> > > >
> > > > > Thanks for this! I'm looking forward to going through the full
> > proposal
> > > > in
> > > > > detail soon; a few early questions:
> > > > >
> > > > > First: what happens when a consumer rebalances in the middle of a
> > > > > transaction? The full documentation suggests that such a
> transaction
> > > > ought
> > > > > to be rejected:
> > > > >
> > > > > > [...] if a rebalance has happened and this consumer
> > > > > > instance becomes a zombie, even if this offset message is
> appended
> > in
> > > > the
> > > > > > offset topic, the transaction will be rejected later on when it
> > tries
> > > > to
> > > > > > commit the transaction via the EndTxnRequest.
> > > > >
> > > > > ...but it's unclear to me how we ensure that a transaction can't
> > > complete
> > > > > if a rebalance has happened. (It's quite possible I'm missing
> > something
> > > > > obvious!)
> > > > >
> > > > > As a concrete example: suppose a process with PID 1 adds offsets
> for
> > > some
> > > > > partition to a transaction; a consumer rebalance happens that
> assigns
> > > the
> > > > > partition to a process with PID 2, which adds some offsets to its
> > > current
> > > > > transaction; both processes try and commit. Allowing both commits
> > would
> > > > > cause the messages to be processed twice -- how is that avoided?
> > > > >
> > > > > Second: App IDs normally map to a single PID. It seems like one
> could
> > > do
> > > > > away with the PID concept entirely, and just use App IDs in most
> > places
> > > > > that require a PID. This feels like it would be significantly
> > simpler,
> > > > > though it does increase the message size. Are there other reasons
> why
> > > the
> > > > > App ID / PID split is necessary?
> > > > >
> > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > delivery
> > > > > > semantics:
> > > > > >
> > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > > >
> > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > idempotent
> > > > > > producer implementation to make sure that 1) duplicated messages
> > sent
> > > > > from
> > > > > > the same identified producer can be detected on the broker side,
> > and
> > > > 2) a
> > > > > > group of messages sent within a transaction will atomically be
> > either
> > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > >
> > > > > > The above wiki page provides a high-level view of the proposed
> > > changes
> > > > as
> > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > implementation
> > > > > > design is described in this Google doc:
> > > > > >
> > > > > > https://docs.google.com/document/d/11Jqy_
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8
> > > > > >
> > > > > >
> > > > > > We would love to hear your comments and suggestions.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Rajini,

I think my original response to your point 15 was not accurate. The regular
definition of durability is that data once committed would never be lost.
So it is not enough for only the control messages to be flushed before
being acknowledged -- all the messages (and offset commits) which are part
of the transaction would need to be flushed before being acknowledged as
well.

Otherwise, it is possible that if all replicas of a topic partition crash
before the transactional messages are flushed, those messages will be lost
even if the commit marker exists in the log. In this case, the transaction
would be 'committed' with incomplete data.

Right now, there isn't any config which will ensure that the flush to disk
happens before the acknowledgement. We could add it in the future, and get
durability guarantees for kafka transactions.

I hope this clarifies the situation. The present KIP does not intend to add
the aforementioned config, so even the control messages are susceptible to
being lost if there is a simultaneous crash across all replicas. So
transactions are only as durable as existing Kafka messages. We don't
strengthen any durability guarantees as part of this KIP.

Thanks,
Apurva


On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io> wrote:

> Hi Apurva,
>
> Thank you for the answers. Just one follow-on.
>
> 15. Let me rephrase my original question. If all control messages (messages
> to transaction logs and markers on user logs) were acknowledged only after
> flushing the log segment, will transactions become durable in the
> traditional sense (i.e. not restricted to min.insync.replicas failures) ?
> This is not a suggestion to update the KIP. It seems to me that the design
> enables full durability if required in the future with a rather
> non-intrusive change. I just wanted to make sure I haven't missed anything
> fundamental that prevents Kafka from doing this.
>
>
>
> On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
>
> > Hi Apurva,
> >
> > Thanks for the detailed answers... and sorry for the late reply!
> >
> > It does sound like, if the input-partitions-to-app-id mapping never
> > changes, the existing fencing mechanisms should prevent duplicates.
> Great!
> > I'm a bit concerned the proposed API will be delicate to program against
> > successfully -- even in the simple case, we need to create a new producer
> > instance per input partition, and anything fancier is going to need its
> own
> > implementation of the Streams/Samza-style 'task' idea -- but that may be
> > fine for this sort of advanced feature.
> >
> > For the second question, I notice that Jason also elaborated on this
> > downthread:
> >
> > > We also looked at removing the producer ID.
> > > This was discussed somewhere above, but basically the idea is to store
> > the
> > > AppID in the message set header directly and avoid the mapping to
> > producer
> > > ID altogether. As long as batching isn't too bad, the impact on total
> > size
> > > may not be too bad, but we were ultimately more comfortable with a
> fixed
> > > size ID.
> >
> > ...which suggests that the distinction is useful for performance, but not
> > necessary for correctness, which makes good sense to me. (Would a 128-bid
> > ID be a reasonable compromise? That's enough room for a UUID, or a
> > reasonable hash of an arbitrary string, and has only a marginal increase
> on
> > the message size.)
> >
> > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Hi Ben,
> > >
> > > Now, on to your first question of how deal with consumer rebalances.
> The
> > > short answer is that the application needs to ensure that the the
> > > assignment of input partitions to appId is consistent across
> rebalances.
> > >
> > > For Kafka streams, they already ensure that the mapping of input
> > partitions
> > > to task Id is invariant across rebalances by implementing a custom
> sticky
> > > assignor. Other non-streams apps can trivially have one producer per
> > input
> > > partition and have the appId be the same as the partition number to
> > achieve
> > > the same effect.
> > >
> > > With this precondition in place, we can maintain transactions across
> > > rebalances.
> > >
> > > Hope this answers your question.
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> > >
> > > > Thanks for this! I'm looking forward to going through the full
> proposal
> > > in
> > > > detail soon; a few early questions:
> > > >
> > > > First: what happens when a consumer rebalances in the middle of a
> > > > transaction? The full documentation suggests that such a transaction
> > > ought
> > > > to be rejected:
> > > >
> > > > > [...] if a rebalance has happened and this consumer
> > > > > instance becomes a zombie, even if this offset message is appended
> in
> > > the
> > > > > offset topic, the transaction will be rejected later on when it
> tries
> > > to
> > > > > commit the transaction via the EndTxnRequest.
> > > >
> > > > ...but it's unclear to me how we ensure that a transaction can't
> > complete
> > > > if a rebalance has happened. (It's quite possible I'm missing
> something
> > > > obvious!)
> > > >
> > > > As a concrete example: suppose a process with PID 1 adds offsets for
> > some
> > > > partition to a transaction; a consumer rebalance happens that assigns
> > the
> > > > partition to a process with PID 2, which adds some offsets to its
> > current
> > > > transaction; both processes try and commit. Allowing both commits
> would
> > > > cause the messages to be processed twice -- how is that avoided?
> > > >
> > > > Second: App IDs normally map to a single PID. It seems like one could
> > do
> > > > away with the PID concept entirely, and just use App IDs in most
> places
> > > > that require a PID. This feels like it would be significantly
> simpler,
> > > > though it does increase the message size. Are there other reasons why
> > the
> > > > App ID / PID split is necessary?
> > > >
> > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > delivery
> > > > > semantics:
> > > > >
> > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > >
> > > > > This KIP adds a transactional messaging mechanism along with an
> > > > idempotent
> > > > > producer implementation to make sure that 1) duplicated messages
> sent
> > > > from
> > > > > the same identified producer can be detected on the broker side,
> and
> > > 2) a
> > > > > group of messages sent within a transaction will atomically be
> either
> > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > The above wiki page provides a high-level view of the proposed
> > changes
> > > as
> > > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > > design is described in this Google doc:
> > > > >
> > > > > https://docs.google.com/document/d/11Jqy_
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8
> > > > >
> > > > >
> > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Eugen Dueck <eu...@dueck.org>.
 > The PID and sequence numbers are totally transparent to applications.

Now that you say it, the cwiki makes that point already pretty clear by 
the way the Producer API is (not) being changed. Sorry for taking your 
time on this.

In other words, at the point where messages enter the Kafka system for 
the first time, KIP-98 does not provide exactly-once guarantees across 
producer sessions.

This all makes sense, and what I am trying to do is probably best 
tackled on the business layer - where it needs to be done anyway if one 
cares about end-to-end guarantees - so I will just resend all 
potentially unsent messages when a producer fails, and use a 
business-level id inside the message to de-duplicate messages further 
downstream.

This also exhibits good performance (no additional persisting to disk of 
seq-id information). As de-duplication processing is stateful, a 
consumer might want to maintain (and persist to another topic) an x-hour 
"LRU business ids" cache - but that seems straightforward with Kafka.

Anyway, this is no longer of interest to KIP-98. Thanks for the 
clarification!

Cheers
Eugen

On 2017\u5e7401\u670831\u65e5 04:39, Apurva Mehta wrote:
>>
>> Eugen, moving your email to the main thread so that it doesn't get split.
>>>
>>> The `transaction.app.id` is a prerequisite for using transactional APIs.
>>> And only messages wrapped inside transactions will enjoy idempotent
>>> guarantees across sessions, and that too only when they employ a
>>> consume-process-produce pattern.
>>>
>>
>> Say I have a producer, producing messages into a topic and I only want to
>> guarantee the producer cannot insert duplicates. In other words, there's no
>> downstream consumer/processor to be worried about - which, when considering
>> the correctness of the data only, is all I need for idempotent producers,
>> as every message has a unique id (offset), so downstream processes can take
>> care of exactly once processing by any number of means. (If you need
>> transactional all-or-none behavior, which KIP-98 also addresses, that's of
>> course a more complex story)
>>
>> I was under the impression that KIP-98 would fulfill above requirement,
>> i.e. the prevention of duplicate inserts of the same message into a topic
>> per producer, without using transactions, and guaranteed across tcp
>> connections to handle producer/broker crashes and network problems.
>
>
> The KIP-98 idempotent producer solution only protects against duplicates in
> the stream when there are broker failures and network problems. For
> instance, if a producer writes a message, and the leader commits and
> replicates the message but dies before the acknowledgement is sent to the
> client. Today, the client will resend the message which will be accepted by
> the new leader, hence causing duplicates. Also, the offsets of the
> duplicate messages in this case will be unique, so they can't be de-duped
> downstream based on the offset.
>
> If the client application itself dies, it needs to know which messages were
> previously sent so that it doesn't resend them when it comes back online.
> The proposed solution to this situation is to use transactional APIs and
> the consume-process-produce pattern. If you do so, partially processed
> previous inputs will be discarded, and processing will resume from the last
> committed state.
>
>
>>
>> In other words, producers where the `transaction.app.id` is specified will
>>> not enjoy idempotence across sessions unless their messages are
>>> transactional. ie. that the sends  are wrapped between `beginTransaction`,
>>> `sendOffsets`, and `commitTransaction`.
>>>
>>
>> From the KIP-98 wiki and the design document, I understand that AppIDs,
>> PIDs, and sequence numbers are enforced regardless of their being wrapped
>> in a transaction or not. Is that not so?
>>
>>
> The PID and sequence numbers are totally transparent to applications. If
> you enable idempotent production, these will be created and managed by
> Kafka.
>
> AppIds only need to be specified if you use the four new transactional
> APIs. This is enforced at runtime.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
>
> Eugen, moving your email to the main thread so that it doesn't get split.
>>
>> The `transaction.app.id` is a prerequisite for using transactional APIs.
>> And only messages wrapped inside transactions will enjoy idempotent
>> guarantees across sessions, and that too only when they employ a
>> consume-process-produce pattern.
>>
>
> Say I have a producer, producing messages into a topic and I only want to
> guarantee the producer cannot insert duplicates. In other words, there's no
> downstream consumer/processor to be worried about - which, when considering
> the correctness of the data only, is all I need for idempotent producers,
> as every message has a unique id (offset), so downstream processes can take
> care of exactly once processing by any number of means. (If you need
> transactional all-or-none behavior, which KIP-98 also addresses, that's of
> course a more complex story)
>
> I was under the impression that KIP-98 would fulfill above requirement,
> i.e. the prevention of duplicate inserts of the same message into a topic
> per producer, without using transactions, and guaranteed across tcp
> connections to handle producer/broker crashes and network problems.


The KIP-98 idempotent producer solution only protects against duplicates in
the stream when there are broker failures and network problems. For
instance, if a producer writes a message, and the leader commits and
replicates the message but dies before the acknowledgement is sent to the
client. Today, the client will resend the message which will be accepted by
the new leader, hence causing duplicates. Also, the offsets of the
duplicate messages in this case will be unique, so they can't be de-duped
downstream based on the offset.

If the client application itself dies, it needs to know which messages were
previously sent so that it doesn't resend them when it comes back online.
The proposed solution to this situation is to use transactional APIs and
the consume-process-produce pattern. If you do so, partially processed
previous inputs will be discarded, and processing will resume from the last
committed state.


>
> In other words, producers where the `transaction.app.id` is specified will
>> not enjoy idempotence across sessions unless their messages are
>> transactional. ie. that the sends  are wrapped between `beginTransaction`,
>> `sendOffsets`, and `commitTransaction`.
>>
>
> From the KIP-98 wiki and the design document, I understand that AppIDs,
> PIDs, and sequence numbers are enforced regardless of their being wrapped
> in a transaction or not. Is that not so?
>
>
The PID and sequence numbers are totally transparent to applications. If
you enable idempotent production, these will be created and managed by
Kafka.

AppIds only need to be specified if you use the four new transactional
APIs. This is enforced at runtime.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Eugen Dueck <eu...@dueck.org>.
Thanks Apurva - replies inline.

On 2017\u5e7401\u670827\u65e5 15:19, Apurva Mehta wrote:
> Eugen, moving your email to the main thread so that it doesn't get split.
>
> The `transaction.app.id` is a prerequisite for using transactional APIs.
> And only messages wrapped inside transactions will enjoy idempotent
> guarantees across sessions, and that too only when they employ a
> consume-process-produce pattern.

Say I have a producer, producing messages into a topic and I only want 
to guarantee the producer cannot insert duplicates. In other words, 
there's no downstream consumer/processor to be worried about - which, 
when considering the correctness of the data only, is all I need for 
idempotent producers, as every message has a unique id (offset), so 
downstream processes can take care of exactly once processing by any 
number of means. (If you need transactional all-or-none behavior, which 
KIP-98 also addresses, that's of course a more complex story)

I was under the impression that KIP-98 would fulfill above requirement, 
i.e. the prevention of duplicate inserts of the same message into a 
topic per producer, without using transactions, and guaranteed across 
tcp connections to handle producer/broker crashes and network problems.

> In other words, producers where the `transaction.app.id` is specified will
> not enjoy idempotence across sessions unless their messages are
> transactional. ie. that the sends  are wrapped between `beginTransaction`,
> `sendOffsets`, and `commitTransaction`.

 From the KIP-98 wiki and the design document, I understand that AppIDs, 
PIDs, and sequence numbers are enforced regardless of their being 
wrapped in a transaction or not. Is that not so?

Cheers,
Eugen

> The comment about the heartbeat was just a passing comment about the fact
> that an AppId could be expired if a producer doesn't use transactions for a
> long time. We don't plan to implement heartbeats in V1, though we might in
> the future.
>
> Hope this clarified things.
>
> Regards,
> Apurva
>
>
> KIP-98 says
>>  > transaction.app.id: A unique and persistent way to identify a
>> producer. This is used to ensure idempotency and to enable transaction
>> recovery or rollback across producer sessions. This is optional: you will
>> lose cross-session guarantees if this is blank.
>> which might suggest that a producer that does not use the transactional
>> features, but does set the transaction.app.id, could get cross-session
>> idempotency. But the design document "Exactly Once Delivery and
>> Transactional Messaging in Kafka" rules that out:
>>  > For the idempotent producer (i.e., producer that do not use
>> transactional APIs), currently we do not make any cross-session guarantees
>> in any case. In the future, we can extend this guarantee by having the
>> producer to periodically send InitPIDRequest to the transaction coordinator
>> to keep the AppID from expiring, which preserves the producer's zombie
>> defence.
>> Until that point in the future, could my non-transactional producer send a
>> InitPIDRequest once and then heartbeat via BeginTxnRequest/EndTxnRequest(ABORT)
>> in intervals less than transaction.app.id.timeout.ms in order to
>> guarantee cross-session itempotency? Or is that not guaranteed because
>> "currently we do not make any cross-session guarantees in any case"? I know
>> this is would be an ugly hack.
>> I guess that is also what the recently added "Producer HeartBeat" feature
>> proposal would address - although it is described to prevent idle
>> transactional producers from having their AppIds expired.
>>
>> Related question: If KIP-98 does not make cross-session guarantees for
>> idempotent producers, is the only improvement over the current idempotency
>> situation the prevention of duplicate messages in case of a partition
>> leader migration? Because if a broker fails or the publisher fails, KIP-98
>> does not seem to change the risk of dupes for non-transactional producers.
>
>
>
>
>
>>
>> Btw: Good job! Both in terms of Kafka in general, and KIP-98 in particular
>
>
> Cheers
>
> On Wed, Jan 25, 2017 at 6:00 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
>>
>>
>> On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:
>>
>>> Hi Jun,
>>>
>>> Some answers in line.
>>>
>>>
>>> 109. Could you describe when Producer.send() will receive an Unrecognized
>>>
>>> MessageException?
>>>
>>>
>>> This exception will be thrown if the producer sends a sequence number
>>> which is greater than the sequence number expected by the broker (ie. more
>>> than 1 greater than the previously sent sequence number). This can happen
>>> in two cases:
>>>
>>> a) If there is a bug in the producer where sequence numbers are
>>> incremented more than once per message. So the producer itself will send
>>> messages with gaps in sequence numbers.
>>> b) The broker somehow lost a previous message. In a cluster configured
>>> for durability (ie. no unclean leader elections, replication factor of 3,
>>> min.isr of 2, acks=all, etc.), this should not happened.
>>>
>>> So realistically, this exception will only be thrown in clusters
>>> configured for high availability where brokers could lose messages.
>>>
>>> Becket raised the question if we should throw this exception at all in
>>> case b: it indicates a problem with a previously sent message and hence the
>>> semantics are counter intuitive. We are still discussing this point, and
>>> suggestions are most welcome!
>>>
>>>
>> I updated the KIP wiki to clarify when this exception will be raised.
>>
>> First of all, I renamed this to OutOfOrderSequenceException. Based on
>> Jay's suggestion, this is a more precise name that is easier to understand.
>>
>> Secondly, I updated the proposed API so that the send call will never
>> raise this exception directly. Instead this exception will be returned in
>> the future or passed with the callback, if any. Further, since this is a
>> fatal exception, any _future_ invocations of send() or other data
>> generating methods in the producer will raise an IllegalStateException. I
>> think this makes the semantics clearer and addresses the feedback on this
>> part of the API update.
>>
>> Thanks,
>> Apurva
>>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Jun,

Let me start picking off a some of your questions (we're giving the shadow
log suggestion a bit more thought before responding).

100. Security: It seems that if an app is mistakenly configured with the
> appId of an existing producer, it can take over the pid and prevent the
> existing app from publishing. So, I am wondering if it makes sense to add
> ACLs at the TransactionResource level just like we do for
> ConsumerGroupResource. So, a user can only do transactions under a
> particular appId if he/she has the write permission to the
> TransactionResource
> associated with the appId.


I think this makes sense in general. There are a couple points worth
mentioning:

1. Because we only use the AppID in requests to the transaction
coordinator, that's the only point at which we can do authorization in the
current proposal. It is possible for a malicious producer to hijack another
producer's PID and use it to write data. It wouldn't be able to commit or
abort transactions, but it could effectively fence the legitimate producer
from a partition by forcing an epoch bump. We could add the AppID to the
ProduceRequest schema, but we would still need to protect its binding to
the PID somehow. This is one argument in favor of dropping the PID and
using the AppID in the log message format. However, there are still ways in
the current proposal to give better protection if we added the AppID
authorization at the transaction coordinator as you suggest. Note that a
malicious producer would have to be authorized to write to the same topics
used by the transactional producer. So one way to protect those topics is
to only allow write access by the authorized transactional producers. The
transactional producers could still interfere with each other, but perhaps
that's a smaller concern (it's similar in effect to the limitations of
consumer group authorization).

2. It's a bit unfortunate that we don't have something like the consumer's
groupId to use for authorization. The AppID is really more of an instance
ID (we were reluctant to introduce any formal notion of a producer group).
I guess distributed applications could use a common prefix and a wildcard
authorization policy. I don't think we currently support general wildcards,
but that might be helpful for this use case.

-Jason

On Wed, Jan 4, 2017 at 12:55 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Jun,
>
> We had a proposal like this previously. The suppression scheme was slightly
> different. Rather than than attempting to recopy or swap, there was instead
> an aborted offset index maintained along with each segment containing a
> sequential list of aborted offsets. The filtering would happen at fetch
> time and would just ensure that fetch requests never span an aborted
> transaction. That is, if you did a fetch request which would include
> offsets 7,8,9,10,11, but offsets 7 and 10 appears in the index of aborted
> transactions, then the fetch would return 8,9 only even if there was more
> space in the fetch response. This leads to minimal overhead, but
> potentially would give back smaller fetch responses if transactions are
> being continually aborted.
>
> One downside to this approach (both your proposal and the variation that I
> just described is that it does not allow the possibility of consuming in
> transaction commit order. Consuming in transaction commit order means that
> the only delay you incur is the delay in committing a given transaction.
> Consuming in offset order means you cannot consume a given offset until ALL
> previously begun transactions are committed or aborted. KIP-98 doesn't
> propose making this change now, but since it is consumer side it is
> possible.
>
> -Jay
>
> On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Just to follow up on Radai's idea of pushing the buffering logic to the
> > broker. It may be possible to do this efficiently if we assume aborted
> > transactions are rare. The following is a draft proposal. For each
> > partition, the broker maintains the last stable offset (LSO) as described
> > in the document, and only exposes messages up to this point if the reader
> > is in the read-committed mode. When a new stable offset (NSO) is
> > determined, if there is no aborted message in this window, the broker
> > simply advances the LSO to the NSO. If there is at least one aborted
> > message, the broker first replaces the current log segment with new log
> > segments excluding the aborted messages and then advances the LSO. To
> make
> > the replacement efficient, we can replace the current log segment with 3
> > new segments: (1) a new "shadow" log segment that simply references the
> > portion of the current log segment from the beginning to the LSO, (2) a
> log
> > segment created by copying only committed messages between the LSO and
> the
> > NSO, (3) a new "shadow" log segment that references the portion of the
> > current log segment from the NSO (open ended). Note that only (2)
> involves
> > real data copying. If aborted transactions are rare, this overhead will
> be
> > insignificant. Assuming that applications typically don't abort
> > transactions, transactions will only be aborted by transaction
> coordinators
> > during hard failure of the producers, which should be rare.
> >
> > This way, the consumer library's logic will be simplified. We can still
> > expose uncommitted messages to readers in the read-uncommitted mode and
> > therefore leave the door open for speculative reader in the future.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > Hi Joel,
> > >
> > > The alternatives are embedded in the 'discussion' sections which are
> > spread
> > > throughout the google doc.
> > >
> > > Admittedly, we have not covered high level alternatives like those
> which
> > > have been brought up in this thread. In particular, having a separate
> log
> > > for transactional mesages and also having multiple producers
> participate
> > in
> > > a single transaction.
> > >
> > > This is an omission which we will correct.
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > >
> > > > >
> > > > >
> > > > > @Joel,
> > > > >
> > > > > I read over your wiki, and apart from the introduction of the
> notion
> > of
> > > > > journal partitions --whose pros and cons are already being
> > discussed--
> > > > you
> > > > > also introduce the notion of a 'producer group' which enables
> > multiple
> > > > > producers to participate in a single transaction. This is
> completely
> > > > > opposite of the model in the KIP where a transaction is defined by
> a
> > > > > producer id, and hence there is a 1-1 mapping between producers and
> > > > > transactions. Further, each producer can have exactly one in-flight
> > > > > transaction at a time in the KIP.
> > > > >
> > > >
> > > > Hi Apurva - yes I did notice those differences among other things :)
> > > BTW, I
> > > > haven't yet gone through the google-doc carefully but on a skim it
> does
> > > not
> > > > seem to contain any rejected alternatives as the wiki states.
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Jun,

We had a proposal like this previously. The suppression scheme was slightly
different. Rather than than attempting to recopy or swap, there was instead
an aborted offset index maintained along with each segment containing a
sequential list of aborted offsets. The filtering would happen at fetch
time and would just ensure that fetch requests never span an aborted
transaction. That is, if you did a fetch request which would include
offsets 7,8,9,10,11, but offsets 7 and 10 appears in the index of aborted
transactions, then the fetch would return 8,9 only even if there was more
space in the fetch response. This leads to minimal overhead, but
potentially would give back smaller fetch responses if transactions are
being continually aborted.

One downside to this approach (both your proposal and the variation that I
just described is that it does not allow the possibility of consuming in
transaction commit order. Consuming in transaction commit order means that
the only delay you incur is the delay in committing a given transaction.
Consuming in offset order means you cannot consume a given offset until ALL
previously begun transactions are committed or aborted. KIP-98 doesn't
propose making this change now, but since it is consumer side it is
possible.

-Jay

On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:

> Just to follow up on Radai's idea of pushing the buffering logic to the
> broker. It may be possible to do this efficiently if we assume aborted
> transactions are rare. The following is a draft proposal. For each
> partition, the broker maintains the last stable offset (LSO) as described
> in the document, and only exposes messages up to this point if the reader
> is in the read-committed mode. When a new stable offset (NSO) is
> determined, if there is no aborted message in this window, the broker
> simply advances the LSO to the NSO. If there is at least one aborted
> message, the broker first replaces the current log segment with new log
> segments excluding the aborted messages and then advances the LSO. To make
> the replacement efficient, we can replace the current log segment with 3
> new segments: (1) a new "shadow" log segment that simply references the
> portion of the current log segment from the beginning to the LSO, (2) a log
> segment created by copying only committed messages between the LSO and the
> NSO, (3) a new "shadow" log segment that references the portion of the
> current log segment from the NSO (open ended). Note that only (2) involves
> real data copying. If aborted transactions are rare, this overhead will be
> insignificant. Assuming that applications typically don't abort
> transactions, transactions will only be aborted by transaction coordinators
> during hard failure of the producers, which should be rare.
>
> This way, the consumer library's logic will be simplified. We can still
> expose uncommitted messages to readers in the read-uncommitted mode and
> therefore leave the door open for speculative reader in the future.
>
> Thanks,
>
> Jun
>
>
> On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
>
> > Hi Joel,
> >
> > The alternatives are embedded in the 'discussion' sections which are
> spread
> > throughout the google doc.
> >
> > Admittedly, we have not covered high level alternatives like those which
> > have been brought up in this thread. In particular, having a separate log
> > for transactional mesages and also having multiple producers participate
> in
> > a single transaction.
> >
> > This is an omission which we will correct.
> >
> > Thanks,
> > Apurva
> >
> > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> >
> > > >
> > > >
> > > > @Joel,
> > > >
> > > > I read over your wiki, and apart from the introduction of the notion
> of
> > > > journal partitions --whose pros and cons are already being
> discussed--
> > > you
> > > > also introduce the notion of a 'producer group' which enables
> multiple
> > > > producers to participate in a single transaction. This is completely
> > > > opposite of the model in the KIP where a transaction is defined by a
> > > > producer id, and hence there is a 1-1 mapping between producers and
> > > > transactions. Further, each producer can have exactly one in-flight
> > > > transaction at a time in the KIP.
> > > >
> > >
> > > Hi Apurva - yes I did notice those differences among other things :)
> > BTW, I
> > > haven't yet gone through the google-doc carefully but on a skim it does
> > not
> > > seem to contain any rejected alternatives as the wiki states.
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jun Rao <ju...@confluent.io>.
Hi, Jason,

Thanks for the reply. They sound good to me.

Jun

On Fri, Jan 27, 2017 at 4:42 PM, Jason Gustafson <ja...@confluent.io> wrote:

> A few more responses:
>
>
> > 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> to
> > the new version, but the broker message format is still the old one. If a
> > new producer uses the transaction feature, should the producer get an
> error
> > in this case? A tricky case can be that the leader broker is on the new
> > message format, but the follower broker is still on the old message
> format.
> > In this case, the transactional info will be lost in the follower due to
> > down conversion. Should we failed the transactional requests when the
> > followers are still on the old message format?
>
>
> We've added some more details to the document about migration. Please take
> a look. Two points worth mentioning:
>
> 1. Replicas currently take the message format used by the leader. As long
> as users do the usual procedure of two rolling bounces, it should be safe
> to upgrade the message format.
>
> 2. There is no way to support idempotent or transactional features if we
> downgrade the message format in the produce request handler. We've modified
> the design document to only permit message downgrades if the producer has
> disabled idempotence. Otherwise, we will return an
> UNSUPPORTED_FOR_MESSAGE_FORMAT error.
>
> 110. Transaction log:
> > 110.1 "Key => Version AppID Version" It seems that Version should really
> be
> > Type?
> > 110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
> > Should we store [Topic [Partition]] instead?
> > 110.3 To expire an AppId, do we need to insert a tombstone with the
> expired
> > AppID as the key to physically remove the existing AppID entries in the
> > transaction log?
>
>
> Fixed in the document. For 110.3, yes, we need to insert a tombstone after
> the AppID has expired. This will work in much the same way as the consumer
> coordinator expires offsets using a periodic task.
>
> 116. ProducerRequest: The existing format doesn't have "MessageSetSize" at
> > the partition level.
>
>
> This was intentional, but it is easy to overlook. The idea is to modify the
> ProduceRequest so that only one message set is included for each partition.
> Since the message set contains its own length field, it seemed unnecessary
> to have a separate field. The justification for this change was to make the
> produce request atomic. With only a single message set for each partition,
> either it will be written successfully or not, so an error in the response
> will be unambiguous. We are uncertain whether there are legitimate use
> cases that require producing smaller message sets in the ProduceRequest, so
> we would love to hear feedback on this.
>
> Thanks,
> Jason
>
> On Fri, Jan 27, 2017 at 4:21 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi again Jun,
> >
> > I have update the document to address your comments below, but including
> > the responses inline to make it easier for everyone to stay on top of the
> > conversation.
> >
> >
> >
> > > 106. Compacted topics.
> > > 106.1. When all messages in a transaction are removed, we could remove
> > the
> > > commit/abort marker for that transaction too. However, we have to be a
> > bit
> > > careful. If the marker is removed too quickly, it's possible for a
> > consumer
> > > to see a message in that transaction, but not to see the marker, and
> > > therefore will be stuck in that transaction forever. We have a similar
> > > issue when dealing with tombstones. The solution is to preserve the
> > > tombstone for at least a preconfigured amount of time after the
> cleaning
> > > has passed the tombstone. Then, as long as a consumer can finish
> reading
> > to
> > > the cleaning point within the configured amount of time, it's
> guaranteed
> > > not to miss the tombstone after it has seen a non-tombstone message on
> > the
> > > same key. I am wondering if we should do something similar here.
> > >
> >
> > This is a good point. As we discussed offline, the solution for the
> removal
> > of control messages will be the same as the solution for problem of
> > tombstone removal documented in
> > https://issues.apache.org/jira/browse/KAFKA-4545.
> >
> > 106.2. "To address this problem, we propose to preserve the last epoch
> and
> > > sequence number written by each producer for a fixed amount of time as
> an
> > > empty message set. This is allowed by the new message format we are
> > > proposing in this document. The time to preserve the sequence number
> will
> > > be governed by the log retention settings. " Could you be a bit more
> > > specific on what retention time will be used since by default, there is
> > no
> > > retention time for compacted (but not delete) topic?
> > >
> >
> > We discussed this offline, and the consensus that it is reasonable to use
> > brokers global log.retention.* settings for these messages.
> >
> >
> > > 106.3 "As for control messages, if the broker does not have any
> > > corresponding transaction cached with the PID when encountering a
> control
> > > message, that message can be safely removed."
> > > Do controlled messages have keys? If not, do we need to relax the
> >
> > constraint that messages in a compacted topic must have keys?
> > >
> >
> > The key of a control messages is the control message type. As such,
> regular
> > compaction logic based on key will not apply to control messages. We will
> > have to update the log cleaner to ignore messages which have the control
> > message bit set.
> >
> > Control messages can be removed at some point after the last messages of
> > the corresponding transaction are removed. As suggested in KAFKA-4545, we
> > can use the timestamp associated with the log segment to deduce the safe
> > expiration time for control messages in that segment.
> >
> >
> >
> > > 112. Control message: Will control messages be used for timestamp
> > indexing?
> > > If so, what timestamp will we use if the timestamp type is creation
> time?
> > >
> > >
> > Control messages will not be used for timestamp indexing. Each control
> > message will have the log append time for the timestamp, but these
> messages
> > will be ignored when building the timestamp index. Since control messages
> > are for system use only and will never be exposed to users, it doesn't
> make
> > sense to include them in the timestamp index.
> >
> > Further, as you mentioned, when a topic uses creation time, it is
> > impossible to ensure that control messages will not skew the time based
> > index, since these messages are sent by the transaction coordinator which
> > has no notion of the application level message creation time.
> >
> > Thanks,
> > Apurva
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
A few more responses:


> 101. Compatibility during upgrade: Suppose that the brokers are upgraded to
> the new version, but the broker message format is still the old one. If a
> new producer uses the transaction feature, should the producer get an error
> in this case? A tricky case can be that the leader broker is on the new
> message format, but the follower broker is still on the old message format.
> In this case, the transactional info will be lost in the follower due to
> down conversion. Should we failed the transactional requests when the
> followers are still on the old message format?


We've added some more details to the document about migration. Please take
a look. Two points worth mentioning:

1. Replicas currently take the message format used by the leader. As long
as users do the usual procedure of two rolling bounces, it should be safe
to upgrade the message format.

2. There is no way to support idempotent or transactional features if we
downgrade the message format in the produce request handler. We've modified
the design document to only permit message downgrades if the producer has
disabled idempotence. Otherwise, we will return an
UNSUPPORTED_FOR_MESSAGE_FORMAT error.

110. Transaction log:
> 110.1 "Key => Version AppID Version" It seems that Version should really be
> Type?
> 110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
> Should we store [Topic [Partition]] instead?
> 110.3 To expire an AppId, do we need to insert a tombstone with the expired
> AppID as the key to physically remove the existing AppID entries in the
> transaction log?


Fixed in the document. For 110.3, yes, we need to insert a tombstone after
the AppID has expired. This will work in much the same way as the consumer
coordinator expires offsets using a periodic task.

116. ProducerRequest: The existing format doesn't have "MessageSetSize" at
> the partition level.


This was intentional, but it is easy to overlook. The idea is to modify the
ProduceRequest so that only one message set is included for each partition.
Since the message set contains its own length field, it seemed unnecessary
to have a separate field. The justification for this change was to make the
produce request atomic. With only a single message set for each partition,
either it will be written successfully or not, so an error in the response
will be unambiguous. We are uncertain whether there are legitimate use
cases that require producing smaller message sets in the ProduceRequest, so
we would love to hear feedback on this.

Thanks,
Jason

On Fri, Jan 27, 2017 at 4:21 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi again Jun,
>
> I have update the document to address your comments below, but including
> the responses inline to make it easier for everyone to stay on top of the
> conversation.
>
>
>
> > 106. Compacted topics.
> > 106.1. When all messages in a transaction are removed, we could remove
> the
> > commit/abort marker for that transaction too. However, we have to be a
> bit
> > careful. If the marker is removed too quickly, it's possible for a
> consumer
> > to see a message in that transaction, but not to see the marker, and
> > therefore will be stuck in that transaction forever. We have a similar
> > issue when dealing with tombstones. The solution is to preserve the
> > tombstone for at least a preconfigured amount of time after the cleaning
> > has passed the tombstone. Then, as long as a consumer can finish reading
> to
> > the cleaning point within the configured amount of time, it's guaranteed
> > not to miss the tombstone after it has seen a non-tombstone message on
> the
> > same key. I am wondering if we should do something similar here.
> >
>
> This is a good point. As we discussed offline, the solution for the removal
> of control messages will be the same as the solution for problem of
> tombstone removal documented in
> https://issues.apache.org/jira/browse/KAFKA-4545.
>
> 106.2. "To address this problem, we propose to preserve the last epoch and
> > sequence number written by each producer for a fixed amount of time as an
> > empty message set. This is allowed by the new message format we are
> > proposing in this document. The time to preserve the sequence number will
> > be governed by the log retention settings. " Could you be a bit more
> > specific on what retention time will be used since by default, there is
> no
> > retention time for compacted (but not delete) topic?
> >
>
> We discussed this offline, and the consensus that it is reasonable to use
> brokers global log.retention.* settings for these messages.
>
>
> > 106.3 "As for control messages, if the broker does not have any
> > corresponding transaction cached with the PID when encountering a control
> > message, that message can be safely removed."
> > Do controlled messages have keys? If not, do we need to relax the
>
> constraint that messages in a compacted topic must have keys?
> >
>
> The key of a control messages is the control message type. As such, regular
> compaction logic based on key will not apply to control messages. We will
> have to update the log cleaner to ignore messages which have the control
> message bit set.
>
> Control messages can be removed at some point after the last messages of
> the corresponding transaction are removed. As suggested in KAFKA-4545, we
> can use the timestamp associated with the log segment to deduce the safe
> expiration time for control messages in that segment.
>
>
>
> > 112. Control message: Will control messages be used for timestamp
> indexing?
> > If so, what timestamp will we use if the timestamp type is creation time?
> >
> >
> Control messages will not be used for timestamp indexing. Each control
> message will have the log append time for the timestamp, but these messages
> will be ignored when building the timestamp index. Since control messages
> are for system use only and will never be exposed to users, it doesn't make
> sense to include them in the timestamp index.
>
> Further, as you mentioned, when a topic uses creation time, it is
> impossible to ensure that control messages will not skew the time based
> index, since these messages are sent by the transaction coordinator which
> has no notion of the application level message creation time.
>
> Thanks,
> Apurva
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi again Jun,

I have update the document to address your comments below, but including
the responses inline to make it easier for everyone to stay on top of the
conversation.



> 106. Compacted topics.
> 106.1. When all messages in a transaction are removed, we could remove the
> commit/abort marker for that transaction too. However, we have to be a bit
> careful. If the marker is removed too quickly, it's possible for a consumer
> to see a message in that transaction, but not to see the marker, and
> therefore will be stuck in that transaction forever. We have a similar
> issue when dealing with tombstones. The solution is to preserve the
> tombstone for at least a preconfigured amount of time after the cleaning
> has passed the tombstone. Then, as long as a consumer can finish reading to
> the cleaning point within the configured amount of time, it's guaranteed
> not to miss the tombstone after it has seen a non-tombstone message on the
> same key. I am wondering if we should do something similar here.
>

This is a good point. As we discussed offline, the solution for the removal
of control messages will be the same as the solution for problem of
tombstone removal documented in
https://issues.apache.org/jira/browse/KAFKA-4545.

106.2. "To address this problem, we propose to preserve the last epoch and
> sequence number written by each producer for a fixed amount of time as an
> empty message set. This is allowed by the new message format we are
> proposing in this document. The time to preserve the sequence number will
> be governed by the log retention settings. " Could you be a bit more
> specific on what retention time will be used since by default, there is no
> retention time for compacted (but not delete) topic?
>

We discussed this offline, and the consensus that it is reasonable to use
brokers global log.retention.* settings for these messages.


> 106.3 "As for control messages, if the broker does not have any
> corresponding transaction cached with the PID when encountering a control
> message, that message can be safely removed."
> Do controlled messages have keys? If not, do we need to relax the

constraint that messages in a compacted topic must have keys?
>

The key of a control messages is the control message type. As such, regular
compaction logic based on key will not apply to control messages. We will
have to update the log cleaner to ignore messages which have the control
message bit set.

Control messages can be removed at some point after the last messages of
the corresponding transaction are removed. As suggested in KAFKA-4545, we
can use the timestamp associated with the log segment to deduce the safe
expiration time for control messages in that segment.



> 112. Control message: Will control messages be used for timestamp indexing?
> If so, what timestamp will we use if the timestamp type is creation time?
>
>
Control messages will not be used for timestamp indexing. Each control
message will have the log append time for the timestamp, but these messages
will be ignored when building the timestamp index. Since control messages
are for system use only and will never be exposed to users, it doesn't make
sense to include them in the timestamp index.

Further, as you mentioned, when a topic uses creation time, it is
impossible to ensure that control messages will not skew the time based
index, since these messages are sent by the transaction coordinator which
has no notion of the application level message creation time.

Thanks,
Apurva

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jun,

Thanks for the comments!! Some responses below:

*> 100. Security: **We could also include AppId in produce request..*

On brokers only PIDs are maintained and they are unaware of the AppIds, so
I think it would be costly to prevent writes on the AppId level. On the
other hand, having security based on AppId for transactional requests
trying to write to the transaction logs seems sufficient to me, since
producers always need to talk to the transaction coordinator first in order
to send data to partition leaders.


*> 101. A tricky case can be that the leader broker is on the new message
format, but the follower broker is still on the old message format...*

The upgrade path has been updated in the wiki page
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-Compatibility,Deprecation,andMigrationPlan>.
Note that we will only let clients to start using the idempotent /
transactional features after the whole cluster has completed upgrading
(both inter-broker protocol and message format). But to reduce temporary
performance degradation we can consider letting clients to upgrade without
using the new features so that they will send / consume data following the
new message format, as indicated in step 3.


*> 102. When there is a correlated hard failure (e.g., power outage),
it's possible that an existing commit/abort marker is lost in all
replicas...*

As Apurva mentioned, we can provide an admin tool to let operators to fix
such issues when correlated hard failure happens.

Another potential solution is to let brokers to fsync on transaction
boundaries (i.e. when the markers are being written), so that the
likelihood of such hard failures causing markers to be completely lost can
be reduced.


*> 105. When the transaction coordinator changes (due to leadership
changes), it's possible for both the old and the new coordinator sending
requests to a broker at the same time (for a short period of time)...*

This is a good question. We have updated the design doc to add a
coordinator epoch in the WriteTxnMarkerRequest as well as added it in the
transaction message's value payload and the PID snapshot file (see here
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.ptdscx8pzota>
for details).
The coordinator epoch corresponds to the transaction log's leader epoch.


*> 107. Could you include the default values for the newly introduced
configs?*

Have updated the design doc with the default values of newly added configs,
see here
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.78p7cgjqcbnx>,
here
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.iixbdsg65d7k>
and here
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.wvdrakld4019>
.


*> 117. UpdateTxnRequest: Could you explain the format of Marker?*

Note that we have renamed UpdateTxnRequest to WriteTxnMarkerRequest to be
more specific.
We have update the doc
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.jtpvkrldhb7>
to include its current possible values.


*> 118. TxnOffsetCommitRequest: How is retention time determined? Do we
need a new config in producer or just default it to -1 as the consumer?*

-1 will be used as the consumer. Corresponding section
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.5695qbm2hne>
is updated.


*> 121. The ordering is important with idempotent producer, which
means that max.in.flight.requests.per.connection should be set to 1. Do we
want to enforce this?*

I think it is actually not necessary, since the brokers will "strictly"
check the sequence number that must be current sequence + 1, so as long as
the first request fails, the rest will doom to fail as well.


*> 122. Currently, since we don't know the number of messages in a
compressed set, to finish the iteration, we rely on catching EOF in the
decompressor, which adds a bit overhead in the consumer.*

The logic is not to only relying on catching EOF, but also depending on the
offsetDelta to determine the "higher bound" of the number of messages. So
only if log compaction is triggered and the last message(s) are compacted,
then we need to rely on catching EOFs, whose cost would be much less than
KAFKA-4293.


*> 123. I am wondering if the coordinator needs to add a "BEGIN
transaction message" on a BeginTxnRequest. **Could we just wait until an
AddPartitionsToTxnRequest?*

It is possible, though no likely, that a client sends an
AddOffsetsToTxnRequest right after a BeginTxnRequest, in this case we need
to make sure that there is already an on-going transaction.



Guozhang


On Wed, Jan 25, 2017 at 6:00 PM, Apurva Mehta <ap...@confluent.io> wrote:

> On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Jun,
> >
> > Some answers in line.
> >
> >
> > 109. Could you describe when Producer.send() will receive an Unrecognized
> >
> > MessageException?
> >
> >
> > This exception will be thrown if the producer sends a sequence number
> > which is greater than the sequence number expected by the broker (ie.
> more
> > than 1 greater than the previously sent sequence number). This can happen
> > in two cases:
> >
> > a) If there is a bug in the producer where sequence numbers are
> > incremented more than once per message. So the producer itself will send
> > messages with gaps in sequence numbers.
> > b) The broker somehow lost a previous message. In a cluster configured
> for
> > durability (ie. no unclean leader elections, replication factor of 3,
> > min.isr of 2, acks=all, etc.), this should not happened.
> >
> > So realistically, this exception will only be thrown in clusters
> > configured for high availability where brokers could lose messages.
> >
> > Becket raised the question if we should throw this exception at all in
> > case b: it indicates a problem with a previously sent message and hence
> the
> > semantics are counter intuitive. We are still discussing this point, and
> > suggestions are most welcome!
> >
> >
> I updated the KIP wiki to clarify when this exception will be raised.
>
> First of all, I renamed this to OutOfOrderSequenceException. Based on Jay's
> suggestion, this is a more precise name that is easier to understand.
>
> Secondly, I updated the proposed API so that the send call will never raise
> this exception directly. Instead this exception will be returned in the
> future or passed with the callback, if any. Further, since this is a fatal
> exception, any _future_ invocations of send() or other data generating
> methods in the producer will raise an IllegalStateException. I think this
> makes the semantics clearer and addresses the feedback on this part of the
> API update.
>
> Thanks,
> Apurva
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Eugen, moving your email to the main thread so that it doesn't get split.

The `transaction.app.id` is a prerequisite for using transactional APIs.
And only messages wrapped inside transactions will enjoy idempotent
guarantees across sessions, and that too only when they employ a
consume-process-produce pattern.

In other words, producers where the `transaction.app.id` is specified will
not enjoy idempotence across sessions unless their messages are
transactional. ie. that the sends  are wrapped between `beginTransaction`,
`sendOffsets`, and `commitTransaction`.

The comment about the heartbeat was just a passing comment about the fact
that an AppId could be expired if a producer doesn't use transactions for a
long time. We don't plan to implement heartbeats in V1, though we might in
the future.

Hope this clarified things.

Regards,
Apurva


KIP-98 says
>  > transaction.app.id: A unique and persistent way to identify a
> producer. This is used to ensure idempotency and to enable transaction
> recovery or rollback across producer sessions. This is optional: you will
> lose cross-session guarantees if this is blank.
> which might suggest that a producer that does not use the transactional
> features, but does set the transaction.app.id, could get cross-session
> idempotency. But the design document "Exactly Once Delivery and
> Transactional Messaging in Kafka" rules that out:
>  > For the idempotent producer (i.e., producer that do not use
> transactional APIs), currently we do not make any cross-session guarantees
> in any case. In the future, we can extend this guarantee by having the
> producer to periodically send InitPIDRequest to the transaction coordinator
> to keep the AppID from expiring, which preserves the producer's zombie
> defence.
> Until that point in the future, could my non-transactional producer send a
> InitPIDRequest once and then heartbeat via BeginTxnRequest/EndTxnRequest(ABORT)
> in intervals less than transaction.app.id.timeout.ms in order to
> guarantee cross-session itempotency? Or is that not guaranteed because
> "currently we do not make any cross-session guarantees in any case"? I know
> this is would be an ugly hack.
> I guess that is also what the recently added "Producer HeartBeat" feature
> proposal would address - although it is described to prevent idle
> transactional producers from having their AppIds expired.
>
> Related question: If KIP-98 does not make cross-session guarantees for
> idempotent producers, is the only improvement over the current idempotency
> situation the prevention of duplicate messages in case of a partition
> leader migration? Because if a broker fails or the publisher fails, KIP-98
> does not seem to change the risk of dupes for non-transactional producers.





>
> Btw: Good job! Both in terms of Kafka in general, and KIP-98 in particular


Cheers

On Wed, Jan 25, 2017 at 6:00 PM, Apurva Mehta <ap...@confluent.io> wrote:

>
>
> On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Hi Jun,
>>
>> Some answers in line.
>>
>>
>> 109. Could you describe when Producer.send() will receive an Unrecognized
>>
>> MessageException?
>>
>>
>> This exception will be thrown if the producer sends a sequence number
>> which is greater than the sequence number expected by the broker (ie. more
>> than 1 greater than the previously sent sequence number). This can happen
>> in two cases:
>>
>> a) If there is a bug in the producer where sequence numbers are
>> incremented more than once per message. So the producer itself will send
>> messages with gaps in sequence numbers.
>> b) The broker somehow lost a previous message. In a cluster configured
>> for durability (ie. no unclean leader elections, replication factor of 3,
>> min.isr of 2, acks=all, etc.), this should not happened.
>>
>> So realistically, this exception will only be thrown in clusters
>> configured for high availability where brokers could lose messages.
>>
>> Becket raised the question if we should throw this exception at all in
>> case b: it indicates a problem with a previously sent message and hence the
>> semantics are counter intuitive. We are still discussing this point, and
>> suggestions are most welcome!
>>
>>
> I updated the KIP wiki to clarify when this exception will be raised.
>
> First of all, I renamed this to OutOfOrderSequenceException. Based on
> Jay's suggestion, this is a more precise name that is easier to understand.
>
> Secondly, I updated the proposed API so that the send call will never
> raise this exception directly. Instead this exception will be returned in
> the future or passed with the callback, if any. Further, since this is a
> fatal exception, any _future_ invocations of send() or other data
> generating methods in the producer will raise an IllegalStateException. I
> think this makes the semantics clearer and addresses the feedback on this
> part of the API update.
>
> Thanks,
> Apurva
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Jun,
>
> Some answers in line.
>
>
> 109. Could you describe when Producer.send() will receive an Unrecognized
>
> MessageException?
>
>
> This exception will be thrown if the producer sends a sequence number
> which is greater than the sequence number expected by the broker (ie. more
> than 1 greater than the previously sent sequence number). This can happen
> in two cases:
>
> a) If there is a bug in the producer where sequence numbers are
> incremented more than once per message. So the producer itself will send
> messages with gaps in sequence numbers.
> b) The broker somehow lost a previous message. In a cluster configured for
> durability (ie. no unclean leader elections, replication factor of 3,
> min.isr of 2, acks=all, etc.), this should not happened.
>
> So realistically, this exception will only be thrown in clusters
> configured for high availability where brokers could lose messages.
>
> Becket raised the question if we should throw this exception at all in
> case b: it indicates a problem with a previously sent message and hence the
> semantics are counter intuitive. We are still discussing this point, and
> suggestions are most welcome!
>
>
I updated the KIP wiki to clarify when this exception will be raised.

First of all, I renamed this to OutOfOrderSequenceException. Based on Jay's
suggestion, this is a more precise name that is easier to understand.

Secondly, I updated the proposed API so that the send call will never raise
this exception directly. Instead this exception will be returned in the
future or passed with the callback, if any. Further, since this is a fatal
exception, any _future_ invocations of send() or other data generating
methods in the producer will raise an IllegalStateException. I think this
makes the semantics clearer and addresses the feedback on this part of the
API update.

Thanks,
Apurva

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Quick update: I have merged the abort index proposal linked above into the
main design document. We are now working on tying up the loose ends raised
by Jun and others.

Thanks,
Jason

On Tue, Jan 17, 2017 at 8:53 PM, Apurva Mehta <ap...@confluent.io> wrote:

> >
> > 114.The section on Discussion on Pro-active Transaction Timeout: "If
> there
> > is no other instance with the same PID yet, or it has not started a
> > transaction, or it has not appended a message to some of the partitions,
> > then the zombie can continue appending messages to those partitions after
> > the abort marker whose epoch has not been incremented yet, but its
> > commitTxn call will fail.”
>
> Is that correct, in earlier discussion, it seems that if a transaction is
> > timed out by the coordinator, the coordinator will bump up epoch and
> write
> > the abort marker to those inserted partitions.
>
>
> You are correct. The zombie producer will not be able to write to any
> partitions after the abort because the epoch on each broker would have been
> bumped by the abort message, causing the zombie to be fenced of totally.
> Will correct the doc.
>
> Thanks,
> Apurva
>
> On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Jun,
> >
> > Some answers in line.
> >
> >
> > 109. Could you describe when Producer.send() will receive an Unrecognized
> >
> > MessageException?
> >
> >
> > This exception will be thrown if the producer sends a sequence number
> > which is greater than the sequence number expected by the broker (ie.
> more
> > than 1 greater than the previously sent sequence number). This can happen
> > in two cases:
> >
> > a) If there is a bug in the producer where sequence numbers are
> > incremented more than once per message. So the producer itself will send
> > messages with gaps in sequence numbers.
> > b) The broker somehow lost a previous message. In a cluster configured
> for
> > durability (ie. no unclean leader elections, replication factor of 3,
> > min.isr of 2, acks=all, etc.), this should not happened.
> >
> > So realistically, this exception will only be thrown in clusters
> > configured for high availability where brokers could lose messages.
> >
> > Becket raised the question if we should throw this exception at all in
> > case b: it indicates a problem with a previously sent message and hence
> the
> > semantics are counter intuitive. We are still discussing this point, and
> > suggestions are most welcome!
> >
> >
> >> 111. Transaction coordinator startup: "Verify that there is already an
> >> entry with the PID in the AppID map. If there is not, raise an
> exception."
> >> For completed transactions, it seems that it's possible that their
> >> AppId->pid has been compacted out. But that shouldn't trigger an
> >> exception?
> >
> >
> > This exception will only be raised if the coordinator encounters
> > transaction status message in the log (Begin, AddTPToTransaction,
> Prepare,
> > Commit/Abort). We will compact out the AppId->PID mapping along with the
> > transaction status messages for that PID, so we should not encounter one
> > without the other. We will clarify that status messages for completed
> > transactions can be compacted out aggressively.
> >
> > 113. Zombie producer:
> >> "If the zombie has an ongoing transaction with its old PID while its
> AppID
> >> is being expired by the coordinator, by the time the zombie is about to
> >> commit the transaction it needs to talk to coordinator again and will be
> >> notified its PID is unrecognized and hence need to re-register its AppID
> >> with the InitPIDRequest. At this time, if there is already another
> >> registered producer with the same AppID, then this request will be
> >> rejected
> >> with the fatal ProducerFenced error code."
> >> Is that right? According the the coordinator request handling logic, it
> >> seems that the InitPIDRequest will bump up the epoch of the pid and
> >> succeed?
> >
> >
> > This is a good point. The InitPIDRequest will not fail, but will fence
> off
> > the other producer. In this case, the CommitTxn should fail, since there
> > would be no ongoing transaction for the PID/Epoch pair. I will update the
> > document to reflect this.
> >
> >
> > On Wed, Jan 4, 2017 at 10:54 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> >> Thanks for the proposal. A few more detailed comments.
> >>
> >> 100. Security: It seems that if an app is mistakenly configured with the
> >> appId of an existing producer, it can take over the pid and prevent the
> >> existing app from publishing. So, I am wondering if it makes sense to
> add
> >> ACLs at the TransactionResource level just like we do for
> >> ConsumerGroupResource. So, a user can only do transactions under a
> >> particular appId if he/she has the write permission to the
> >> TransactionResource
> >> associated with the appId.
> >>
> >> 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> >> to
> >> the new version, but the broker message format is still the old one. If
> a
> >> new producer uses the transaction feature, should the producer get an
> >> error
> >> in this case? A tricky case can be that the leader broker is on the new
> >> message format, but the follower broker is still on the old message
> >> format.
> >> In this case, the transactional info will be lost in the follower due to
> >> down conversion. Should we failed the transactional requests when the
> >> followers are still on the old message format?
> >>
> >> 102. When there is a correlated hard failure (e.g., power outage), it's
> >> possible that an existing commit/abort marker is lost in all replicas.
> >> This
> >> may not be fixed by the transaction coordinator automatically and the
> >> consumer may get stuck on that incomplete transaction forever. Not sure
> >> what's the best way to address this. Perhaps, one way is to run a tool
> to
> >> add an abort maker for all pids in all affected partitions.
> >>
> >> 103. Currently, there is no check for producer liveness. This means that
> >> if
> >> a producer has not been sending transactional requests for a long time,
> >> its
> >> appId will be expired by the coordinator. Have we considered having
> >> producers sending heartbeatRequest just like the consumer to keep it
> >> alive?
> >>
> >> 104. The logic for handling follower truncation can be a bit tricker
> now.
> >> The truncation may rewind the sequence number for some pids. The
> question
> >> is how to quickly recover the last sequence number of those pids. Do we
> >> plan to reload from a PID snapshot and scan forward?
> >>
> >> 105. When the transaction coordinator changes (due to leadership
> changes),
> >> it's possible for both the old and the new coordinator sending requests
> to
> >> a broker at the same time (for a short period of time). I am wondering
> if
> >> we need to add logic to fence off the old coordinator. One way to do
> that
> >> is to include the leaderEpoch of the partition associated with the
> >> coordinator in the coordinator to broker request and control messages.
> >>
> >> 106. Compacted topics.
> >> 106.1. When all messages in a transaction are removed, we could remove
> the
> >> commit/abort marker for that transaction too. However, we have to be a
> bit
> >> careful. If the marker is removed too quickly, it's possible for a
> >> consumer
> >> to see a message in that transaction, but not to see the marker, and
> >> therefore will be stuck in that transaction forever. We have a similar
> >> issue when dealing with tombstones. The solution is to preserve the
> >> tombstone for at least a preconfigured amount of time after the cleaning
> >> has passed the tombstone. Then, as long as a consumer can finish reading
> >> to
> >> the cleaning point within the configured amount of time, it's guaranteed
> >> not to miss the tombstone after it has seen a non-tombstone message on
> the
> >> same key. I am wondering if we should do something similar here.
> >> 106.2. "To address this problem, we propose to preserve the last epoch
> and
> >> sequence number written by each producer for a fixed amount of time as
> an
> >> empty message set. This is allowed by the new message format we are
> >> proposing in this document. The time to preserve the sequence number
> will
> >> be governed by the log retention settings. " Could you be a bit more
> >> specific on what retention time will be used since by default, there is
> no
> >> retention time for compacted (but not delete) topic?
> >> 106.3 "As for control messages, if the broker does not have any
> >> corresponding transaction cached with the PID when encountering a
> control
> >> message, that message can be safely removed."
> >> Do controlled messages have keys? If not, do we need to relax the
> >> constraint that messages in a compacted topic must have keys?
> >>
> >> 107. Could you include the default values for the newly introduced
> >> configs?
> >>
> >> 108. Could you describe the format of the PID snapshot file?
> >>
> >> 109. Could you describe when Producer.send() will receive an
> UnrecognizedM
> >> essageException?
> >>
> >> 110. Transaction log:
> >> 110.1 "Key => Version AppID Version" It seems that Version should really
> >> be
> >> Type?
> >> 110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
> >> Should we store [Topic [Partition]] instead?
> >> 110.3 To expire an AppId, do we need to insert a tombstone with the
> >> expired
> >> AppID as the key to physically remove the existing AppID entries in the
> >> transaction log?
> >>
> >> 111. Transaction coordinator startup: "Verify that there is already an
> >> entry with the PID in the AppID map. If there is not, raise an
> exception."
> >> For completed transactions, it seems that it's possible that their
> >> AppId->pid has been compacted out. But that shouldn't trigger an
> >> exception?
> >>
> >> 112. Control message: Will control messages be used for timestamp
> >> indexing?
> >> If so, what timestamp will we use if the timestamp type is creation
> time?
> >>
> >> 113. Zombie producer:
> >> "If the zombie has an ongoing transaction with its old PID while its
> AppID
> >> is being expired by the coordinator, by the time the zombie is about to
> >> commit the transaction it needs to talk to coordinator again and will be
> >> notified its PID is unrecognized and hence need to re-register its AppID
> >> with the InitPIDRequest. At this time, if there is already another
> >> registered producer with the same AppID, then this request will be
> >> rejected
> >> with the fatal ProducerFenced error code."
> >> Is that right? According the the coordinator request handling logic, it
> >> seems that the InitPIDRequest will bump up the epoch of the pid and
> >> succeed?
> >>
> >> 114.The section on Discussion on Pro-active Transaction Timeout: "If
> there
> >> is no other instance with the same PID yet, or it has not started a
> >> transaction, or it has not appended a message to some of the partitions,
> >> then the zombie can continue appending messages to those partitions
> after
> >> the abort marker whose epoch has not been incremented yet, but its
> >> commitTxn
> >> call will fail."
> >> Is that correct, in earlier discussion, it seems that if a transaction
> is
> >> timed out by the coordinator, the coordinator will bump up epoch and
> write
> >> the abort marker to those inserted partitions.
> >>
> >> 115. Message format:
> >> 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> >> since it's possible for a producer to be restarted 10s thousands of
> times?
> >> 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> >> producer to publish more than 2 billion messages in a session.
> >> 115.3 "Null-value bit is 1: skip the key-length (since it can now be
> >> calculated) and value fields." It seems that it's unnatural for the
> format
> >> of key to depend on value. It seems it's easier to just skip value in
> this
> >> case?
> >>
> >> 116. ProducerRequest: The existing format doesn't have "MessageSetSize"
> at
> >> the partition level.
> >>
> >> 117. UpdateTxnRequest: Could you explain the format of Marker?
> >>
> >> 118. TxnOffsetCommitRequest: How is retention time determined? Do we
> need
> >> a
> >> new config in producer or just default it to -1 as the consumer?
> >>
> >> 119. InitPIDRequest
> >> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7
> >> CP1SnQGdp2eF0wSw9ra8/edit#heading=h.z99xar1h2enr>
> >> : Should we write the completion of open transactions before append the
> >> pid
> >> with bumped up epoch to the transaction log?
> >>
> >> 120. transaction.app.id: An app may have multiple concurrent instances.
> >> Perhaps we should name it transaction.instance.id or just instance.id?
> >>
> >> 121. The ordering is important with idempotent producer, which means
> >> that max.in.flight.requests.per.connection should be set to 1. Do we
> want
> >> to enforce this?
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >>
> >> On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com>
> wrote:
> >>
> >> > @jun - good proposal. i was willing to concede that read-uncommitted
> was
> >> > impossible under my proposal but if LSO/NSO is introduced is becomes
> >> > possible.
> >>
> >> >
> >> >
> >> > On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Just to follow up on Radai's idea of pushing the buffering logic to
> >> the
> >> > > broker. It may be possible to do this efficiently if we assume
> aborted
> >> > > transactions are rare. The following is a draft proposal. For each
> >> > > partition, the broker maintains the last stable offset (LSO) as
> >> described
> >> > > in the document, and only exposes messages up to this point if the
> >> reader
> >> > > is in the read-committed mode. When a new stable offset (NSO) is
> >> > > determined, if there is no aborted message in this window, the
> broker
> >> > > simply advances the LSO to the NSO. If there is at least one aborted
> >> > > message, the broker first replaces the current log segment with new
> >> log
> >> > > segments excluding the aborted messages and then advances the LSO.
> To
> >> > make
> >> > > the replacement efficient, we can replace the current log segment
> >> with 3
> >> > > new segments: (1) a new "shadow" log segment that simply references
> >> the
> >> > > portion of the current log segment from the beginning to the LSO,
> (2)
> >> a
> >> > log
> >> > > segment created by copying only committed messages between the LSO
> and
> >> > the
> >> > > NSO, (3) a new "shadow" log segment that references the portion of
> the
> >> > > current log segment from the NSO (open ended). Note that only (2)
> >> > involves
> >> > > real data copying. If aborted transactions are rare, this overhead
> >> will
> >> > be
> >> > > insignificant. Assuming that applications typically don't abort
> >> > > transactions, transactions will only be aborted by transaction
> >> > coordinators
> >> > > during hard failure of the producers, which should be rare.
> >> > >
> >> > > This way, the consumer library's logic will be simplified. We can
> >> still
> >> > > expose uncommitted messages to readers in the read-uncommitted mode
> >> and
> >> > > therefore leave the door open for speculative reader in the future.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > >
> >> > > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <apurva@confluent.io
> >
> >> > > wrote:
> >> > >
> >> > > > Hi Joel,
> >> > > >
> >> > > > The alternatives are embedded in the 'discussion' sections which
> are
> >> > > spread
> >> > > > throughout the google doc.
> >> > > >
> >> > > > Admittedly, we have not covered high level alternatives like those
> >> > which
> >> > > > have been brought up in this thread. In particular, having a
> >> separate
> >> > log
> >> > > > for transactional mesages and also having multiple producers
> >> > participate
> >> > > in
> >> > > > a single transaction.
> >> > > >
> >> > > > This is an omission which we will correct.
> >> > > >
> >> > > > Thanks,
> >> > > > Apurva
> >> > > >
> >> > > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jjkoshy.w@gmail.com
> >
> >> > > wrote:
> >> > > >
> >> > > > > >
> >> > > > > >
> >> > > > > > @Joel,
> >> > > > > >
> >> > > > > > I read over your wiki, and apart from the introduction of the
> >> > notion
> >> > > of
> >> > > > > > journal partitions --whose pros and cons are already being
> >> > > discussed--
> >> > > > > you
> >> > > > > > also introduce the notion of a 'producer group' which enables
> >> > > multiple
> >> > > > > > producers to participate in a single transaction. This is
> >> > completely
> >> > > > > > opposite of the model in the KIP where a transaction is
> defined
> >> by
> >> > a
> >> > > > > > producer id, and hence there is a 1-1 mapping between
> producers
> >> and
> >> > > > > > transactions. Further, each producer can have exactly one
> >> in-flight
> >> > > > > > transaction at a time in the KIP.
> >> > > > > >
> >> > > > >
> >> > > > > Hi Apurva - yes I did notice those differences among other
> things
> >> :)
> >> > > > BTW, I
> >> > > > > haven't yet gone through the google-doc carefully but on a skim
> it
> >> > does
> >> > > > not
> >> > > > > seem to contain any rejected alternatives as the wiki states.
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
>
> 114.The section on Discussion on Pro-active Transaction Timeout: "If there
> is no other instance with the same PID yet, or it has not started a
> transaction, or it has not appended a message to some of the partitions,
> then the zombie can continue appending messages to those partitions after
> the abort marker whose epoch has not been incremented yet, but its
> commitTxn call will fail.”

Is that correct, in earlier discussion, it seems that if a transaction is
> timed out by the coordinator, the coordinator will bump up epoch and write
> the abort marker to those inserted partitions.


You are correct. The zombie producer will not be able to write to any
partitions after the abort because the epoch on each broker would have been
bumped by the abort message, causing the zombie to be fenced of totally.
Will correct the doc.

Thanks,
Apurva

On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Jun,
>
> Some answers in line.
>
>
> 109. Could you describe when Producer.send() will receive an Unrecognized
>
> MessageException?
>
>
> This exception will be thrown if the producer sends a sequence number
> which is greater than the sequence number expected by the broker (ie. more
> than 1 greater than the previously sent sequence number). This can happen
> in two cases:
>
> a) If there is a bug in the producer where sequence numbers are
> incremented more than once per message. So the producer itself will send
> messages with gaps in sequence numbers.
> b) The broker somehow lost a previous message. In a cluster configured for
> durability (ie. no unclean leader elections, replication factor of 3,
> min.isr of 2, acks=all, etc.), this should not happened.
>
> So realistically, this exception will only be thrown in clusters
> configured for high availability where brokers could lose messages.
>
> Becket raised the question if we should throw this exception at all in
> case b: it indicates a problem with a previously sent message and hence the
> semantics are counter intuitive. We are still discussing this point, and
> suggestions are most welcome!
>
>
>> 111. Transaction coordinator startup: "Verify that there is already an
>> entry with the PID in the AppID map. If there is not, raise an exception."
>> For completed transactions, it seems that it's possible that their
>> AppId->pid has been compacted out. But that shouldn't trigger an
>> exception?
>
>
> This exception will only be raised if the coordinator encounters
> transaction status message in the log (Begin, AddTPToTransaction, Prepare,
> Commit/Abort). We will compact out the AppId->PID mapping along with the
> transaction status messages for that PID, so we should not encounter one
> without the other. We will clarify that status messages for completed
> transactions can be compacted out aggressively.
>
> 113. Zombie producer:
>> "If the zombie has an ongoing transaction with its old PID while its AppID
>> is being expired by the coordinator, by the time the zombie is about to
>> commit the transaction it needs to talk to coordinator again and will be
>> notified its PID is unrecognized and hence need to re-register its AppID
>> with the InitPIDRequest. At this time, if there is already another
>> registered producer with the same AppID, then this request will be
>> rejected
>> with the fatal ProducerFenced error code."
>> Is that right? According the the coordinator request handling logic, it
>> seems that the InitPIDRequest will bump up the epoch of the pid and
>> succeed?
>
>
> This is a good point. The InitPIDRequest will not fail, but will fence off
> the other producer. In this case, the CommitTxn should fail, since there
> would be no ongoing transaction for the PID/Epoch pair. I will update the
> document to reflect this.
>
>
> On Wed, Jan 4, 2017 at 10:54 AM, Jun Rao <ju...@confluent.io> wrote:
>
>> Thanks for the proposal. A few more detailed comments.
>>
>> 100. Security: It seems that if an app is mistakenly configured with the
>> appId of an existing producer, it can take over the pid and prevent the
>> existing app from publishing. So, I am wondering if it makes sense to add
>> ACLs at the TransactionResource level just like we do for
>> ConsumerGroupResource. So, a user can only do transactions under a
>> particular appId if he/she has the write permission to the
>> TransactionResource
>> associated with the appId.
>>
>> 101. Compatibility during upgrade: Suppose that the brokers are upgraded
>> to
>> the new version, but the broker message format is still the old one. If a
>> new producer uses the transaction feature, should the producer get an
>> error
>> in this case? A tricky case can be that the leader broker is on the new
>> message format, but the follower broker is still on the old message
>> format.
>> In this case, the transactional info will be lost in the follower due to
>> down conversion. Should we failed the transactional requests when the
>> followers are still on the old message format?
>>
>> 102. When there is a correlated hard failure (e.g., power outage), it's
>> possible that an existing commit/abort marker is lost in all replicas.
>> This
>> may not be fixed by the transaction coordinator automatically and the
>> consumer may get stuck on that incomplete transaction forever. Not sure
>> what's the best way to address this. Perhaps, one way is to run a tool to
>> add an abort maker for all pids in all affected partitions.
>>
>> 103. Currently, there is no check for producer liveness. This means that
>> if
>> a producer has not been sending transactional requests for a long time,
>> its
>> appId will be expired by the coordinator. Have we considered having
>> producers sending heartbeatRequest just like the consumer to keep it
>> alive?
>>
>> 104. The logic for handling follower truncation can be a bit tricker now.
>> The truncation may rewind the sequence number for some pids. The question
>> is how to quickly recover the last sequence number of those pids. Do we
>> plan to reload from a PID snapshot and scan forward?
>>
>> 105. When the transaction coordinator changes (due to leadership changes),
>> it's possible for both the old and the new coordinator sending requests to
>> a broker at the same time (for a short period of time). I am wondering if
>> we need to add logic to fence off the old coordinator. One way to do that
>> is to include the leaderEpoch of the partition associated with the
>> coordinator in the coordinator to broker request and control messages.
>>
>> 106. Compacted topics.
>> 106.1. When all messages in a transaction are removed, we could remove the
>> commit/abort marker for that transaction too. However, we have to be a bit
>> careful. If the marker is removed too quickly, it's possible for a
>> consumer
>> to see a message in that transaction, but not to see the marker, and
>> therefore will be stuck in that transaction forever. We have a similar
>> issue when dealing with tombstones. The solution is to preserve the
>> tombstone for at least a preconfigured amount of time after the cleaning
>> has passed the tombstone. Then, as long as a consumer can finish reading
>> to
>> the cleaning point within the configured amount of time, it's guaranteed
>> not to miss the tombstone after it has seen a non-tombstone message on the
>> same key. I am wondering if we should do something similar here.
>> 106.2. "To address this problem, we propose to preserve the last epoch and
>> sequence number written by each producer for a fixed amount of time as an
>> empty message set. This is allowed by the new message format we are
>> proposing in this document. The time to preserve the sequence number will
>> be governed by the log retention settings. " Could you be a bit more
>> specific on what retention time will be used since by default, there is no
>> retention time for compacted (but not delete) topic?
>> 106.3 "As for control messages, if the broker does not have any
>> corresponding transaction cached with the PID when encountering a control
>> message, that message can be safely removed."
>> Do controlled messages have keys? If not, do we need to relax the
>> constraint that messages in a compacted topic must have keys?
>>
>> 107. Could you include the default values for the newly introduced
>> configs?
>>
>> 108. Could you describe the format of the PID snapshot file?
>>
>> 109. Could you describe when Producer.send() will receive an UnrecognizedM
>> essageException?
>>
>> 110. Transaction log:
>> 110.1 "Key => Version AppID Version" It seems that Version should really
>> be
>> Type?
>> 110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
>> Should we store [Topic [Partition]] instead?
>> 110.3 To expire an AppId, do we need to insert a tombstone with the
>> expired
>> AppID as the key to physically remove the existing AppID entries in the
>> transaction log?
>>
>> 111. Transaction coordinator startup: "Verify that there is already an
>> entry with the PID in the AppID map. If there is not, raise an exception."
>> For completed transactions, it seems that it's possible that their
>> AppId->pid has been compacted out. But that shouldn't trigger an
>> exception?
>>
>> 112. Control message: Will control messages be used for timestamp
>> indexing?
>> If so, what timestamp will we use if the timestamp type is creation time?
>>
>> 113. Zombie producer:
>> "If the zombie has an ongoing transaction with its old PID while its AppID
>> is being expired by the coordinator, by the time the zombie is about to
>> commit the transaction it needs to talk to coordinator again and will be
>> notified its PID is unrecognized and hence need to re-register its AppID
>> with the InitPIDRequest. At this time, if there is already another
>> registered producer with the same AppID, then this request will be
>> rejected
>> with the fatal ProducerFenced error code."
>> Is that right? According the the coordinator request handling logic, it
>> seems that the InitPIDRequest will bump up the epoch of the pid and
>> succeed?
>>
>> 114.The section on Discussion on Pro-active Transaction Timeout: "If there
>> is no other instance with the same PID yet, or it has not started a
>> transaction, or it has not appended a message to some of the partitions,
>> then the zombie can continue appending messages to those partitions after
>> the abort marker whose epoch has not been incremented yet, but its
>> commitTxn
>> call will fail."
>> Is that correct, in earlier discussion, it seems that if a transaction is
>> timed out by the coordinator, the coordinator will bump up epoch and write
>> the abort marker to those inserted partitions.
>>
>> 115. Message format:
>> 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
>> since it's possible for a producer to be restarted 10s thousands of times?
>> 115.2 Sequence number int32. Does it wrap after max? It's possible for a
>> producer to publish more than 2 billion messages in a session.
>> 115.3 "Null-value bit is 1: skip the key-length (since it can now be
>> calculated) and value fields." It seems that it's unnatural for the format
>> of key to depend on value. It seems it's easier to just skip value in this
>> case?
>>
>> 116. ProducerRequest: The existing format doesn't have "MessageSetSize" at
>> the partition level.
>>
>> 117. UpdateTxnRequest: Could you explain the format of Marker?
>>
>> 118. TxnOffsetCommitRequest: How is retention time determined? Do we need
>> a
>> new config in producer or just default it to -1 as the consumer?
>>
>> 119. InitPIDRequest
>> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7
>> CP1SnQGdp2eF0wSw9ra8/edit#heading=h.z99xar1h2enr>
>> : Should we write the completion of open transactions before append the
>> pid
>> with bumped up epoch to the transaction log?
>>
>> 120. transaction.app.id: An app may have multiple concurrent instances.
>> Perhaps we should name it transaction.instance.id or just instance.id?
>>
>> 121. The ordering is important with idempotent producer, which means
>> that max.in.flight.requests.per.connection should be set to 1. Do we want
>> to enforce this?
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com> wrote:
>>
>> > @jun - good proposal. i was willing to concede that read-uncommitted was
>> > impossible under my proposal but if LSO/NSO is introduced is becomes
>> > possible.
>>
>> >
>> >
>> > On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Just to follow up on Radai's idea of pushing the buffering logic to
>> the
>> > > broker. It may be possible to do this efficiently if we assume aborted
>> > > transactions are rare. The following is a draft proposal. For each
>> > > partition, the broker maintains the last stable offset (LSO) as
>> described
>> > > in the document, and only exposes messages up to this point if the
>> reader
>> > > is in the read-committed mode. When a new stable offset (NSO) is
>> > > determined, if there is no aborted message in this window, the broker
>> > > simply advances the LSO to the NSO. If there is at least one aborted
>> > > message, the broker first replaces the current log segment with new
>> log
>> > > segments excluding the aborted messages and then advances the LSO. To
>> > make
>> > > the replacement efficient, we can replace the current log segment
>> with 3
>> > > new segments: (1) a new "shadow" log segment that simply references
>> the
>> > > portion of the current log segment from the beginning to the LSO, (2)
>> a
>> > log
>> > > segment created by copying only committed messages between the LSO and
>> > the
>> > > NSO, (3) a new "shadow" log segment that references the portion of the
>> > > current log segment from the NSO (open ended). Note that only (2)
>> > involves
>> > > real data copying. If aborted transactions are rare, this overhead
>> will
>> > be
>> > > insignificant. Assuming that applications typically don't abort
>> > > transactions, transactions will only be aborted by transaction
>> > coordinators
>> > > during hard failure of the producers, which should be rare.
>> > >
>> > > This way, the consumer library's logic will be simplified. We can
>> still
>> > > expose uncommitted messages to readers in the read-uncommitted mode
>> and
>> > > therefore leave the door open for speculative reader in the future.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
>> > > wrote:
>> > >
>> > > > Hi Joel,
>> > > >
>> > > > The alternatives are embedded in the 'discussion' sections which are
>> > > spread
>> > > > throughout the google doc.
>> > > >
>> > > > Admittedly, we have not covered high level alternatives like those
>> > which
>> > > > have been brought up in this thread. In particular, having a
>> separate
>> > log
>> > > > for transactional mesages and also having multiple producers
>> > participate
>> > > in
>> > > > a single transaction.
>> > > >
>> > > > This is an omission which we will correct.
>> > > >
>> > > > Thanks,
>> > > > Apurva
>> > > >
>> > > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
>> > > wrote:
>> > > >
>> > > > > >
>> > > > > >
>> > > > > > @Joel,
>> > > > > >
>> > > > > > I read over your wiki, and apart from the introduction of the
>> > notion
>> > > of
>> > > > > > journal partitions --whose pros and cons are already being
>> > > discussed--
>> > > > > you
>> > > > > > also introduce the notion of a 'producer group' which enables
>> > > multiple
>> > > > > > producers to participate in a single transaction. This is
>> > completely
>> > > > > > opposite of the model in the KIP where a transaction is defined
>> by
>> > a
>> > > > > > producer id, and hence there is a 1-1 mapping between producers
>> and
>> > > > > > transactions. Further, each producer can have exactly one
>> in-flight
>> > > > > > transaction at a time in the KIP.
>> > > > > >
>> > > > >
>> > > > > Hi Apurva - yes I did notice those differences among other things
>> :)
>> > > > BTW, I
>> > > > > haven't yet gone through the google-doc carefully but on a skim it
>> > does
>> > > > not
>> > > > > seem to contain any rejected alternatives as the wiki states.
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Jun,

Some answers in line.


109. Could you describe when Producer.send() will receive an Unrecognized

MessageException?


This exception will be thrown if the producer sends a sequence number which
is greater than the sequence number expected by the broker (ie. more than 1
greater than the previously sent sequence number). This can happen in two
cases:

a) If there is a bug in the producer where sequence numbers are incremented
more than once per message. So the producer itself will send messages with
gaps in sequence numbers.
b) The broker somehow lost a previous message. In a cluster configured for
durability (ie. no unclean leader elections, replication factor of 3,
min.isr of 2, acks=all, etc.), this should not happened.

So realistically, this exception will only be thrown in clusters configured
for high availability where brokers could lose messages.

Becket raised the question if we should throw this exception at all in case
b: it indicates a problem with a previously sent message and hence the
semantics are counter intuitive. We are still discussing this point, and
suggestions are most welcome!


> 111. Transaction coordinator startup: "Verify that there is already an
> entry with the PID in the AppID map. If there is not, raise an exception."
> For completed transactions, it seems that it's possible that their
> AppId->pid has been compacted out. But that shouldn't trigger an exception?


This exception will only be raised if the coordinator encounters
transaction status message in the log (Begin, AddTPToTransaction, Prepare,
Commit/Abort). We will compact out the AppId->PID mapping along with the
transaction status messages for that PID, so we should not encounter one
without the other. We will clarify that status messages for completed
transactions can be compacted out aggressively.

113. Zombie producer:
> "If the zombie has an ongoing transaction with its old PID while its AppID
> is being expired by the coordinator, by the time the zombie is about to
> commit the transaction it needs to talk to coordinator again and will be
> notified its PID is unrecognized and hence need to re-register its AppID
> with the InitPIDRequest. At this time, if there is already another
> registered producer with the same AppID, then this request will be rejected
> with the fatal ProducerFenced error code."
> Is that right? According the the coordinator request handling logic, it
> seems that the InitPIDRequest will bump up the epoch of the pid and
> succeed?


This is a good point. The InitPIDRequest will not fail, but will fence off
the other producer. In this case, the CommitTxn should fail, since there
would be no ongoing transaction for the PID/Epoch pair. I will update the
document to reflect this.


On Wed, Jan 4, 2017 at 10:54 AM, Jun Rao <ju...@confluent.io> wrote:

> Thanks for the proposal. A few more detailed comments.
>
> 100. Security: It seems that if an app is mistakenly configured with the
> appId of an existing producer, it can take over the pid and prevent the
> existing app from publishing. So, I am wondering if it makes sense to add
> ACLs at the TransactionResource level just like we do for
> ConsumerGroupResource. So, a user can only do transactions under a
> particular appId if he/she has the write permission to the
> TransactionResource
> associated with the appId.
>
> 101. Compatibility during upgrade: Suppose that the brokers are upgraded to
> the new version, but the broker message format is still the old one. If a
> new producer uses the transaction feature, should the producer get an error
> in this case? A tricky case can be that the leader broker is on the new
> message format, but the follower broker is still on the old message format.
> In this case, the transactional info will be lost in the follower due to
> down conversion. Should we failed the transactional requests when the
> followers are still on the old message format?
>
> 102. When there is a correlated hard failure (e.g., power outage), it's
> possible that an existing commit/abort marker is lost in all replicas. This
> may not be fixed by the transaction coordinator automatically and the
> consumer may get stuck on that incomplete transaction forever. Not sure
> what's the best way to address this. Perhaps, one way is to run a tool to
> add an abort maker for all pids in all affected partitions.
>
> 103. Currently, there is no check for producer liveness. This means that if
> a producer has not been sending transactional requests for a long time, its
> appId will be expired by the coordinator. Have we considered having
> producers sending heartbeatRequest just like the consumer to keep it alive?
>
> 104. The logic for handling follower truncation can be a bit tricker now.
> The truncation may rewind the sequence number for some pids. The question
> is how to quickly recover the last sequence number of those pids. Do we
> plan to reload from a PID snapshot and scan forward?
>
> 105. When the transaction coordinator changes (due to leadership changes),
> it's possible for both the old and the new coordinator sending requests to
> a broker at the same time (for a short period of time). I am wondering if
> we need to add logic to fence off the old coordinator. One way to do that
> is to include the leaderEpoch of the partition associated with the
> coordinator in the coordinator to broker request and control messages.
>
> 106. Compacted topics.
> 106.1. When all messages in a transaction are removed, we could remove the
> commit/abort marker for that transaction too. However, we have to be a bit
> careful. If the marker is removed too quickly, it's possible for a consumer
> to see a message in that transaction, but not to see the marker, and
> therefore will be stuck in that transaction forever. We have a similar
> issue when dealing with tombstones. The solution is to preserve the
> tombstone for at least a preconfigured amount of time after the cleaning
> has passed the tombstone. Then, as long as a consumer can finish reading to
> the cleaning point within the configured amount of time, it's guaranteed
> not to miss the tombstone after it has seen a non-tombstone message on the
> same key. I am wondering if we should do something similar here.
> 106.2. "To address this problem, we propose to preserve the last epoch and
> sequence number written by each producer for a fixed amount of time as an
> empty message set. This is allowed by the new message format we are
> proposing in this document. The time to preserve the sequence number will
> be governed by the log retention settings. " Could you be a bit more
> specific on what retention time will be used since by default, there is no
> retention time for compacted (but not delete) topic?
> 106.3 "As for control messages, if the broker does not have any
> corresponding transaction cached with the PID when encountering a control
> message, that message can be safely removed."
> Do controlled messages have keys? If not, do we need to relax the
> constraint that messages in a compacted topic must have keys?
>
> 107. Could you include the default values for the newly introduced configs?
>
> 108. Could you describe the format of the PID snapshot file?
>
> 109. Could you describe when Producer.send() will receive an UnrecognizedM
> essageException?
>
> 110. Transaction log:
> 110.1 "Key => Version AppID Version" It seems that Version should really be
> Type?
> 110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
> Should we store [Topic [Partition]] instead?
> 110.3 To expire an AppId, do we need to insert a tombstone with the expired
> AppID as the key to physically remove the existing AppID entries in the
> transaction log?
>
> 111. Transaction coordinator startup: "Verify that there is already an
> entry with the PID in the AppID map. If there is not, raise an exception."
> For completed transactions, it seems that it's possible that their
> AppId->pid has been compacted out. But that shouldn't trigger an exception?
>
> 112. Control message: Will control messages be used for timestamp indexing?
> If so, what timestamp will we use if the timestamp type is creation time?
>
> 113. Zombie producer:
> "If the zombie has an ongoing transaction with its old PID while its AppID
> is being expired by the coordinator, by the time the zombie is about to
> commit the transaction it needs to talk to coordinator again and will be
> notified its PID is unrecognized and hence need to re-register its AppID
> with the InitPIDRequest. At this time, if there is already another
> registered producer with the same AppID, then this request will be rejected
> with the fatal ProducerFenced error code."
> Is that right? According the the coordinator request handling logic, it
> seems that the InitPIDRequest will bump up the epoch of the pid and
> succeed?
>
> 114.The section on Discussion on Pro-active Transaction Timeout: "If there
> is no other instance with the same PID yet, or it has not started a
> transaction, or it has not appended a message to some of the partitions,
> then the zombie can continue appending messages to those partitions after
> the abort marker whose epoch has not been incremented yet, but its
> commitTxn
> call will fail."
> Is that correct, in earlier discussion, it seems that if a transaction is
> timed out by the coordinator, the coordinator will bump up epoch and write
> the abort marker to those inserted partitions.
>
> 115. Message format:
> 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> since it's possible for a producer to be restarted 10s thousands of times?
> 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> producer to publish more than 2 billion messages in a session.
> 115.3 "Null-value bit is 1: skip the key-length (since it can now be
> calculated) and value fields." It seems that it's unnatural for the format
> of key to depend on value. It seems it's easier to just skip value in this
> case?
>
> 116. ProducerRequest: The existing format doesn't have "MessageSetSize" at
> the partition level.
>
> 117. UpdateTxnRequest: Could you explain the format of Marker?
>
> 118. TxnOffsetCommitRequest: How is retention time determined? Do we need a
> new config in producer or just default it to -1 as the consumer?
>
> 119. InitPIDRequest
> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#heading=h.z99xar1h2enr>
> : Should we write the completion of open transactions before append the pid
> with bumped up epoch to the transaction log?
>
> 120. transaction.app.id: An app may have multiple concurrent instances.
> Perhaps we should name it transaction.instance.id or just instance.id?
>
> 121. The ordering is important with idempotent producer, which means
> that max.in.flight.requests.per.connection should be set to 1. Do we want
> to enforce this?
>
> Thanks,
>
> Jun
>
>
> On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com> wrote:
>
> > @jun - good proposal. i was willing to concede that read-uncommitted was
> > impossible under my proposal but if LSO/NSO is introduced is becomes
> > possible.
> >
> >
> > On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Just to follow up on Radai's idea of pushing the buffering logic to the
> > > broker. It may be possible to do this efficiently if we assume aborted
> > > transactions are rare. The following is a draft proposal. For each
> > > partition, the broker maintains the last stable offset (LSO) as
> described
> > > in the document, and only exposes messages up to this point if the
> reader
> > > is in the read-committed mode. When a new stable offset (NSO) is
> > > determined, if there is no aborted message in this window, the broker
> > > simply advances the LSO to the NSO. If there is at least one aborted
> > > message, the broker first replaces the current log segment with new log
> > > segments excluding the aborted messages and then advances the LSO. To
> > make
> > > the replacement efficient, we can replace the current log segment with
> 3
> > > new segments: (1) a new "shadow" log segment that simply references the
> > > portion of the current log segment from the beginning to the LSO, (2) a
> > log
> > > segment created by copying only committed messages between the LSO and
> > the
> > > NSO, (3) a new "shadow" log segment that references the portion of the
> > > current log segment from the NSO (open ended). Note that only (2)
> > involves
> > > real data copying. If aborted transactions are rare, this overhead will
> > be
> > > insignificant. Assuming that applications typically don't abort
> > > transactions, transactions will only be aborted by transaction
> > coordinators
> > > during hard failure of the producers, which should be rare.
> > >
> > > This way, the consumer library's logic will be simplified. We can still
> > > expose uncommitted messages to readers in the read-uncommitted mode and
> > > therefore leave the door open for speculative reader in the future.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > >
> > > > Hi Joel,
> > > >
> > > > The alternatives are embedded in the 'discussion' sections which are
> > > spread
> > > > throughout the google doc.
> > > >
> > > > Admittedly, we have not covered high level alternatives like those
> > which
> > > > have been brought up in this thread. In particular, having a separate
> > log
> > > > for transactional mesages and also having multiple producers
> > participate
> > > in
> > > > a single transaction.
> > > >
> > > > This is an omission which we will correct.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > >
> > > > > >
> > > > > >
> > > > > > @Joel,
> > > > > >
> > > > > > I read over your wiki, and apart from the introduction of the
> > notion
> > > of
> > > > > > journal partitions --whose pros and cons are already being
> > > discussed--
> > > > > you
> > > > > > also introduce the notion of a 'producer group' which enables
> > > multiple
> > > > > > producers to participate in a single transaction. This is
> > completely
> > > > > > opposite of the model in the KIP where a transaction is defined
> by
> > a
> > > > > > producer id, and hence there is a 1-1 mapping between producers
> and
> > > > > > transactions. Further, each producer can have exactly one
> in-flight
> > > > > > transaction at a time in the KIP.
> > > > > >
> > > > >
> > > > > Hi Apurva - yes I did notice those differences among other things
> :)
> > > > BTW, I
> > > > > haven't yet gone through the google-doc carefully but on a skim it
> > does
> > > > not
> > > > > seem to contain any rejected alternatives as the wiki states.
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Gwen Shapira <gw...@confluent.io>.
Understood, thank you!

On Fri, Feb 10, 2017 at 2:54 PM, Apurva Mehta <ap...@confluent.io> wrote:
> The point about management of transactionalId was raised before, and Jason
> tackled it here (note it was called 'AppId' back then):
> https://www.mail-archive.com/dev@kafka.apache.org/msg61200.html
>
> 1. Essentially, the full transactional functionality really only makes
> sense for a stateful application. Further, in a stateful and partitioned
> application, each shard must have a stable unique id. This can be used
> directly as the transactionalId.
>
> 2. For non-stateful applications which want transactions within a single
> session of a producer, they can generate a UUID per session, and use that
> as the transactionalId.
>
> So yes, managing transactionalIds are not trivial, but stateful
> applications will have the means to do so.
>
> The final note is that the primary way to use these transactional apis is
> through Kafka streams, and in that context, the transactionalId will be a
> combination of the streams appId and the streams taskId. Note that this is
> essentially the shardId alluded to in point 1 above, and Streams can take
> care of it transparently.
>
> Thanks,
> Apurva
>
>
>
> On Fri, Feb 10, 2017 at 2:33 PM, Gwen Shapira <gw...@confluent.io> wrote:
>
>> Thank you so much for the detailed explanation!
>> I got the READ COMMITTED behavior. It is very odd that we will
>> routinely see parts of a committed transaction but not other parts.
>> But I understand the reasons.
>>
>> Regarding transactional.id: I read the document but I'm still not 100%
>> clear on how unique transactional ids can happen in practice.
>> I'm sure you've thought of this, but I'd like to understand:
>> Right now, I'm writing an consume-process-produce app, I give it a
>> consumer group.id, and it is safe to deploy the app multiple times
>> with the same configuration - because all the instances will have the
>> same group.id. But it sounds like each instance will need its own
>> transactional.id, so it can commit its own transactions? Is the idea
>> that I'll read the configuration from a file and an admin will be
>> responsible to come up with unique transactional IDs while deploying?
>> This sounds a bit more complex than I'd like. Maybe you have a simple
>> solution I am not seeing...
>>
>> Gwen
>>
>>
>>
>>
>> On Fri, Feb 10, 2017 at 12:25 AM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>> > Hey Gwen,
>> >
>> > Thanks for the questions and comments. Responses below:
>> >
>> > I not sure I'm clear on the expected behavior of READ_COMMITTED in
>> >> some interleaved cases:
>> >> * If a transaction starts, sends few messages and someone writes
>> >> non-transactional event into the same topic/partition, few more events
>> >> from that transaction and then a commit. I think the producer will
>> >> block until the commit, but I'm not sure in what order I'll see events
>> >> after that.
>> >
>> > * Same for transaction A starts, transactions B starts, transaction B
>> >> commits, transaction A commits... when will we unblock? and what will
>> >> we see?
>> >
>> >
>> > In READ_COMMITTED (as well as in READ_UNCOMMITTED), the messages are
>> always
>> > returned in the order of their offsets. What controls the visibility of
>> > messages in READ_COMMITTED is the last stable offset (LSO). This is
>> defined
>> > as the last offset in the log such that all messages with smaller offsets
>> > have been decided (either committed or aborted).
>> >
>> > Take the first case: non-transactional data interleaved with
>> transactional
>> > data. Let N be a non-transactional message, T be a transactional message,
>> > and C be a commit marker. Suppose we have the following state of the log
>> > (the first row is the offsets, the second is the messages):
>> >
>> > 0, 1, 2
>> > N, T, N
>> >
>> > The LSO in this log is 1, which means the non-transactional message at
>> > offset 0 can be read by a consumer, but the one at offset 2 cannot
>> because
>> > the transaction beginning at offset 1 has not completed. Later we get
>> some
>> > more data:
>> >
>> > 0, 1, 2, 3, 4
>> > N, T, N, N, T
>> >
>> > The LSO still cannot advance because the transaction beginning at offset
>> 1
>> > has not completed. Finally we get the commit marker:
>> >
>> > 0, 1, 2, 3, 4, 5
>> > N, T, N, N, T, C
>> >
>> > At this point, the LSO advances to 6, and all of the messages from
>> offset 1
>> > to 5 are returned, in that order, to consumers.
>> >
>> > Now consider interleaved transactions. Let T1 be a transactional message
>> > from producer 1, and T2 be a transactional message from producer 2. Both
>> > producers begin a transaction:
>> >
>> > 0, 1, 2
>> > T1, T2, T1
>> >
>> > The LSO is 0 since the transaction starting from that offset has not been
>> > decided. Now suppose T1 commits.
>> >
>> > 0, 1, 2, 3
>> > T1, T2, T1, C1
>> >
>> > The LSO then advances to offset 1. The message at offset 0 becomes
>> visible
>> > to consumers in READ_COMMITTED mode, but the remaining messages from the
>> > transaction are blocked until T2 is decided. Now T2 commits:
>> >
>> > 0, 1, 2, 3, 4, 5
>> > T1, T2, T1, C1, T2, C2
>> >
>> > The LSO advances to 6, and just like above, all of the data from offsets
>> 1
>> > to 5 becomes visible.
>> >
>> > This ordering is what we refer to as "offset order." We always return
>> > messages in the order of their offsets. Another interesting option to
>> > consider is "transaction order." Consider again the state after T1
>> > committed:
>> >
>> > 0, 1, 2, 3
>> > T1, T2, T1, C1
>> >
>> > In transaction order, all of the messages from T1 become visible
>> > immediately upon receiving the commit marker. Later when T2 commits, its
>> > messages will also becomes visible. The main advantage is lower latency,
>> > but it is quite a bit more complicated to implement (see the design
>> > document for the discussion). We went with the simpler approach and left
>> > this for possible future work.
>> >
>> > i'm concerned about transactional.id uniqueness in real-world
>> conditions.
>> >> When users were forced to manually define unique broker ids, we saw
>> >> lots of errors with "duplicate broker id" on startup show up on the
>> >> mailing list.
>> >> When we added a persistent (to filesystem) auto-generated broker ID,
>> >> the number of issues dropped dramatically.
>> >> I wonder if this makes sense here too.
>> >
>> >
>> > There has been a lot of discussion about this. I will link you to the
>> > section in the design document:
>> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> 0wSw9ra8/edit#heading=h.lizhp2urqn59
>> > .
>> >
>> > It sounds like producers will now be idempotent by default. I.e. - if
>> >> we upgrade and do nothing different, we have idempotent producer? and
>> >> absolutely no way to get the old behavior back?
>> >
>> >
>> > This may have been a case of the KIP wiki getting out of sync with the
>> > design document. I've updated it to include the `enable.idempotence`
>> > producer config which was added previously to the design document. By
>> > default, idempotence will be disabled. Down the road, it may make sense
>> to
>> > change that, but we're uncomfortable turning it on in the first version
>> > because of the subtle changes in behavior (in particular, the fact that
>> we
>> > raise the fatal OutOfSequenceException in the case of message loss or PID
>> > expiration).
>> >
>> > Thanks,
>> > Jason
>> >
>> > On Thu, Feb 9, 2017 at 10:41 PM, Gwen Shapira <gw...@confluent.io> wrote:
>> >
>> >> It sounds like producers will now be idempotent by default. I.e. - if
>> >> we upgrade and do nothing different, we have idempotent producer? and
>> >> absolutely no way to get the old behavior back?
>> >>
>> >> I don't think anyone needs non-idempotent producer, just want to
>> clarify :)
>> >>
>> >> On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >> > A couple follow-ups from Ismael's previous comments.
>> >> >
>> >> > 1. We have removed the individual message CRC field from the message
>> >> > format. Because this field can already change on the broker in some
>> >> > situations, we feel it is probably not wise to let clients depend on
>> it,
>> >> > and removing it saves some space and the redundant verification on the
>> >> > broker. We have also proposed to deprecate the checksum() APIs which
>> are
>> >> > exposed in RecordMetadata and ConsumerRecord.
>> >> > 2. We changed the message timestamp field to be variable length. The
>> >> values
>> >> > are computed relative to the timestamp of the first message in the
>> set.
>> >> We
>> >> > considered my previous suggestion to make the first message timestamp
>> >> > absolute with later messages relative to their previous timestamp, but
>> >> > ultimately felt it was simpler to stick with the same pattern that is
>> >> used
>> >> > for relative offsets. To enable this, we added an initial timestamp
>> field
>> >> > to the message set.
>> >> >
>> >> > Thanks,
>> >> > Jason
>> >> >
>> >> > On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io>
>> >> wrote:
>> >> >
>> >> >> Hello,
>> >> >>
>> >> >> I have gone ahead and updated the KIP wiki with a summary of the
>> >> changes to
>> >> >> the RPC protocol. The KIP wiki should now have _all_ the public
>> facing
>> >> >> changes being proposed.
>> >> >>
>> >> >> The proposed changes were always in the Google doc, and now we are
>> >> simply
>> >> >> making good on our promise to copy them over to the wiki since the
>> >> design
>> >> >> is almost finalized.
>> >> >>
>> >> >> Thanks,
>> >> >> Apurva
>> >> >>
>> >> >> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
>> >> >> wrote:
>> >> >>
>> >> >> > Hey Tom,
>> >> >> >
>> >> >> > Re; complexity. This is always a tradeoff with new features. The
>> >> changes
>> >> >> > we've made during the design and review process have greatly
>> >> simplified
>> >> >> the
>> >> >> > implementation for clients, and especially for the consumer, but
>> >> there is
>> >> >> > nontrivial work needed here to support transactions on the
>> producer.
>> >> I'm
>> >> >> > not sure how it could be otherwise and we've spent a ton of time
>> >> thinking
>> >> >> > about this. It's also worth mentioning that there's a relatively
>> low
>> >> bar
>> >> >> to
>> >> >> > support the idempotent producer while keeping the client thin (it
>> >> >> requires
>> >> >> > support for one new request type and some sequence bookkeeping).
>> >> >> >
>> >> >> > Ultimately, we have to decide whether the improved semantics are
>> worth
>> >> >> the
>> >> >> > cost of the complexity. In my opinion, they are. The benefit of
>> having
>> >> >> > exactly-once processing in streaming applications is huge. And to
>> >> manage
>> >> >> > the complexity, we've intentionally used patterns that were already
>> >> >> > familiar in the codebase (e.g. our approach to maintaining
>> transaction
>> >> >> > state through a coordinator is copied from how consumer offsets are
>> >> >> > managed). We've moved as much work from the clients to the broker
>> as
>> >> >> > possible, and we have resisted at every turn complicating the
>> client
>> >> APIs
>> >> >> > even where it may have simplified some internals.
>> >> >> >
>> >> >> > -Jason
>> >> >> >
>> >> >> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
>> >> >> wrote:
>> >> >> >
>> >> >> > > Hi Tom,
>> >> >> > >
>> >> >> > > I updated the KIP with a note our plans for performance testing:
>> >> >> > >
>> >> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >> >> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
>> >> >> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
>> >> >> > >
>> >> >> > > Thanks for pointing that out.
>> >> >> > >
>> >> >> > > Regards,
>> >> >> > > Apurva
>> >> >> > >
>> >> >> > >
>> >> >> > >
>> >> >> > >
>> >> >> > >
>> >> >> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <
>> tcrayford@heroku.com>
>> >> >> > wrote:
>> >> >> > >
>> >> >> > > > I think the updated wiki page makes sense with respect to ACLs,
>> >> there
>> >> >> > > seems
>> >> >> > > > to be little potential for abuse there (other than the noted
>> and
>> >> >> known
>> >> >> > > > issues).
>> >> >> > > >
>> >> >> > > > I am going to note that this is a major complexity increase for
>> >> >> Kafka,
>> >> >> > > and
>> >> >> > > > that I'm concerned about performance impact (the JVM is quite…
>> >> >> pedantic
>> >> >> > > > about method size, for example, and even adding conditionals to
>> >> >> larger
>> >> >> > > > methods could impact this). The KIP doesn't note plans for
>> >> >> performance
>> >> >> > > > testing.
>> >> >> > > >
>> >> >> > > > I'm also concerned about the impact on non-JVM client
>> libraries -
>> >> >> > > writing a
>> >> >> > > > client for Kafka is already a very complicated undertaking, and
>> >> this
>> >> >> > adds
>> >> >> > > > to that complexity significantly.
>> >> >> > > >
>> >> >> > > > However, the approach seems ok enough. It does also violate the
>> >> >> "Kafka
>> >> >> > > has
>> >> >> > > > dumb brokers and smart clients" (which I realize is in direct
>> >> >> > > contradiction
>> >> >> > > > of the previous statement about client implementation being
>> hard).
>> >> >> I'd
>> >> >> > > love
>> >> >> > > > to see some discussion in either the doc or the wiki as to why
>> >> much
>> >> >> of
>> >> >> > > this
>> >> >> > > > transactional work isn't a client-side part of Kafka Streams.
>> >> >> > > >
>> >> >> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <
>> >> jason@confluent.io>
>> >> >> > > > wrote:
>> >> >> > > >
>> >> >> > > > > One additional note on the authorization. The WriteTxnMarker
>> >> API is
>> >> >> > > > > restricted to inter-broker usage, so it requires Cluster
>> >> >> > authorization
>> >> >> > > > > (just like other inter-broker APIs). I've updated the
>> document
>> >> and
>> >> >> > wiki
>> >> >> > > > to
>> >> >> > > > > reflect this.
>> >> >> > > > >
>> >> >> > > > > Also, I have renamed GroupCoordinatorRequest to
>> >> >> > FindCoordinatorRequest
>> >> >> > > > > since there is no group for transactional producers. Let me
>> >> know if
>> >> >> > > there
>> >> >> > > > > are any concerns.
>> >> >> > > > >
>> >> >> > > > > -Jason
>> >> >> > > > >
>> >> >> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
>> >> >> jason@confluent.io>
>> >> >> > > > > wrote:
>> >> >> > > > >
>> >> >> > > > > > Hi Tom,
>> >> >> > > > > >
>> >> >> > > > > > I said this in the voting thread, but can the authors
>> include
>> >> a
>> >> >> > > section
>> >> >> > > > > >> about new ACLs if there are going to be ACLs for
>> >> >> TransactionalId.
>> >> >> > > It's
>> >> >> > > > > >> mentioned in the google doc, but I think new ACLs should
>> be
>> >> in a
>> >> >> > KIP
>> >> >> > > > > >> directly.
>> >> >> > > > > >
>> >> >> > > > > >
>> >> >> > > > > > We've updated the wiki. Can you take a look and let us
>> know if
>> >> >> you
>> >> >> > > have
>> >> >> > > > > > additional concerns?
>> >> >> > > > > >
>> >> >> > > > > > Thanks,
>> >> >> > > > > > Jason
>> >> >> > > > > >
>> >> >> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
>> >> >> > > > rajinisivaram@gmail.com>
>> >> >> > > > > > wrote:
>> >> >> > > > > >
>> >> >> > > > > >> Hi Jason,
>> >> >> > > > > >>
>> >> >> > > > > >> Thank you for the responses. Agree that authorizing
>> >> >> > > transactional.id
>> >> >> > > > in
>> >> >> > > > > >> the
>> >> >> > > > > >> producer requests will be good enough for version 1. And
>> >> making
>> >> >> it
>> >> >> > > > > tighter
>> >> >> > > > > >> in future based on delegation tokens sounds good too.
>> >> >> > > > > >>
>> >> >> > > > > >> Regards,
>> >> >> > > > > >>
>> >> >> > > > > >> Rajini
>> >> >> > > > > >>
>> >> >> > > > > >>
>> >> >> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
>> >> >> > jason@confluent.io
>> >> >> > > >
>> >> >> > > > > >> wrote:
>> >> >> > > > > >>
>> >> >> > > > > >> > Hey Rajini,
>> >> >> > > > > >> >
>> >> >> > > > > >> > Thanks for the questions. Responses below:
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > > 1. Will the transaction coordinator check topic ACLs
>> >> based
>> >> >> on
>> >> >> > > the
>> >> >> > > > > >> > > requesting client's credentials? Access to transaction
>> >> logs,
>> >> >> > > > topics
>> >> >> > > > > >> being
>> >> >> > > > > >> > > added for transaction etc?
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > Good question. I think it makes sense to check topic
>> Write
>> >> >> > > > permission
>> >> >> > > > > >> when
>> >> >> > > > > >> > adding partitions to the transaction. I'll add this to
>> the
>> >> >> > > document.
>> >> >> > > > > >> > Perhaps authorization to the transaction log itself,
>> >> however,
>> >> >> > can
>> >> >> > > be
>> >> >> > > > > >> > assumed from having access to the
>> ProducerTransactionalId
>> >> >> > > resource?
>> >> >> > > > > This
>> >> >> > > > > >> > would be similar to how access to __consumer_offsets is
>> >> >> assumed
>> >> >> > if
>> >> >> > > > the
>> >> >> > > > > >> > client has access to the Group resource.
>> >> >> > > > > >> >
>> >> >> > > > > >> > 2. If I create a transactional produce request (by hand,
>> >> not
>> >> >> > using
>> >> >> > > > the
>> >> >> > > > > >> > > producer API) with a random PID (random, hence
>> unlikely
>> >> to
>> >> >> be
>> >> >> > in
>> >> >> > > > > use),
>> >> >> > > > > >> > will
>> >> >> > > > > >> > > the broker append a transactional message to the logs,
>> >> >> > > preventing
>> >> >> > > > > LSO
>> >> >> > > > > >> > from
>> >> >> > > > > >> > > moving forward? What validation will broker do for
>> PIDs?
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > Yes, that is correct. Validation of the TransactionalId
>> to
>> >> PID
>> >> >> > > > binding
>> >> >> > > > > >> is a
>> >> >> > > > > >> > known gap in the current proposal, and is discussed in
>> the
>> >> >> > design
>> >> >> > > > > >> document.
>> >> >> > > > > >> > Now that I'm thinking about it a bit more, I think there
>> >> is a
>> >> >> > good
>> >> >> > > > > case
>> >> >> > > > > >> for
>> >> >> > > > > >> > including the TransactionalId in the ProduceRequest (I
>> >> think
>> >> >> Jun
>> >> >> > > > > >> suggested
>> >> >> > > > > >> > this previously). Verifying it does not ensure that the
>> >> >> included
>> >> >> > > PID
>> >> >> > > > > is
>> >> >> > > > > >> > correct, but it does ensure that the client is
>> authorized
>> >> to
>> >> >> use
>> >> >> > > > > >> > transactions. If the client wanted to do an "endless
>> >> >> transaction
>> >> >> > > > > >> attack,"
>> >> >> > > > > >> > having Write access to the topic and an authorized
>> >> >> > transactionalID
>> >> >> > > > is
>> >> >> > > > > >> all
>> >> >> > > > > >> > they would need anyway even if we could authorize the
>> PID
>> >> >> > itself.
>> >> >> > > > This
>> >> >> > > > > >> > seems like a worthwhile improvement.
>> >> >> > > > > >> >
>> >> >> > > > > >> > For future work, my half-baked idea to authorize the PID
>> >> >> binding
>> >> >> > > is
>> >> >> > > > to
>> >> >> > > > > >> > leverage the delegation work in KIP-48. When the PID is
>> >> >> > generated,
>> >> >> > > > we
>> >> >> > > > > >> can
>> >> >> > > > > >> > give the producer a token which is then used in produce
>> >> >> requests
>> >> >> > > > (say
>> >> >> > > > > an
>> >> >> > > > > >> > hmac covering the TransactionalId and PID).
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > > 3. Will every broker check that a client sending
>> >> >> transactional
>> >> >> > > > > produce
>> >> >> > > > > >> > > requests at least has write access to transaction log
>> >> topic
>> >> >> > > since
>> >> >> > > > it
>> >> >> > > > > >> is
>> >> >> > > > > >> > not
>> >> >> > > > > >> > > validating transactional.id (for every produce
>> request)?
>> >> >> > > > > >> >
>> >> >> > > > > >> >  4. I understand that brokers cannot authorize the
>> >> >> transactional
>> >> >> > > id
>> >> >> > > > > for
>> >> >> > > > > >> > each
>> >> >> > > > > >> > > produce request since requests contain only the PID.
>> But
>> >> >> since
>> >> >> > > > there
>> >> >> > > > > >> is a
>> >> >> > > > > >> > > one-to-one mapping between PID and transactional.id,
>> >> and a
>> >> >> > > > > >> connection is
>> >> >> > > > > >> > > never expected to change its transactional.id,
>> perhaps
>> >> it
>> >> >> is
>> >> >> > > > > >> feasible to
>> >> >> > > > > >> > > add authorization and cache the results in the
>> Session?
>> >> >> > Perhaps
>> >> >> > > > not
>> >> >> > > > > >> for
>> >> >> > > > > >> > > version 1, but feels like it will be good to close the
>> >> >> > security
>> >> >> > > > gap
>> >> >> > > > > >> here.
>> >> >> > > > > >> > > Obviously it would be simpler if transactional.id
>> was in
>> >> >> the
>> >> >> > > > > produce
>> >> >> > > > > >> > > request if the overhead was acceptable.
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > I think my response above addresses both of these. We
>> >> should
>> >> >> > > include
>> >> >> > > > > the
>> >> >> > > > > >> > TransactionalId in the ProduceRequest. Of course it need
>> >> not
>> >> >> be
>> >> >> > > > > >> included in
>> >> >> > > > > >> > the message format, so I'm not too concerned about the
>> >> >> > additional
>> >> >> > > > > >> overhead
>> >> >> > > > > >> > it adds.
>> >> >> > > > > >> >
>> >> >> > > > > >> > Thanks,
>> >> >> > > > > >> > Jason
>> >> >> > > > > >> >
>> >> >> > > > > >> >
>> >> >> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
>> >> >> ismael@juma.me.uk>
>> >> >> > > > > wrote:
>> >> >> > > > > >> >
>> >> >> > > > > >> > > Comments inline.
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
>> >> >> > > > jason@confluent.io
>> >> >> > > > > >
>> >> >> > > > > >> > > wrote:
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > > Took me a while to remember why we didn't do this.
>> The
>> >> >> > > timestamp
>> >> >> > > > > >> that
>> >> >> > > > > >> > is
>> >> >> > > > > >> > > > included at the message set level is the max
>> timestamp
>> >> of
>> >> >> > all
>> >> >> > > > > >> messages
>> >> >> > > > > >> > in
>> >> >> > > > > >> > > > the message set as is the case in the current
>> message
>> >> >> format
>> >> >> > > (I
>> >> >> > > > > will
>> >> >> > > > > >> > > update
>> >> >> > > > > >> > > > the document to make this explicit). We could make
>> the
>> >> >> > message
>> >> >> > > > > >> > timestamps
>> >> >> > > > > >> > > > relative to the max timestamp, but that makes
>> >> >> serialization
>> >> >> > a
>> >> >> > > > bit
>> >> >> > > > > >> > awkward
>> >> >> > > > > >> > > > since the timestamps are not assumed to be
>> increasing
>> >> >> > > > sequentially
>> >> >> > > > > >> or
>> >> >> > > > > >> > > > monotonically. Once the messages in the message set
>> had
>> >> >> been
>> >> >> > > > > >> > determined,
>> >> >> > > > > >> > > we
>> >> >> > > > > >> > > > would need to go back and adjust the relative
>> >> timestamps.
>> >> >> > > > > >> > > >
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > Yes, I thought this would be a bit tricky and hence
>> why I
>> >> >> > > > mentioned
>> >> >> > > > > >> the
>> >> >> > > > > >> > > option of adding a new field at the message set level
>> for
>> >> >> the
>> >> >> > > > first
>> >> >> > > > > >> > > timestamp even though that's not ideal either.
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > Here's one idea. We let the timestamps in the
>> messages be
>> >> >> > > varints,
>> >> >> > > > > >> but we
>> >> >> > > > > >> > > > make their values be relative to the timestamp of
>> the
>> >> >> > previous
>> >> >> > > > > >> message,
>> >> >> > > > > >> > > > with the timestamp of the first message being
>> absolute.
>> >> >> For
>> >> >> > > > > >> example, if
>> >> >> > > > > >> > > we
>> >> >> > > > > >> > > > had timestamps 500, 501, 499, then we would write
>> 500
>> >> for
>> >> >> > the
>> >> >> > > > > first
>> >> >> > > > > >> > > > message, 1 for the next, and -2 for the final
>> message.
>> >> >> Would
>> >> >> > > > that
>> >> >> > > > > >> work?
>> >> >> > > > > >> > > Let
>> >> >> > > > > >> > > > me think a bit about it and see if there are any
>> >> problems.
>> >> >> > > > > >> > > >
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > It's an interesting idea. Comparing to the option of
>> >> having
>> >> >> > the
>> >> >> > > > > first
>> >> >> > > > > >> > > timestamp in the message set, It's a little more space
>> >> >> > efficient
>> >> >> > > > as
>> >> >> > > > > we
>> >> >> > > > > >> > > don't have both a full timestamp in the message set
>> >> _and_ a
>> >> >> > > varint
>> >> >> > > > > in
>> >> >> > > > > >> the
>> >> >> > > > > >> > > first message (which would always be 0, so we avoid
>> the
>> >> >> extra
>> >> >> > > > byte)
>> >> >> > > > > >> and
>> >> >> > > > > >> > > also the deltas could be a little smaller in the
>> common
>> >> >> case.
>> >> >> > > The
>> >> >> > > > > main
>> >> >> > > > > >> > > downside is that it introduces a semantics
>> inconsistency
>> >> >> > between
>> >> >> > > > the
>> >> >> > > > > >> > first
>> >> >> > > > > >> > > message and the rest. Not ideal, but maybe we can live
>> >> with
>> >> >> > > that.
>> >> >> > > > > >> > >
>> >> >> > > > > >> > > Ismael
>> >> >> > > > > >> > >
>> >> >> > > > > >> >
>> >> >> > > > > >>
>> >> >> > > > > >
>> >> >> > > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >> >
>> >> >>
>> >>
>> >>
>> >>
>> >> --
>> >> Gwen Shapira
>> >> Product Manager | Confluent
>> >> 650.450.2760 | @gwenshap
>> >> Follow us: Twitter | blog
>> >>
>>
>>
>>
>> --
>> Gwen Shapira
>> Product Manager | Confluent
>> 650.450.2760 | @gwenshap
>> Follow us: Twitter | blog
>>



-- 
Gwen Shapira
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
The point about management of transactionalId was raised before, and Jason
tackled it here (note it was called 'AppId' back then):
https://www.mail-archive.com/dev@kafka.apache.org/msg61200.html

1. Essentially, the full transactional functionality really only makes
sense for a stateful application. Further, in a stateful and partitioned
application, each shard must have a stable unique id. This can be used
directly as the transactionalId.

2. For non-stateful applications which want transactions within a single
session of a producer, they can generate a UUID per session, and use that
as the transactionalId.

So yes, managing transactionalIds are not trivial, but stateful
applications will have the means to do so.

The final note is that the primary way to use these transactional apis is
through Kafka streams, and in that context, the transactionalId will be a
combination of the streams appId and the streams taskId. Note that this is
essentially the shardId alluded to in point 1 above, and Streams can take
care of it transparently.

Thanks,
Apurva



On Fri, Feb 10, 2017 at 2:33 PM, Gwen Shapira <gw...@confluent.io> wrote:

> Thank you so much for the detailed explanation!
> I got the READ COMMITTED behavior. It is very odd that we will
> routinely see parts of a committed transaction but not other parts.
> But I understand the reasons.
>
> Regarding transactional.id: I read the document but I'm still not 100%
> clear on how unique transactional ids can happen in practice.
> I'm sure you've thought of this, but I'd like to understand:
> Right now, I'm writing an consume-process-produce app, I give it a
> consumer group.id, and it is safe to deploy the app multiple times
> with the same configuration - because all the instances will have the
> same group.id. But it sounds like each instance will need its own
> transactional.id, so it can commit its own transactions? Is the idea
> that I'll read the configuration from a file and an admin will be
> responsible to come up with unique transactional IDs while deploying?
> This sounds a bit more complex than I'd like. Maybe you have a simple
> solution I am not seeing...
>
> Gwen
>
>
>
>
> On Fri, Feb 10, 2017 at 12:25 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
> > Hey Gwen,
> >
> > Thanks for the questions and comments. Responses below:
> >
> > I not sure I'm clear on the expected behavior of READ_COMMITTED in
> >> some interleaved cases:
> >> * If a transaction starts, sends few messages and someone writes
> >> non-transactional event into the same topic/partition, few more events
> >> from that transaction and then a commit. I think the producer will
> >> block until the commit, but I'm not sure in what order I'll see events
> >> after that.
> >
> > * Same for transaction A starts, transactions B starts, transaction B
> >> commits, transaction A commits... when will we unblock? and what will
> >> we see?
> >
> >
> > In READ_COMMITTED (as well as in READ_UNCOMMITTED), the messages are
> always
> > returned in the order of their offsets. What controls the visibility of
> > messages in READ_COMMITTED is the last stable offset (LSO). This is
> defined
> > as the last offset in the log such that all messages with smaller offsets
> > have been decided (either committed or aborted).
> >
> > Take the first case: non-transactional data interleaved with
> transactional
> > data. Let N be a non-transactional message, T be a transactional message,
> > and C be a commit marker. Suppose we have the following state of the log
> > (the first row is the offsets, the second is the messages):
> >
> > 0, 1, 2
> > N, T, N
> >
> > The LSO in this log is 1, which means the non-transactional message at
> > offset 0 can be read by a consumer, but the one at offset 2 cannot
> because
> > the transaction beginning at offset 1 has not completed. Later we get
> some
> > more data:
> >
> > 0, 1, 2, 3, 4
> > N, T, N, N, T
> >
> > The LSO still cannot advance because the transaction beginning at offset
> 1
> > has not completed. Finally we get the commit marker:
> >
> > 0, 1, 2, 3, 4, 5
> > N, T, N, N, T, C
> >
> > At this point, the LSO advances to 6, and all of the messages from
> offset 1
> > to 5 are returned, in that order, to consumers.
> >
> > Now consider interleaved transactions. Let T1 be a transactional message
> > from producer 1, and T2 be a transactional message from producer 2. Both
> > producers begin a transaction:
> >
> > 0, 1, 2
> > T1, T2, T1
> >
> > The LSO is 0 since the transaction starting from that offset has not been
> > decided. Now suppose T1 commits.
> >
> > 0, 1, 2, 3
> > T1, T2, T1, C1
> >
> > The LSO then advances to offset 1. The message at offset 0 becomes
> visible
> > to consumers in READ_COMMITTED mode, but the remaining messages from the
> > transaction are blocked until T2 is decided. Now T2 commits:
> >
> > 0, 1, 2, 3, 4, 5
> > T1, T2, T1, C1, T2, C2
> >
> > The LSO advances to 6, and just like above, all of the data from offsets
> 1
> > to 5 becomes visible.
> >
> > This ordering is what we refer to as "offset order." We always return
> > messages in the order of their offsets. Another interesting option to
> > consider is "transaction order." Consider again the state after T1
> > committed:
> >
> > 0, 1, 2, 3
> > T1, T2, T1, C1
> >
> > In transaction order, all of the messages from T1 become visible
> > immediately upon receiving the commit marker. Later when T2 commits, its
> > messages will also becomes visible. The main advantage is lower latency,
> > but it is quite a bit more complicated to implement (see the design
> > document for the discussion). We went with the simpler approach and left
> > this for possible future work.
> >
> > i'm concerned about transactional.id uniqueness in real-world
> conditions.
> >> When users were forced to manually define unique broker ids, we saw
> >> lots of errors with "duplicate broker id" on startup show up on the
> >> mailing list.
> >> When we added a persistent (to filesystem) auto-generated broker ID,
> >> the number of issues dropped dramatically.
> >> I wonder if this makes sense here too.
> >
> >
> > There has been a lot of discussion about this. I will link you to the
> > section in the design document:
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#heading=h.lizhp2urqn59
> > .
> >
> > It sounds like producers will now be idempotent by default. I.e. - if
> >> we upgrade and do nothing different, we have idempotent producer? and
> >> absolutely no way to get the old behavior back?
> >
> >
> > This may have been a case of the KIP wiki getting out of sync with the
> > design document. I've updated it to include the `enable.idempotence`
> > producer config which was added previously to the design document. By
> > default, idempotence will be disabled. Down the road, it may make sense
> to
> > change that, but we're uncomfortable turning it on in the first version
> > because of the subtle changes in behavior (in particular, the fact that
> we
> > raise the fatal OutOfSequenceException in the case of message loss or PID
> > expiration).
> >
> > Thanks,
> > Jason
> >
> > On Thu, Feb 9, 2017 at 10:41 PM, Gwen Shapira <gw...@confluent.io> wrote:
> >
> >> It sounds like producers will now be idempotent by default. I.e. - if
> >> we upgrade and do nothing different, we have idempotent producer? and
> >> absolutely no way to get the old behavior back?
> >>
> >> I don't think anyone needs non-idempotent producer, just want to
> clarify :)
> >>
> >> On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >> > A couple follow-ups from Ismael's previous comments.
> >> >
> >> > 1. We have removed the individual message CRC field from the message
> >> > format. Because this field can already change on the broker in some
> >> > situations, we feel it is probably not wise to let clients depend on
> it,
> >> > and removing it saves some space and the redundant verification on the
> >> > broker. We have also proposed to deprecate the checksum() APIs which
> are
> >> > exposed in RecordMetadata and ConsumerRecord.
> >> > 2. We changed the message timestamp field to be variable length. The
> >> values
> >> > are computed relative to the timestamp of the first message in the
> set.
> >> We
> >> > considered my previous suggestion to make the first message timestamp
> >> > absolute with later messages relative to their previous timestamp, but
> >> > ultimately felt it was simpler to stick with the same pattern that is
> >> used
> >> > for relative offsets. To enable this, we added an initial timestamp
> field
> >> > to the message set.
> >> >
> >> > Thanks,
> >> > Jason
> >> >
> >> > On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io>
> >> wrote:
> >> >
> >> >> Hello,
> >> >>
> >> >> I have gone ahead and updated the KIP wiki with a summary of the
> >> changes to
> >> >> the RPC protocol. The KIP wiki should now have _all_ the public
> facing
> >> >> changes being proposed.
> >> >>
> >> >> The proposed changes were always in the Google doc, and now we are
> >> simply
> >> >> making good on our promise to copy them over to the wiki since the
> >> design
> >> >> is almost finalized.
> >> >>
> >> >> Thanks,
> >> >> Apurva
> >> >>
> >> >> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
> >> >> wrote:
> >> >>
> >> >> > Hey Tom,
> >> >> >
> >> >> > Re; complexity. This is always a tradeoff with new features. The
> >> changes
> >> >> > we've made during the design and review process have greatly
> >> simplified
> >> >> the
> >> >> > implementation for clients, and especially for the consumer, but
> >> there is
> >> >> > nontrivial work needed here to support transactions on the
> producer.
> >> I'm
> >> >> > not sure how it could be otherwise and we've spent a ton of time
> >> thinking
> >> >> > about this. It's also worth mentioning that there's a relatively
> low
> >> bar
> >> >> to
> >> >> > support the idempotent producer while keeping the client thin (it
> >> >> requires
> >> >> > support for one new request type and some sequence bookkeeping).
> >> >> >
> >> >> > Ultimately, we have to decide whether the improved semantics are
> worth
> >> >> the
> >> >> > cost of the complexity. In my opinion, they are. The benefit of
> having
> >> >> > exactly-once processing in streaming applications is huge. And to
> >> manage
> >> >> > the complexity, we've intentionally used patterns that were already
> >> >> > familiar in the codebase (e.g. our approach to maintaining
> transaction
> >> >> > state through a coordinator is copied from how consumer offsets are
> >> >> > managed). We've moved as much work from the clients to the broker
> as
> >> >> > possible, and we have resisted at every turn complicating the
> client
> >> APIs
> >> >> > even where it may have simplified some internals.
> >> >> >
> >> >> > -Jason
> >> >> >
> >> >> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
> >> >> wrote:
> >> >> >
> >> >> > > Hi Tom,
> >> >> > >
> >> >> > > I updated the KIP with a note our plans for performance testing:
> >> >> > >
> >> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> >> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
> >> >> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
> >> >> > >
> >> >> > > Thanks for pointing that out.
> >> >> > >
> >> >> > > Regards,
> >> >> > > Apurva
> >> >> > >
> >> >> > >
> >> >> > >
> >> >> > >
> >> >> > >
> >> >> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <
> tcrayford@heroku.com>
> >> >> > wrote:
> >> >> > >
> >> >> > > > I think the updated wiki page makes sense with respect to ACLs,
> >> there
> >> >> > > seems
> >> >> > > > to be little potential for abuse there (other than the noted
> and
> >> >> known
> >> >> > > > issues).
> >> >> > > >
> >> >> > > > I am going to note that this is a major complexity increase for
> >> >> Kafka,
> >> >> > > and
> >> >> > > > that I'm concerned about performance impact (the JVM is quite…
> >> >> pedantic
> >> >> > > > about method size, for example, and even adding conditionals to
> >> >> larger
> >> >> > > > methods could impact this). The KIP doesn't note plans for
> >> >> performance
> >> >> > > > testing.
> >> >> > > >
> >> >> > > > I'm also concerned about the impact on non-JVM client
> libraries -
> >> >> > > writing a
> >> >> > > > client for Kafka is already a very complicated undertaking, and
> >> this
> >> >> > adds
> >> >> > > > to that complexity significantly.
> >> >> > > >
> >> >> > > > However, the approach seems ok enough. It does also violate the
> >> >> "Kafka
> >> >> > > has
> >> >> > > > dumb brokers and smart clients" (which I realize is in direct
> >> >> > > contradiction
> >> >> > > > of the previous statement about client implementation being
> hard).
> >> >> I'd
> >> >> > > love
> >> >> > > > to see some discussion in either the doc or the wiki as to why
> >> much
> >> >> of
> >> >> > > this
> >> >> > > > transactional work isn't a client-side part of Kafka Streams.
> >> >> > > >
> >> >> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <
> >> jason@confluent.io>
> >> >> > > > wrote:
> >> >> > > >
> >> >> > > > > One additional note on the authorization. The WriteTxnMarker
> >> API is
> >> >> > > > > restricted to inter-broker usage, so it requires Cluster
> >> >> > authorization
> >> >> > > > > (just like other inter-broker APIs). I've updated the
> document
> >> and
> >> >> > wiki
> >> >> > > > to
> >> >> > > > > reflect this.
> >> >> > > > >
> >> >> > > > > Also, I have renamed GroupCoordinatorRequest to
> >> >> > FindCoordinatorRequest
> >> >> > > > > since there is no group for transactional producers. Let me
> >> know if
> >> >> > > there
> >> >> > > > > are any concerns.
> >> >> > > > >
> >> >> > > > > -Jason
> >> >> > > > >
> >> >> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
> >> >> jason@confluent.io>
> >> >> > > > > wrote:
> >> >> > > > >
> >> >> > > > > > Hi Tom,
> >> >> > > > > >
> >> >> > > > > > I said this in the voting thread, but can the authors
> include
> >> a
> >> >> > > section
> >> >> > > > > >> about new ACLs if there are going to be ACLs for
> >> >> TransactionalId.
> >> >> > > It's
> >> >> > > > > >> mentioned in the google doc, but I think new ACLs should
> be
> >> in a
> >> >> > KIP
> >> >> > > > > >> directly.
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > We've updated the wiki. Can you take a look and let us
> know if
> >> >> you
> >> >> > > have
> >> >> > > > > > additional concerns?
> >> >> > > > > >
> >> >> > > > > > Thanks,
> >> >> > > > > > Jason
> >> >> > > > > >
> >> >> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> >> >> > > > rajinisivaram@gmail.com>
> >> >> > > > > > wrote:
> >> >> > > > > >
> >> >> > > > > >> Hi Jason,
> >> >> > > > > >>
> >> >> > > > > >> Thank you for the responses. Agree that authorizing
> >> >> > > transactional.id
> >> >> > > > in
> >> >> > > > > >> the
> >> >> > > > > >> producer requests will be good enough for version 1. And
> >> making
> >> >> it
> >> >> > > > > tighter
> >> >> > > > > >> in future based on delegation tokens sounds good too.
> >> >> > > > > >>
> >> >> > > > > >> Regards,
> >> >> > > > > >>
> >> >> > > > > >> Rajini
> >> >> > > > > >>
> >> >> > > > > >>
> >> >> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
> >> >> > jason@confluent.io
> >> >> > > >
> >> >> > > > > >> wrote:
> >> >> > > > > >>
> >> >> > > > > >> > Hey Rajini,
> >> >> > > > > >> >
> >> >> > > > > >> > Thanks for the questions. Responses below:
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > > 1. Will the transaction coordinator check topic ACLs
> >> based
> >> >> on
> >> >> > > the
> >> >> > > > > >> > > requesting client's credentials? Access to transaction
> >> logs,
> >> >> > > > topics
> >> >> > > > > >> being
> >> >> > > > > >> > > added for transaction etc?
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > Good question. I think it makes sense to check topic
> Write
> >> >> > > > permission
> >> >> > > > > >> when
> >> >> > > > > >> > adding partitions to the transaction. I'll add this to
> the
> >> >> > > document.
> >> >> > > > > >> > Perhaps authorization to the transaction log itself,
> >> however,
> >> >> > can
> >> >> > > be
> >> >> > > > > >> > assumed from having access to the
> ProducerTransactionalId
> >> >> > > resource?
> >> >> > > > > This
> >> >> > > > > >> > would be similar to how access to __consumer_offsets is
> >> >> assumed
> >> >> > if
> >> >> > > > the
> >> >> > > > > >> > client has access to the Group resource.
> >> >> > > > > >> >
> >> >> > > > > >> > 2. If I create a transactional produce request (by hand,
> >> not
> >> >> > using
> >> >> > > > the
> >> >> > > > > >> > > producer API) with a random PID (random, hence
> unlikely
> >> to
> >> >> be
> >> >> > in
> >> >> > > > > use),
> >> >> > > > > >> > will
> >> >> > > > > >> > > the broker append a transactional message to the logs,
> >> >> > > preventing
> >> >> > > > > LSO
> >> >> > > > > >> > from
> >> >> > > > > >> > > moving forward? What validation will broker do for
> PIDs?
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > Yes, that is correct. Validation of the TransactionalId
> to
> >> PID
> >> >> > > > binding
> >> >> > > > > >> is a
> >> >> > > > > >> > known gap in the current proposal, and is discussed in
> the
> >> >> > design
> >> >> > > > > >> document.
> >> >> > > > > >> > Now that I'm thinking about it a bit more, I think there
> >> is a
> >> >> > good
> >> >> > > > > case
> >> >> > > > > >> for
> >> >> > > > > >> > including the TransactionalId in the ProduceRequest (I
> >> think
> >> >> Jun
> >> >> > > > > >> suggested
> >> >> > > > > >> > this previously). Verifying it does not ensure that the
> >> >> included
> >> >> > > PID
> >> >> > > > > is
> >> >> > > > > >> > correct, but it does ensure that the client is
> authorized
> >> to
> >> >> use
> >> >> > > > > >> > transactions. If the client wanted to do an "endless
> >> >> transaction
> >> >> > > > > >> attack,"
> >> >> > > > > >> > having Write access to the topic and an authorized
> >> >> > transactionalID
> >> >> > > > is
> >> >> > > > > >> all
> >> >> > > > > >> > they would need anyway even if we could authorize the
> PID
> >> >> > itself.
> >> >> > > > This
> >> >> > > > > >> > seems like a worthwhile improvement.
> >> >> > > > > >> >
> >> >> > > > > >> > For future work, my half-baked idea to authorize the PID
> >> >> binding
> >> >> > > is
> >> >> > > > to
> >> >> > > > > >> > leverage the delegation work in KIP-48. When the PID is
> >> >> > generated,
> >> >> > > > we
> >> >> > > > > >> can
> >> >> > > > > >> > give the producer a token which is then used in produce
> >> >> requests
> >> >> > > > (say
> >> >> > > > > an
> >> >> > > > > >> > hmac covering the TransactionalId and PID).
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > > 3. Will every broker check that a client sending
> >> >> transactional
> >> >> > > > > produce
> >> >> > > > > >> > > requests at least has write access to transaction log
> >> topic
> >> >> > > since
> >> >> > > > it
> >> >> > > > > >> is
> >> >> > > > > >> > not
> >> >> > > > > >> > > validating transactional.id (for every produce
> request)?
> >> >> > > > > >> >
> >> >> > > > > >> >  4. I understand that brokers cannot authorize the
> >> >> transactional
> >> >> > > id
> >> >> > > > > for
> >> >> > > > > >> > each
> >> >> > > > > >> > > produce request since requests contain only the PID.
> But
> >> >> since
> >> >> > > > there
> >> >> > > > > >> is a
> >> >> > > > > >> > > one-to-one mapping between PID and transactional.id,
> >> and a
> >> >> > > > > >> connection is
> >> >> > > > > >> > > never expected to change its transactional.id,
> perhaps
> >> it
> >> >> is
> >> >> > > > > >> feasible to
> >> >> > > > > >> > > add authorization and cache the results in the
> Session?
> >> >> > Perhaps
> >> >> > > > not
> >> >> > > > > >> for
> >> >> > > > > >> > > version 1, but feels like it will be good to close the
> >> >> > security
> >> >> > > > gap
> >> >> > > > > >> here.
> >> >> > > > > >> > > Obviously it would be simpler if transactional.id
> was in
> >> >> the
> >> >> > > > > produce
> >> >> > > > > >> > > request if the overhead was acceptable.
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > I think my response above addresses both of these. We
> >> should
> >> >> > > include
> >> >> > > > > the
> >> >> > > > > >> > TransactionalId in the ProduceRequest. Of course it need
> >> not
> >> >> be
> >> >> > > > > >> included in
> >> >> > > > > >> > the message format, so I'm not too concerned about the
> >> >> > additional
> >> >> > > > > >> overhead
> >> >> > > > > >> > it adds.
> >> >> > > > > >> >
> >> >> > > > > >> > Thanks,
> >> >> > > > > >> > Jason
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
> >> >> ismael@juma.me.uk>
> >> >> > > > > wrote:
> >> >> > > > > >> >
> >> >> > > > > >> > > Comments inline.
> >> >> > > > > >> > >
> >> >> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> >> >> > > > jason@confluent.io
> >> >> > > > > >
> >> >> > > > > >> > > wrote:
> >> >> > > > > >> > >
> >> >> > > > > >> > > > Took me a while to remember why we didn't do this.
> The
> >> >> > > timestamp
> >> >> > > > > >> that
> >> >> > > > > >> > is
> >> >> > > > > >> > > > included at the message set level is the max
> timestamp
> >> of
> >> >> > all
> >> >> > > > > >> messages
> >> >> > > > > >> > in
> >> >> > > > > >> > > > the message set as is the case in the current
> message
> >> >> format
> >> >> > > (I
> >> >> > > > > will
> >> >> > > > > >> > > update
> >> >> > > > > >> > > > the document to make this explicit). We could make
> the
> >> >> > message
> >> >> > > > > >> > timestamps
> >> >> > > > > >> > > > relative to the max timestamp, but that makes
> >> >> serialization
> >> >> > a
> >> >> > > > bit
> >> >> > > > > >> > awkward
> >> >> > > > > >> > > > since the timestamps are not assumed to be
> increasing
> >> >> > > > sequentially
> >> >> > > > > >> or
> >> >> > > > > >> > > > monotonically. Once the messages in the message set
> had
> >> >> been
> >> >> > > > > >> > determined,
> >> >> > > > > >> > > we
> >> >> > > > > >> > > > would need to go back and adjust the relative
> >> timestamps.
> >> >> > > > > >> > > >
> >> >> > > > > >> > >
> >> >> > > > > >> > > Yes, I thought this would be a bit tricky and hence
> why I
> >> >> > > > mentioned
> >> >> > > > > >> the
> >> >> > > > > >> > > option of adding a new field at the message set level
> for
> >> >> the
> >> >> > > > first
> >> >> > > > > >> > > timestamp even though that's not ideal either.
> >> >> > > > > >> > >
> >> >> > > > > >> > > Here's one idea. We let the timestamps in the
> messages be
> >> >> > > varints,
> >> >> > > > > >> but we
> >> >> > > > > >> > > > make their values be relative to the timestamp of
> the
> >> >> > previous
> >> >> > > > > >> message,
> >> >> > > > > >> > > > with the timestamp of the first message being
> absolute.
> >> >> For
> >> >> > > > > >> example, if
> >> >> > > > > >> > > we
> >> >> > > > > >> > > > had timestamps 500, 501, 499, then we would write
> 500
> >> for
> >> >> > the
> >> >> > > > > first
> >> >> > > > > >> > > > message, 1 for the next, and -2 for the final
> message.
> >> >> Would
> >> >> > > > that
> >> >> > > > > >> work?
> >> >> > > > > >> > > Let
> >> >> > > > > >> > > > me think a bit about it and see if there are any
> >> problems.
> >> >> > > > > >> > > >
> >> >> > > > > >> > >
> >> >> > > > > >> > > It's an interesting idea. Comparing to the option of
> >> having
> >> >> > the
> >> >> > > > > first
> >> >> > > > > >> > > timestamp in the message set, It's a little more space
> >> >> > efficient
> >> >> > > > as
> >> >> > > > > we
> >> >> > > > > >> > > don't have both a full timestamp in the message set
> >> _and_ a
> >> >> > > varint
> >> >> > > > > in
> >> >> > > > > >> the
> >> >> > > > > >> > > first message (which would always be 0, so we avoid
> the
> >> >> extra
> >> >> > > > byte)
> >> >> > > > > >> and
> >> >> > > > > >> > > also the deltas could be a little smaller in the
> common
> >> >> case.
> >> >> > > The
> >> >> > > > > main
> >> >> > > > > >> > > downside is that it introduces a semantics
> inconsistency
> >> >> > between
> >> >> > > > the
> >> >> > > > > >> > first
> >> >> > > > > >> > > message and the rest. Not ideal, but maybe we can live
> >> with
> >> >> > > that.
> >> >> > > > > >> > >
> >> >> > > > > >> > > Ismael
> >> >> > > > > >> > >
> >> >> > > > > >> >
> >> >> > > > > >>
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> >>
> >>
> >> --
> >> Gwen Shapira
> >> Product Manager | Confluent
> >> 650.450.2760 | @gwenshap
> >> Follow us: Twitter | blog
> >>
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Gwen Shapira <gw...@confluent.io>.
Thank you so much for the detailed explanation!
I got the READ COMMITTED behavior. It is very odd that we will
routinely see parts of a committed transaction but not other parts.
But I understand the reasons.

Regarding transactional.id: I read the document but I'm still not 100%
clear on how unique transactional ids can happen in practice.
I'm sure you've thought of this, but I'd like to understand:
Right now, I'm writing an consume-process-produce app, I give it a
consumer group.id, and it is safe to deploy the app multiple times
with the same configuration - because all the instances will have the
same group.id. But it sounds like each instance will need its own
transactional.id, so it can commit its own transactions? Is the idea
that I'll read the configuration from a file and an admin will be
responsible to come up with unique transactional IDs while deploying?
This sounds a bit more complex than I'd like. Maybe you have a simple
solution I am not seeing...

Gwen




On Fri, Feb 10, 2017 at 12:25 AM, Jason Gustafson <ja...@confluent.io> wrote:
> Hey Gwen,
>
> Thanks for the questions and comments. Responses below:
>
> I not sure I'm clear on the expected behavior of READ_COMMITTED in
>> some interleaved cases:
>> * If a transaction starts, sends few messages and someone writes
>> non-transactional event into the same topic/partition, few more events
>> from that transaction and then a commit. I think the producer will
>> block until the commit, but I'm not sure in what order I'll see events
>> after that.
>
> * Same for transaction A starts, transactions B starts, transaction B
>> commits, transaction A commits... when will we unblock? and what will
>> we see?
>
>
> In READ_COMMITTED (as well as in READ_UNCOMMITTED), the messages are always
> returned in the order of their offsets. What controls the visibility of
> messages in READ_COMMITTED is the last stable offset (LSO). This is defined
> as the last offset in the log such that all messages with smaller offsets
> have been decided (either committed or aborted).
>
> Take the first case: non-transactional data interleaved with transactional
> data. Let N be a non-transactional message, T be a transactional message,
> and C be a commit marker. Suppose we have the following state of the log
> (the first row is the offsets, the second is the messages):
>
> 0, 1, 2
> N, T, N
>
> The LSO in this log is 1, which means the non-transactional message at
> offset 0 can be read by a consumer, but the one at offset 2 cannot because
> the transaction beginning at offset 1 has not completed. Later we get some
> more data:
>
> 0, 1, 2, 3, 4
> N, T, N, N, T
>
> The LSO still cannot advance because the transaction beginning at offset 1
> has not completed. Finally we get the commit marker:
>
> 0, 1, 2, 3, 4, 5
> N, T, N, N, T, C
>
> At this point, the LSO advances to 6, and all of the messages from offset 1
> to 5 are returned, in that order, to consumers.
>
> Now consider interleaved transactions. Let T1 be a transactional message
> from producer 1, and T2 be a transactional message from producer 2. Both
> producers begin a transaction:
>
> 0, 1, 2
> T1, T2, T1
>
> The LSO is 0 since the transaction starting from that offset has not been
> decided. Now suppose T1 commits.
>
> 0, 1, 2, 3
> T1, T2, T1, C1
>
> The LSO then advances to offset 1. The message at offset 0 becomes visible
> to consumers in READ_COMMITTED mode, but the remaining messages from the
> transaction are blocked until T2 is decided. Now T2 commits:
>
> 0, 1, 2, 3, 4, 5
> T1, T2, T1, C1, T2, C2
>
> The LSO advances to 6, and just like above, all of the data from offsets 1
> to 5 becomes visible.
>
> This ordering is what we refer to as "offset order." We always return
> messages in the order of their offsets. Another interesting option to
> consider is "transaction order." Consider again the state after T1
> committed:
>
> 0, 1, 2, 3
> T1, T2, T1, C1
>
> In transaction order, all of the messages from T1 become visible
> immediately upon receiving the commit marker. Later when T2 commits, its
> messages will also becomes visible. The main advantage is lower latency,
> but it is quite a bit more complicated to implement (see the design
> document for the discussion). We went with the simpler approach and left
> this for possible future work.
>
> i'm concerned about transactional.id uniqueness in real-world conditions.
>> When users were forced to manually define unique broker ids, we saw
>> lots of errors with "duplicate broker id" on startup show up on the
>> mailing list.
>> When we added a persistent (to filesystem) auto-generated broker ID,
>> the number of issues dropped dramatically.
>> I wonder if this makes sense here too.
>
>
> There has been a lot of discussion about this. I will link you to the
> section in the design document:
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.lizhp2urqn59
> .
>
> It sounds like producers will now be idempotent by default. I.e. - if
>> we upgrade and do nothing different, we have idempotent producer? and
>> absolutely no way to get the old behavior back?
>
>
> This may have been a case of the KIP wiki getting out of sync with the
> design document. I've updated it to include the `enable.idempotence`
> producer config which was added previously to the design document. By
> default, idempotence will be disabled. Down the road, it may make sense to
> change that, but we're uncomfortable turning it on in the first version
> because of the subtle changes in behavior (in particular, the fact that we
> raise the fatal OutOfSequenceException in the case of message loss or PID
> expiration).
>
> Thanks,
> Jason
>
> On Thu, Feb 9, 2017 at 10:41 PM, Gwen Shapira <gw...@confluent.io> wrote:
>
>> It sounds like producers will now be idempotent by default. I.e. - if
>> we upgrade and do nothing different, we have idempotent producer? and
>> absolutely no way to get the old behavior back?
>>
>> I don't think anyone needs non-idempotent producer, just want to clarify :)
>>
>> On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>> > A couple follow-ups from Ismael's previous comments.
>> >
>> > 1. We have removed the individual message CRC field from the message
>> > format. Because this field can already change on the broker in some
>> > situations, we feel it is probably not wise to let clients depend on it,
>> > and removing it saves some space and the redundant verification on the
>> > broker. We have also proposed to deprecate the checksum() APIs which are
>> > exposed in RecordMetadata and ConsumerRecord.
>> > 2. We changed the message timestamp field to be variable length. The
>> values
>> > are computed relative to the timestamp of the first message in the set.
>> We
>> > considered my previous suggestion to make the first message timestamp
>> > absolute with later messages relative to their previous timestamp, but
>> > ultimately felt it was simpler to stick with the same pattern that is
>> used
>> > for relative offsets. To enable this, we added an initial timestamp field
>> > to the message set.
>> >
>> > Thanks,
>> > Jason
>> >
>> > On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io>
>> wrote:
>> >
>> >> Hello,
>> >>
>> >> I have gone ahead and updated the KIP wiki with a summary of the
>> changes to
>> >> the RPC protocol. The KIP wiki should now have _all_ the public facing
>> >> changes being proposed.
>> >>
>> >> The proposed changes were always in the Google doc, and now we are
>> simply
>> >> making good on our promise to copy them over to the wiki since the
>> design
>> >> is almost finalized.
>> >>
>> >> Thanks,
>> >> Apurva
>> >>
>> >> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >>
>> >> > Hey Tom,
>> >> >
>> >> > Re; complexity. This is always a tradeoff with new features. The
>> changes
>> >> > we've made during the design and review process have greatly
>> simplified
>> >> the
>> >> > implementation for clients, and especially for the consumer, but
>> there is
>> >> > nontrivial work needed here to support transactions on the producer.
>> I'm
>> >> > not sure how it could be otherwise and we've spent a ton of time
>> thinking
>> >> > about this. It's also worth mentioning that there's a relatively low
>> bar
>> >> to
>> >> > support the idempotent producer while keeping the client thin (it
>> >> requires
>> >> > support for one new request type and some sequence bookkeeping).
>> >> >
>> >> > Ultimately, we have to decide whether the improved semantics are worth
>> >> the
>> >> > cost of the complexity. In my opinion, they are. The benefit of having
>> >> > exactly-once processing in streaming applications is huge. And to
>> manage
>> >> > the complexity, we've intentionally used patterns that were already
>> >> > familiar in the codebase (e.g. our approach to maintaining transaction
>> >> > state through a coordinator is copied from how consumer offsets are
>> >> > managed). We've moved as much work from the clients to the broker as
>> >> > possible, and we have resisted at every turn complicating the client
>> APIs
>> >> > even where it may have simplified some internals.
>> >> >
>> >> > -Jason
>> >> >
>> >> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
>> >> wrote:
>> >> >
>> >> > > Hi Tom,
>> >> > >
>> >> > > I updated the KIP with a note our plans for performance testing:
>> >> > >
>> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
>> >> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
>> >> > >
>> >> > > Thanks for pointing that out.
>> >> > >
>> >> > > Regards,
>> >> > > Apurva
>> >> > >
>> >> > >
>> >> > >
>> >> > >
>> >> > >
>> >> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com>
>> >> > wrote:
>> >> > >
>> >> > > > I think the updated wiki page makes sense with respect to ACLs,
>> there
>> >> > > seems
>> >> > > > to be little potential for abuse there (other than the noted and
>> >> known
>> >> > > > issues).
>> >> > > >
>> >> > > > I am going to note that this is a major complexity increase for
>> >> Kafka,
>> >> > > and
>> >> > > > that I'm concerned about performance impact (the JVM is quite…
>> >> pedantic
>> >> > > > about method size, for example, and even adding conditionals to
>> >> larger
>> >> > > > methods could impact this). The KIP doesn't note plans for
>> >> performance
>> >> > > > testing.
>> >> > > >
>> >> > > > I'm also concerned about the impact on non-JVM client libraries -
>> >> > > writing a
>> >> > > > client for Kafka is already a very complicated undertaking, and
>> this
>> >> > adds
>> >> > > > to that complexity significantly.
>> >> > > >
>> >> > > > However, the approach seems ok enough. It does also violate the
>> >> "Kafka
>> >> > > has
>> >> > > > dumb brokers and smart clients" (which I realize is in direct
>> >> > > contradiction
>> >> > > > of the previous statement about client implementation being hard).
>> >> I'd
>> >> > > love
>> >> > > > to see some discussion in either the doc or the wiki as to why
>> much
>> >> of
>> >> > > this
>> >> > > > transactional work isn't a client-side part of Kafka Streams.
>> >> > > >
>> >> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <
>> jason@confluent.io>
>> >> > > > wrote:
>> >> > > >
>> >> > > > > One additional note on the authorization. The WriteTxnMarker
>> API is
>> >> > > > > restricted to inter-broker usage, so it requires Cluster
>> >> > authorization
>> >> > > > > (just like other inter-broker APIs). I've updated the document
>> and
>> >> > wiki
>> >> > > > to
>> >> > > > > reflect this.
>> >> > > > >
>> >> > > > > Also, I have renamed GroupCoordinatorRequest to
>> >> > FindCoordinatorRequest
>> >> > > > > since there is no group for transactional producers. Let me
>> know if
>> >> > > there
>> >> > > > > are any concerns.
>> >> > > > >
>> >> > > > > -Jason
>> >> > > > >
>> >> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
>> >> jason@confluent.io>
>> >> > > > > wrote:
>> >> > > > >
>> >> > > > > > Hi Tom,
>> >> > > > > >
>> >> > > > > > I said this in the voting thread, but can the authors include
>> a
>> >> > > section
>> >> > > > > >> about new ACLs if there are going to be ACLs for
>> >> TransactionalId.
>> >> > > It's
>> >> > > > > >> mentioned in the google doc, but I think new ACLs should be
>> in a
>> >> > KIP
>> >> > > > > >> directly.
>> >> > > > > >
>> >> > > > > >
>> >> > > > > > We've updated the wiki. Can you take a look and let us know if
>> >> you
>> >> > > have
>> >> > > > > > additional concerns?
>> >> > > > > >
>> >> > > > > > Thanks,
>> >> > > > > > Jason
>> >> > > > > >
>> >> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
>> >> > > > rajinisivaram@gmail.com>
>> >> > > > > > wrote:
>> >> > > > > >
>> >> > > > > >> Hi Jason,
>> >> > > > > >>
>> >> > > > > >> Thank you for the responses. Agree that authorizing
>> >> > > transactional.id
>> >> > > > in
>> >> > > > > >> the
>> >> > > > > >> producer requests will be good enough for version 1. And
>> making
>> >> it
>> >> > > > > tighter
>> >> > > > > >> in future based on delegation tokens sounds good too.
>> >> > > > > >>
>> >> > > > > >> Regards,
>> >> > > > > >>
>> >> > > > > >> Rajini
>> >> > > > > >>
>> >> > > > > >>
>> >> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
>> >> > jason@confluent.io
>> >> > > >
>> >> > > > > >> wrote:
>> >> > > > > >>
>> >> > > > > >> > Hey Rajini,
>> >> > > > > >> >
>> >> > > > > >> > Thanks for the questions. Responses below:
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > > 1. Will the transaction coordinator check topic ACLs
>> based
>> >> on
>> >> > > the
>> >> > > > > >> > > requesting client's credentials? Access to transaction
>> logs,
>> >> > > > topics
>> >> > > > > >> being
>> >> > > > > >> > > added for transaction etc?
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > Good question. I think it makes sense to check topic Write
>> >> > > > permission
>> >> > > > > >> when
>> >> > > > > >> > adding partitions to the transaction. I'll add this to the
>> >> > > document.
>> >> > > > > >> > Perhaps authorization to the transaction log itself,
>> however,
>> >> > can
>> >> > > be
>> >> > > > > >> > assumed from having access to the ProducerTransactionalId
>> >> > > resource?
>> >> > > > > This
>> >> > > > > >> > would be similar to how access to __consumer_offsets is
>> >> assumed
>> >> > if
>> >> > > > the
>> >> > > > > >> > client has access to the Group resource.
>> >> > > > > >> >
>> >> > > > > >> > 2. If I create a transactional produce request (by hand,
>> not
>> >> > using
>> >> > > > the
>> >> > > > > >> > > producer API) with a random PID (random, hence unlikely
>> to
>> >> be
>> >> > in
>> >> > > > > use),
>> >> > > > > >> > will
>> >> > > > > >> > > the broker append a transactional message to the logs,
>> >> > > preventing
>> >> > > > > LSO
>> >> > > > > >> > from
>> >> > > > > >> > > moving forward? What validation will broker do for PIDs?
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > Yes, that is correct. Validation of the TransactionalId to
>> PID
>> >> > > > binding
>> >> > > > > >> is a
>> >> > > > > >> > known gap in the current proposal, and is discussed in the
>> >> > design
>> >> > > > > >> document.
>> >> > > > > >> > Now that I'm thinking about it a bit more, I think there
>> is a
>> >> > good
>> >> > > > > case
>> >> > > > > >> for
>> >> > > > > >> > including the TransactionalId in the ProduceRequest (I
>> think
>> >> Jun
>> >> > > > > >> suggested
>> >> > > > > >> > this previously). Verifying it does not ensure that the
>> >> included
>> >> > > PID
>> >> > > > > is
>> >> > > > > >> > correct, but it does ensure that the client is authorized
>> to
>> >> use
>> >> > > > > >> > transactions. If the client wanted to do an "endless
>> >> transaction
>> >> > > > > >> attack,"
>> >> > > > > >> > having Write access to the topic and an authorized
>> >> > transactionalID
>> >> > > > is
>> >> > > > > >> all
>> >> > > > > >> > they would need anyway even if we could authorize the PID
>> >> > itself.
>> >> > > > This
>> >> > > > > >> > seems like a worthwhile improvement.
>> >> > > > > >> >
>> >> > > > > >> > For future work, my half-baked idea to authorize the PID
>> >> binding
>> >> > > is
>> >> > > > to
>> >> > > > > >> > leverage the delegation work in KIP-48. When the PID is
>> >> > generated,
>> >> > > > we
>> >> > > > > >> can
>> >> > > > > >> > give the producer a token which is then used in produce
>> >> requests
>> >> > > > (say
>> >> > > > > an
>> >> > > > > >> > hmac covering the TransactionalId and PID).
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > > 3. Will every broker check that a client sending
>> >> transactional
>> >> > > > > produce
>> >> > > > > >> > > requests at least has write access to transaction log
>> topic
>> >> > > since
>> >> > > > it
>> >> > > > > >> is
>> >> > > > > >> > not
>> >> > > > > >> > > validating transactional.id (for every produce request)?
>> >> > > > > >> >
>> >> > > > > >> >  4. I understand that brokers cannot authorize the
>> >> transactional
>> >> > > id
>> >> > > > > for
>> >> > > > > >> > each
>> >> > > > > >> > > produce request since requests contain only the PID. But
>> >> since
>> >> > > > there
>> >> > > > > >> is a
>> >> > > > > >> > > one-to-one mapping between PID and transactional.id,
>> and a
>> >> > > > > >> connection is
>> >> > > > > >> > > never expected to change its transactional.id, perhaps
>> it
>> >> is
>> >> > > > > >> feasible to
>> >> > > > > >> > > add authorization and cache the results in the Session?
>> >> > Perhaps
>> >> > > > not
>> >> > > > > >> for
>> >> > > > > >> > > version 1, but feels like it will be good to close the
>> >> > security
>> >> > > > gap
>> >> > > > > >> here.
>> >> > > > > >> > > Obviously it would be simpler if transactional.id was in
>> >> the
>> >> > > > > produce
>> >> > > > > >> > > request if the overhead was acceptable.
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > I think my response above addresses both of these. We
>> should
>> >> > > include
>> >> > > > > the
>> >> > > > > >> > TransactionalId in the ProduceRequest. Of course it need
>> not
>> >> be
>> >> > > > > >> included in
>> >> > > > > >> > the message format, so I'm not too concerned about the
>> >> > additional
>> >> > > > > >> overhead
>> >> > > > > >> > it adds.
>> >> > > > > >> >
>> >> > > > > >> > Thanks,
>> >> > > > > >> > Jason
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
>> >> ismael@juma.me.uk>
>> >> > > > > wrote:
>> >> > > > > >> >
>> >> > > > > >> > > Comments inline.
>> >> > > > > >> > >
>> >> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
>> >> > > > jason@confluent.io
>> >> > > > > >
>> >> > > > > >> > > wrote:
>> >> > > > > >> > >
>> >> > > > > >> > > > Took me a while to remember why we didn't do this. The
>> >> > > timestamp
>> >> > > > > >> that
>> >> > > > > >> > is
>> >> > > > > >> > > > included at the message set level is the max timestamp
>> of
>> >> > all
>> >> > > > > >> messages
>> >> > > > > >> > in
>> >> > > > > >> > > > the message set as is the case in the current message
>> >> format
>> >> > > (I
>> >> > > > > will
>> >> > > > > >> > > update
>> >> > > > > >> > > > the document to make this explicit). We could make the
>> >> > message
>> >> > > > > >> > timestamps
>> >> > > > > >> > > > relative to the max timestamp, but that makes
>> >> serialization
>> >> > a
>> >> > > > bit
>> >> > > > > >> > awkward
>> >> > > > > >> > > > since the timestamps are not assumed to be increasing
>> >> > > > sequentially
>> >> > > > > >> or
>> >> > > > > >> > > > monotonically. Once the messages in the message set had
>> >> been
>> >> > > > > >> > determined,
>> >> > > > > >> > > we
>> >> > > > > >> > > > would need to go back and adjust the relative
>> timestamps.
>> >> > > > > >> > > >
>> >> > > > > >> > >
>> >> > > > > >> > > Yes, I thought this would be a bit tricky and hence why I
>> >> > > > mentioned
>> >> > > > > >> the
>> >> > > > > >> > > option of adding a new field at the message set level for
>> >> the
>> >> > > > first
>> >> > > > > >> > > timestamp even though that's not ideal either.
>> >> > > > > >> > >
>> >> > > > > >> > > Here's one idea. We let the timestamps in the messages be
>> >> > > varints,
>> >> > > > > >> but we
>> >> > > > > >> > > > make their values be relative to the timestamp of the
>> >> > previous
>> >> > > > > >> message,
>> >> > > > > >> > > > with the timestamp of the first message being absolute.
>> >> For
>> >> > > > > >> example, if
>> >> > > > > >> > > we
>> >> > > > > >> > > > had timestamps 500, 501, 499, then we would write 500
>> for
>> >> > the
>> >> > > > > first
>> >> > > > > >> > > > message, 1 for the next, and -2 for the final message.
>> >> Would
>> >> > > > that
>> >> > > > > >> work?
>> >> > > > > >> > > Let
>> >> > > > > >> > > > me think a bit about it and see if there are any
>> problems.
>> >> > > > > >> > > >
>> >> > > > > >> > >
>> >> > > > > >> > > It's an interesting idea. Comparing to the option of
>> having
>> >> > the
>> >> > > > > first
>> >> > > > > >> > > timestamp in the message set, It's a little more space
>> >> > efficient
>> >> > > > as
>> >> > > > > we
>> >> > > > > >> > > don't have both a full timestamp in the message set
>> _and_ a
>> >> > > varint
>> >> > > > > in
>> >> > > > > >> the
>> >> > > > > >> > > first message (which would always be 0, so we avoid the
>> >> extra
>> >> > > > byte)
>> >> > > > > >> and
>> >> > > > > >> > > also the deltas could be a little smaller in the common
>> >> case.
>> >> > > The
>> >> > > > > main
>> >> > > > > >> > > downside is that it introduces a semantics inconsistency
>> >> > between
>> >> > > > the
>> >> > > > > >> > first
>> >> > > > > >> > > message and the rest. Not ideal, but maybe we can live
>> with
>> >> > > that.
>> >> > > > > >> > >
>> >> > > > > >> > > Ismael
>> >> > > > > >> > >
>> >> > > > > >> >
>> >> > > > > >>
>> >> > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>>
>>
>>
>> --
>> Gwen Shapira
>> Product Manager | Confluent
>> 650.450.2760 | @gwenshap
>> Follow us: Twitter | blog
>>



-- 
Gwen Shapira
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Gwen,

Thanks for the questions and comments. Responses below:

I not sure I'm clear on the expected behavior of READ_COMMITTED in
> some interleaved cases:
> * If a transaction starts, sends few messages and someone writes
> non-transactional event into the same topic/partition, few more events
> from that transaction and then a commit. I think the producer will
> block until the commit, but I'm not sure in what order I'll see events
> after that.

* Same for transaction A starts, transactions B starts, transaction B
> commits, transaction A commits... when will we unblock? and what will
> we see?


In READ_COMMITTED (as well as in READ_UNCOMMITTED), the messages are always
returned in the order of their offsets. What controls the visibility of
messages in READ_COMMITTED is the last stable offset (LSO). This is defined
as the last offset in the log such that all messages with smaller offsets
have been decided (either committed or aborted).

Take the first case: non-transactional data interleaved with transactional
data. Let N be a non-transactional message, T be a transactional message,
and C be a commit marker. Suppose we have the following state of the log
(the first row is the offsets, the second is the messages):

0, 1, 2
N, T, N

The LSO in this log is 1, which means the non-transactional message at
offset 0 can be read by a consumer, but the one at offset 2 cannot because
the transaction beginning at offset 1 has not completed. Later we get some
more data:

0, 1, 2, 3, 4
N, T, N, N, T

The LSO still cannot advance because the transaction beginning at offset 1
has not completed. Finally we get the commit marker:

0, 1, 2, 3, 4, 5
N, T, N, N, T, C

At this point, the LSO advances to 6, and all of the messages from offset 1
to 5 are returned, in that order, to consumers.

Now consider interleaved transactions. Let T1 be a transactional message
from producer 1, and T2 be a transactional message from producer 2. Both
producers begin a transaction:

0, 1, 2
T1, T2, T1

The LSO is 0 since the transaction starting from that offset has not been
decided. Now suppose T1 commits.

0, 1, 2, 3
T1, T2, T1, C1

The LSO then advances to offset 1. The message at offset 0 becomes visible
to consumers in READ_COMMITTED mode, but the remaining messages from the
transaction are blocked until T2 is decided. Now T2 commits:

0, 1, 2, 3, 4, 5
T1, T2, T1, C1, T2, C2

The LSO advances to 6, and just like above, all of the data from offsets 1
to 5 becomes visible.

This ordering is what we refer to as "offset order." We always return
messages in the order of their offsets. Another interesting option to
consider is "transaction order." Consider again the state after T1
committed:

0, 1, 2, 3
T1, T2, T1, C1

In transaction order, all of the messages from T1 become visible
immediately upon receiving the commit marker. Later when T2 commits, its
messages will also becomes visible. The main advantage is lower latency,
but it is quite a bit more complicated to implement (see the design
document for the discussion). We went with the simpler approach and left
this for possible future work.

i'm concerned about transactional.id uniqueness in real-world conditions.
> When users were forced to manually define unique broker ids, we saw
> lots of errors with "duplicate broker id" on startup show up on the
> mailing list.
> When we added a persistent (to filesystem) auto-generated broker ID,
> the number of issues dropped dramatically.
> I wonder if this makes sense here too.


There has been a lot of discussion about this. I will link you to the
section in the design document:
https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.lizhp2urqn59
.

It sounds like producers will now be idempotent by default. I.e. - if
> we upgrade and do nothing different, we have idempotent producer? and
> absolutely no way to get the old behavior back?


This may have been a case of the KIP wiki getting out of sync with the
design document. I've updated it to include the `enable.idempotence`
producer config which was added previously to the design document. By
default, idempotence will be disabled. Down the road, it may make sense to
change that, but we're uncomfortable turning it on in the first version
because of the subtle changes in behavior (in particular, the fact that we
raise the fatal OutOfSequenceException in the case of message loss or PID
expiration).

Thanks,
Jason

On Thu, Feb 9, 2017 at 10:41 PM, Gwen Shapira <gw...@confluent.io> wrote:

> It sounds like producers will now be idempotent by default. I.e. - if
> we upgrade and do nothing different, we have idempotent producer? and
> absolutely no way to get the old behavior back?
>
> I don't think anyone needs non-idempotent producer, just want to clarify :)
>
> On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
> > A couple follow-ups from Ismael's previous comments.
> >
> > 1. We have removed the individual message CRC field from the message
> > format. Because this field can already change on the broker in some
> > situations, we feel it is probably not wise to let clients depend on it,
> > and removing it saves some space and the redundant verification on the
> > broker. We have also proposed to deprecate the checksum() APIs which are
> > exposed in RecordMetadata and ConsumerRecord.
> > 2. We changed the message timestamp field to be variable length. The
> values
> > are computed relative to the timestamp of the first message in the set.
> We
> > considered my previous suggestion to make the first message timestamp
> > absolute with later messages relative to their previous timestamp, but
> > ultimately felt it was simpler to stick with the same pattern that is
> used
> > for relative offsets. To enable this, we added an initial timestamp field
> > to the message set.
> >
> > Thanks,
> > Jason
> >
> > On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> >> Hello,
> >>
> >> I have gone ahead and updated the KIP wiki with a summary of the
> changes to
> >> the RPC protocol. The KIP wiki should now have _all_ the public facing
> >> changes being proposed.
> >>
> >> The proposed changes were always in the Google doc, and now we are
> simply
> >> making good on our promise to copy them over to the wiki since the
> design
> >> is almost finalized.
> >>
> >> Thanks,
> >> Apurva
> >>
> >> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >>
> >> > Hey Tom,
> >> >
> >> > Re; complexity. This is always a tradeoff with new features. The
> changes
> >> > we've made during the design and review process have greatly
> simplified
> >> the
> >> > implementation for clients, and especially for the consumer, but
> there is
> >> > nontrivial work needed here to support transactions on the producer.
> I'm
> >> > not sure how it could be otherwise and we've spent a ton of time
> thinking
> >> > about this. It's also worth mentioning that there's a relatively low
> bar
> >> to
> >> > support the idempotent producer while keeping the client thin (it
> >> requires
> >> > support for one new request type and some sequence bookkeeping).
> >> >
> >> > Ultimately, we have to decide whether the improved semantics are worth
> >> the
> >> > cost of the complexity. In my opinion, they are. The benefit of having
> >> > exactly-once processing in streaming applications is huge. And to
> manage
> >> > the complexity, we've intentionally used patterns that were already
> >> > familiar in the codebase (e.g. our approach to maintaining transaction
> >> > state through a coordinator is copied from how consumer offsets are
> >> > managed). We've moved as much work from the clients to the broker as
> >> > possible, and we have resisted at every turn complicating the client
> APIs
> >> > even where it may have simplified some internals.
> >> >
> >> > -Jason
> >> >
> >> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
> >> wrote:
> >> >
> >> > > Hi Tom,
> >> > >
> >> > > I updated the KIP with a note our plans for performance testing:
> >> > >
> >> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
> >> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
> >> > >
> >> > > Thanks for pointing that out.
> >> > >
> >> > > Regards,
> >> > > Apurva
> >> > >
> >> > >
> >> > >
> >> > >
> >> > >
> >> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com>
> >> > wrote:
> >> > >
> >> > > > I think the updated wiki page makes sense with respect to ACLs,
> there
> >> > > seems
> >> > > > to be little potential for abuse there (other than the noted and
> >> known
> >> > > > issues).
> >> > > >
> >> > > > I am going to note that this is a major complexity increase for
> >> Kafka,
> >> > > and
> >> > > > that I'm concerned about performance impact (the JVM is quite…
> >> pedantic
> >> > > > about method size, for example, and even adding conditionals to
> >> larger
> >> > > > methods could impact this). The KIP doesn't note plans for
> >> performance
> >> > > > testing.
> >> > > >
> >> > > > I'm also concerned about the impact on non-JVM client libraries -
> >> > > writing a
> >> > > > client for Kafka is already a very complicated undertaking, and
> this
> >> > adds
> >> > > > to that complexity significantly.
> >> > > >
> >> > > > However, the approach seems ok enough. It does also violate the
> >> "Kafka
> >> > > has
> >> > > > dumb brokers and smart clients" (which I realize is in direct
> >> > > contradiction
> >> > > > of the previous statement about client implementation being hard).
> >> I'd
> >> > > love
> >> > > > to see some discussion in either the doc or the wiki as to why
> much
> >> of
> >> > > this
> >> > > > transactional work isn't a client-side part of Kafka Streams.
> >> > > >
> >> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <
> jason@confluent.io>
> >> > > > wrote:
> >> > > >
> >> > > > > One additional note on the authorization. The WriteTxnMarker
> API is
> >> > > > > restricted to inter-broker usage, so it requires Cluster
> >> > authorization
> >> > > > > (just like other inter-broker APIs). I've updated the document
> and
> >> > wiki
> >> > > > to
> >> > > > > reflect this.
> >> > > > >
> >> > > > > Also, I have renamed GroupCoordinatorRequest to
> >> > FindCoordinatorRequest
> >> > > > > since there is no group for transactional producers. Let me
> know if
> >> > > there
> >> > > > > are any concerns.
> >> > > > >
> >> > > > > -Jason
> >> > > > >
> >> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
> >> jason@confluent.io>
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi Tom,
> >> > > > > >
> >> > > > > > I said this in the voting thread, but can the authors include
> a
> >> > > section
> >> > > > > >> about new ACLs if there are going to be ACLs for
> >> TransactionalId.
> >> > > It's
> >> > > > > >> mentioned in the google doc, but I think new ACLs should be
> in a
> >> > KIP
> >> > > > > >> directly.
> >> > > > > >
> >> > > > > >
> >> > > > > > We've updated the wiki. Can you take a look and let us know if
> >> you
> >> > > have
> >> > > > > > additional concerns?
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Jason
> >> > > > > >
> >> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> >> > > > rajinisivaram@gmail.com>
> >> > > > > > wrote:
> >> > > > > >
> >> > > > > >> Hi Jason,
> >> > > > > >>
> >> > > > > >> Thank you for the responses. Agree that authorizing
> >> > > transactional.id
> >> > > > in
> >> > > > > >> the
> >> > > > > >> producer requests will be good enough for version 1. And
> making
> >> it
> >> > > > > tighter
> >> > > > > >> in future based on delegation tokens sounds good too.
> >> > > > > >>
> >> > > > > >> Regards,
> >> > > > > >>
> >> > > > > >> Rajini
> >> > > > > >>
> >> > > > > >>
> >> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
> >> > jason@confluent.io
> >> > > >
> >> > > > > >> wrote:
> >> > > > > >>
> >> > > > > >> > Hey Rajini,
> >> > > > > >> >
> >> > > > > >> > Thanks for the questions. Responses below:
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > > 1. Will the transaction coordinator check topic ACLs
> based
> >> on
> >> > > the
> >> > > > > >> > > requesting client's credentials? Access to transaction
> logs,
> >> > > > topics
> >> > > > > >> being
> >> > > > > >> > > added for transaction etc?
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > Good question. I think it makes sense to check topic Write
> >> > > > permission
> >> > > > > >> when
> >> > > > > >> > adding partitions to the transaction. I'll add this to the
> >> > > document.
> >> > > > > >> > Perhaps authorization to the transaction log itself,
> however,
> >> > can
> >> > > be
> >> > > > > >> > assumed from having access to the ProducerTransactionalId
> >> > > resource?
> >> > > > > This
> >> > > > > >> > would be similar to how access to __consumer_offsets is
> >> assumed
> >> > if
> >> > > > the
> >> > > > > >> > client has access to the Group resource.
> >> > > > > >> >
> >> > > > > >> > 2. If I create a transactional produce request (by hand,
> not
> >> > using
> >> > > > the
> >> > > > > >> > > producer API) with a random PID (random, hence unlikely
> to
> >> be
> >> > in
> >> > > > > use),
> >> > > > > >> > will
> >> > > > > >> > > the broker append a transactional message to the logs,
> >> > > preventing
> >> > > > > LSO
> >> > > > > >> > from
> >> > > > > >> > > moving forward? What validation will broker do for PIDs?
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > Yes, that is correct. Validation of the TransactionalId to
> PID
> >> > > > binding
> >> > > > > >> is a
> >> > > > > >> > known gap in the current proposal, and is discussed in the
> >> > design
> >> > > > > >> document.
> >> > > > > >> > Now that I'm thinking about it a bit more, I think there
> is a
> >> > good
> >> > > > > case
> >> > > > > >> for
> >> > > > > >> > including the TransactionalId in the ProduceRequest (I
> think
> >> Jun
> >> > > > > >> suggested
> >> > > > > >> > this previously). Verifying it does not ensure that the
> >> included
> >> > > PID
> >> > > > > is
> >> > > > > >> > correct, but it does ensure that the client is authorized
> to
> >> use
> >> > > > > >> > transactions. If the client wanted to do an "endless
> >> transaction
> >> > > > > >> attack,"
> >> > > > > >> > having Write access to the topic and an authorized
> >> > transactionalID
> >> > > > is
> >> > > > > >> all
> >> > > > > >> > they would need anyway even if we could authorize the PID
> >> > itself.
> >> > > > This
> >> > > > > >> > seems like a worthwhile improvement.
> >> > > > > >> >
> >> > > > > >> > For future work, my half-baked idea to authorize the PID
> >> binding
> >> > > is
> >> > > > to
> >> > > > > >> > leverage the delegation work in KIP-48. When the PID is
> >> > generated,
> >> > > > we
> >> > > > > >> can
> >> > > > > >> > give the producer a token which is then used in produce
> >> requests
> >> > > > (say
> >> > > > > an
> >> > > > > >> > hmac covering the TransactionalId and PID).
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > > 3. Will every broker check that a client sending
> >> transactional
> >> > > > > produce
> >> > > > > >> > > requests at least has write access to transaction log
> topic
> >> > > since
> >> > > > it
> >> > > > > >> is
> >> > > > > >> > not
> >> > > > > >> > > validating transactional.id (for every produce request)?
> >> > > > > >> >
> >> > > > > >> >  4. I understand that brokers cannot authorize the
> >> transactional
> >> > > id
> >> > > > > for
> >> > > > > >> > each
> >> > > > > >> > > produce request since requests contain only the PID. But
> >> since
> >> > > > there
> >> > > > > >> is a
> >> > > > > >> > > one-to-one mapping between PID and transactional.id,
> and a
> >> > > > > >> connection is
> >> > > > > >> > > never expected to change its transactional.id, perhaps
> it
> >> is
> >> > > > > >> feasible to
> >> > > > > >> > > add authorization and cache the results in the Session?
> >> > Perhaps
> >> > > > not
> >> > > > > >> for
> >> > > > > >> > > version 1, but feels like it will be good to close the
> >> > security
> >> > > > gap
> >> > > > > >> here.
> >> > > > > >> > > Obviously it would be simpler if transactional.id was in
> >> the
> >> > > > > produce
> >> > > > > >> > > request if the overhead was acceptable.
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > I think my response above addresses both of these. We
> should
> >> > > include
> >> > > > > the
> >> > > > > >> > TransactionalId in the ProduceRequest. Of course it need
> not
> >> be
> >> > > > > >> included in
> >> > > > > >> > the message format, so I'm not too concerned about the
> >> > additional
> >> > > > > >> overhead
> >> > > > > >> > it adds.
> >> > > > > >> >
> >> > > > > >> > Thanks,
> >> > > > > >> > Jason
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
> >> ismael@juma.me.uk>
> >> > > > > wrote:
> >> > > > > >> >
> >> > > > > >> > > Comments inline.
> >> > > > > >> > >
> >> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> >> > > > jason@confluent.io
> >> > > > > >
> >> > > > > >> > > wrote:
> >> > > > > >> > >
> >> > > > > >> > > > Took me a while to remember why we didn't do this. The
> >> > > timestamp
> >> > > > > >> that
> >> > > > > >> > is
> >> > > > > >> > > > included at the message set level is the max timestamp
> of
> >> > all
> >> > > > > >> messages
> >> > > > > >> > in
> >> > > > > >> > > > the message set as is the case in the current message
> >> format
> >> > > (I
> >> > > > > will
> >> > > > > >> > > update
> >> > > > > >> > > > the document to make this explicit). We could make the
> >> > message
> >> > > > > >> > timestamps
> >> > > > > >> > > > relative to the max timestamp, but that makes
> >> serialization
> >> > a
> >> > > > bit
> >> > > > > >> > awkward
> >> > > > > >> > > > since the timestamps are not assumed to be increasing
> >> > > > sequentially
> >> > > > > >> or
> >> > > > > >> > > > monotonically. Once the messages in the message set had
> >> been
> >> > > > > >> > determined,
> >> > > > > >> > > we
> >> > > > > >> > > > would need to go back and adjust the relative
> timestamps.
> >> > > > > >> > > >
> >> > > > > >> > >
> >> > > > > >> > > Yes, I thought this would be a bit tricky and hence why I
> >> > > > mentioned
> >> > > > > >> the
> >> > > > > >> > > option of adding a new field at the message set level for
> >> the
> >> > > > first
> >> > > > > >> > > timestamp even though that's not ideal either.
> >> > > > > >> > >
> >> > > > > >> > > Here's one idea. We let the timestamps in the messages be
> >> > > varints,
> >> > > > > >> but we
> >> > > > > >> > > > make their values be relative to the timestamp of the
> >> > previous
> >> > > > > >> message,
> >> > > > > >> > > > with the timestamp of the first message being absolute.
> >> For
> >> > > > > >> example, if
> >> > > > > >> > > we
> >> > > > > >> > > > had timestamps 500, 501, 499, then we would write 500
> for
> >> > the
> >> > > > > first
> >> > > > > >> > > > message, 1 for the next, and -2 for the final message.
> >> Would
> >> > > > that
> >> > > > > >> work?
> >> > > > > >> > > Let
> >> > > > > >> > > > me think a bit about it and see if there are any
> problems.
> >> > > > > >> > > >
> >> > > > > >> > >
> >> > > > > >> > > It's an interesting idea. Comparing to the option of
> having
> >> > the
> >> > > > > first
> >> > > > > >> > > timestamp in the message set, It's a little more space
> >> > efficient
> >> > > > as
> >> > > > > we
> >> > > > > >> > > don't have both a full timestamp in the message set
> _and_ a
> >> > > varint
> >> > > > > in
> >> > > > > >> the
> >> > > > > >> > > first message (which would always be 0, so we avoid the
> >> extra
> >> > > > byte)
> >> > > > > >> and
> >> > > > > >> > > also the deltas could be a little smaller in the common
> >> case.
> >> > > The
> >> > > > > main
> >> > > > > >> > > downside is that it introduces a semantics inconsistency
> >> > between
> >> > > > the
> >> > > > > >> > first
> >> > > > > >> > > message and the rest. Not ideal, but maybe we can live
> with
> >> > > that.
> >> > > > > >> > >
> >> > > > > >> > > Ismael
> >> > > > > >> > >
> >> > > > > >> >
> >> > > > > >>
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Gwen Shapira <gw...@confluent.io>.
It sounds like producers will now be idempotent by default. I.e. - if
we upgrade and do nothing different, we have idempotent producer? and
absolutely no way to get the old behavior back?

I don't think anyone needs non-idempotent producer, just want to clarify :)

On Tue, Feb 7, 2017 at 3:10 PM, Jason Gustafson <ja...@confluent.io> wrote:
> A couple follow-ups from Ismael's previous comments.
>
> 1. We have removed the individual message CRC field from the message
> format. Because this field can already change on the broker in some
> situations, we feel it is probably not wise to let clients depend on it,
> and removing it saves some space and the redundant verification on the
> broker. We have also proposed to deprecate the checksum() APIs which are
> exposed in RecordMetadata and ConsumerRecord.
> 2. We changed the message timestamp field to be variable length. The values
> are computed relative to the timestamp of the first message in the set. We
> considered my previous suggestion to make the first message timestamp
> absolute with later messages relative to their previous timestamp, but
> ultimately felt it was simpler to stick with the same pattern that is used
> for relative offsets. To enable this, we added an initial timestamp field
> to the message set.
>
> Thanks,
> Jason
>
> On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Hello,
>>
>> I have gone ahead and updated the KIP wiki with a summary of the changes to
>> the RPC protocol. The KIP wiki should now have _all_ the public facing
>> changes being proposed.
>>
>> The proposed changes were always in the Google doc, and now we are simply
>> making good on our promise to copy them over to the wiki since the design
>> is almost finalized.
>>
>> Thanks,
>> Apurva
>>
>> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>>
>> > Hey Tom,
>> >
>> > Re; complexity. This is always a tradeoff with new features. The changes
>> > we've made during the design and review process have greatly simplified
>> the
>> > implementation for clients, and especially for the consumer, but there is
>> > nontrivial work needed here to support transactions on the producer. I'm
>> > not sure how it could be otherwise and we've spent a ton of time thinking
>> > about this. It's also worth mentioning that there's a relatively low bar
>> to
>> > support the idempotent producer while keeping the client thin (it
>> requires
>> > support for one new request type and some sequence bookkeeping).
>> >
>> > Ultimately, we have to decide whether the improved semantics are worth
>> the
>> > cost of the complexity. In my opinion, they are. The benefit of having
>> > exactly-once processing in streaming applications is huge. And to manage
>> > the complexity, we've intentionally used patterns that were already
>> > familiar in the codebase (e.g. our approach to maintaining transaction
>> > state through a coordinator is copied from how consumer offsets are
>> > managed). We've moved as much work from the clients to the broker as
>> > possible, and we have resisted at every turn complicating the client APIs
>> > even where it may have simplified some internals.
>> >
>> > -Jason
>> >
>> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
>> wrote:
>> >
>> > > Hi Tom,
>> > >
>> > > I updated the KIP with a note our plans for performance testing:
>> > >
>> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
>> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
>> > >
>> > > Thanks for pointing that out.
>> > >
>> > > Regards,
>> > > Apurva
>> > >
>> > >
>> > >
>> > >
>> > >
>> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com>
>> > wrote:
>> > >
>> > > > I think the updated wiki page makes sense with respect to ACLs, there
>> > > seems
>> > > > to be little potential for abuse there (other than the noted and
>> known
>> > > > issues).
>> > > >
>> > > > I am going to note that this is a major complexity increase for
>> Kafka,
>> > > and
>> > > > that I'm concerned about performance impact (the JVM is quite…
>> pedantic
>> > > > about method size, for example, and even adding conditionals to
>> larger
>> > > > methods could impact this). The KIP doesn't note plans for
>> performance
>> > > > testing.
>> > > >
>> > > > I'm also concerned about the impact on non-JVM client libraries -
>> > > writing a
>> > > > client for Kafka is already a very complicated undertaking, and this
>> > adds
>> > > > to that complexity significantly.
>> > > >
>> > > > However, the approach seems ok enough. It does also violate the
>> "Kafka
>> > > has
>> > > > dumb brokers and smart clients" (which I realize is in direct
>> > > contradiction
>> > > > of the previous statement about client implementation being hard).
>> I'd
>> > > love
>> > > > to see some discussion in either the doc or the wiki as to why much
>> of
>> > > this
>> > > > transactional work isn't a client-side part of Kafka Streams.
>> > > >
>> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io>
>> > > > wrote:
>> > > >
>> > > > > One additional note on the authorization. The WriteTxnMarker API is
>> > > > > restricted to inter-broker usage, so it requires Cluster
>> > authorization
>> > > > > (just like other inter-broker APIs). I've updated the document and
>> > wiki
>> > > > to
>> > > > > reflect this.
>> > > > >
>> > > > > Also, I have renamed GroupCoordinatorRequest to
>> > FindCoordinatorRequest
>> > > > > since there is no group for transactional producers. Let me know if
>> > > there
>> > > > > are any concerns.
>> > > > >
>> > > > > -Jason
>> > > > >
>> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
>> jason@confluent.io>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi Tom,
>> > > > > >
>> > > > > > I said this in the voting thread, but can the authors include a
>> > > section
>> > > > > >> about new ACLs if there are going to be ACLs for
>> TransactionalId.
>> > > It's
>> > > > > >> mentioned in the google doc, but I think new ACLs should be in a
>> > KIP
>> > > > > >> directly.
>> > > > > >
>> > > > > >
>> > > > > > We've updated the wiki. Can you take a look and let us know if
>> you
>> > > have
>> > > > > > additional concerns?
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Jason
>> > > > > >
>> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
>> > > > rajinisivaram@gmail.com>
>> > > > > > wrote:
>> > > > > >
>> > > > > >> Hi Jason,
>> > > > > >>
>> > > > > >> Thank you for the responses. Agree that authorizing
>> > > transactional.id
>> > > > in
>> > > > > >> the
>> > > > > >> producer requests will be good enough for version 1. And making
>> it
>> > > > > tighter
>> > > > > >> in future based on delegation tokens sounds good too.
>> > > > > >>
>> > > > > >> Regards,
>> > > > > >>
>> > > > > >> Rajini
>> > > > > >>
>> > > > > >>
>> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
>> > jason@confluent.io
>> > > >
>> > > > > >> wrote:
>> > > > > >>
>> > > > > >> > Hey Rajini,
>> > > > > >> >
>> > > > > >> > Thanks for the questions. Responses below:
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > > 1. Will the transaction coordinator check topic ACLs based
>> on
>> > > the
>> > > > > >> > > requesting client's credentials? Access to transaction logs,
>> > > > topics
>> > > > > >> being
>> > > > > >> > > added for transaction etc?
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > Good question. I think it makes sense to check topic Write
>> > > > permission
>> > > > > >> when
>> > > > > >> > adding partitions to the transaction. I'll add this to the
>> > > document.
>> > > > > >> > Perhaps authorization to the transaction log itself, however,
>> > can
>> > > be
>> > > > > >> > assumed from having access to the ProducerTransactionalId
>> > > resource?
>> > > > > This
>> > > > > >> > would be similar to how access to __consumer_offsets is
>> assumed
>> > if
>> > > > the
>> > > > > >> > client has access to the Group resource.
>> > > > > >> >
>> > > > > >> > 2. If I create a transactional produce request (by hand, not
>> > using
>> > > > the
>> > > > > >> > > producer API) with a random PID (random, hence unlikely to
>> be
>> > in
>> > > > > use),
>> > > > > >> > will
>> > > > > >> > > the broker append a transactional message to the logs,
>> > > preventing
>> > > > > LSO
>> > > > > >> > from
>> > > > > >> > > moving forward? What validation will broker do for PIDs?
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > Yes, that is correct. Validation of the TransactionalId to PID
>> > > > binding
>> > > > > >> is a
>> > > > > >> > known gap in the current proposal, and is discussed in the
>> > design
>> > > > > >> document.
>> > > > > >> > Now that I'm thinking about it a bit more, I think there is a
>> > good
>> > > > > case
>> > > > > >> for
>> > > > > >> > including the TransactionalId in the ProduceRequest (I think
>> Jun
>> > > > > >> suggested
>> > > > > >> > this previously). Verifying it does not ensure that the
>> included
>> > > PID
>> > > > > is
>> > > > > >> > correct, but it does ensure that the client is authorized to
>> use
>> > > > > >> > transactions. If the client wanted to do an "endless
>> transaction
>> > > > > >> attack,"
>> > > > > >> > having Write access to the topic and an authorized
>> > transactionalID
>> > > > is
>> > > > > >> all
>> > > > > >> > they would need anyway even if we could authorize the PID
>> > itself.
>> > > > This
>> > > > > >> > seems like a worthwhile improvement.
>> > > > > >> >
>> > > > > >> > For future work, my half-baked idea to authorize the PID
>> binding
>> > > is
>> > > > to
>> > > > > >> > leverage the delegation work in KIP-48. When the PID is
>> > generated,
>> > > > we
>> > > > > >> can
>> > > > > >> > give the producer a token which is then used in produce
>> requests
>> > > > (say
>> > > > > an
>> > > > > >> > hmac covering the TransactionalId and PID).
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > > 3. Will every broker check that a client sending
>> transactional
>> > > > > produce
>> > > > > >> > > requests at least has write access to transaction log topic
>> > > since
>> > > > it
>> > > > > >> is
>> > > > > >> > not
>> > > > > >> > > validating transactional.id (for every produce request)?
>> > > > > >> >
>> > > > > >> >  4. I understand that brokers cannot authorize the
>> transactional
>> > > id
>> > > > > for
>> > > > > >> > each
>> > > > > >> > > produce request since requests contain only the PID. But
>> since
>> > > > there
>> > > > > >> is a
>> > > > > >> > > one-to-one mapping between PID and transactional.id, and a
>> > > > > >> connection is
>> > > > > >> > > never expected to change its transactional.id, perhaps it
>> is
>> > > > > >> feasible to
>> > > > > >> > > add authorization and cache the results in the Session?
>> > Perhaps
>> > > > not
>> > > > > >> for
>> > > > > >> > > version 1, but feels like it will be good to close the
>> > security
>> > > > gap
>> > > > > >> here.
>> > > > > >> > > Obviously it would be simpler if transactional.id was in
>> the
>> > > > > produce
>> > > > > >> > > request if the overhead was acceptable.
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > I think my response above addresses both of these. We should
>> > > include
>> > > > > the
>> > > > > >> > TransactionalId in the ProduceRequest. Of course it need not
>> be
>> > > > > >> included in
>> > > > > >> > the message format, so I'm not too concerned about the
>> > additional
>> > > > > >> overhead
>> > > > > >> > it adds.
>> > > > > >> >
>> > > > > >> > Thanks,
>> > > > > >> > Jason
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
>> ismael@juma.me.uk>
>> > > > > wrote:
>> > > > > >> >
>> > > > > >> > > Comments inline.
>> > > > > >> > >
>> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
>> > > > jason@confluent.io
>> > > > > >
>> > > > > >> > > wrote:
>> > > > > >> > >
>> > > > > >> > > > Took me a while to remember why we didn't do this. The
>> > > timestamp
>> > > > > >> that
>> > > > > >> > is
>> > > > > >> > > > included at the message set level is the max timestamp of
>> > all
>> > > > > >> messages
>> > > > > >> > in
>> > > > > >> > > > the message set as is the case in the current message
>> format
>> > > (I
>> > > > > will
>> > > > > >> > > update
>> > > > > >> > > > the document to make this explicit). We could make the
>> > message
>> > > > > >> > timestamps
>> > > > > >> > > > relative to the max timestamp, but that makes
>> serialization
>> > a
>> > > > bit
>> > > > > >> > awkward
>> > > > > >> > > > since the timestamps are not assumed to be increasing
>> > > > sequentially
>> > > > > >> or
>> > > > > >> > > > monotonically. Once the messages in the message set had
>> been
>> > > > > >> > determined,
>> > > > > >> > > we
>> > > > > >> > > > would need to go back and adjust the relative timestamps.
>> > > > > >> > > >
>> > > > > >> > >
>> > > > > >> > > Yes, I thought this would be a bit tricky and hence why I
>> > > > mentioned
>> > > > > >> the
>> > > > > >> > > option of adding a new field at the message set level for
>> the
>> > > > first
>> > > > > >> > > timestamp even though that's not ideal either.
>> > > > > >> > >
>> > > > > >> > > Here's one idea. We let the timestamps in the messages be
>> > > varints,
>> > > > > >> but we
>> > > > > >> > > > make their values be relative to the timestamp of the
>> > previous
>> > > > > >> message,
>> > > > > >> > > > with the timestamp of the first message being absolute.
>> For
>> > > > > >> example, if
>> > > > > >> > > we
>> > > > > >> > > > had timestamps 500, 501, 499, then we would write 500 for
>> > the
>> > > > > first
>> > > > > >> > > > message, 1 for the next, and -2 for the final message.
>> Would
>> > > > that
>> > > > > >> work?
>> > > > > >> > > Let
>> > > > > >> > > > me think a bit about it and see if there are any problems.
>> > > > > >> > > >
>> > > > > >> > >
>> > > > > >> > > It's an interesting idea. Comparing to the option of having
>> > the
>> > > > > first
>> > > > > >> > > timestamp in the message set, It's a little more space
>> > efficient
>> > > > as
>> > > > > we
>> > > > > >> > > don't have both a full timestamp in the message set _and_ a
>> > > varint
>> > > > > in
>> > > > > >> the
>> > > > > >> > > first message (which would always be 0, so we avoid the
>> extra
>> > > > byte)
>> > > > > >> and
>> > > > > >> > > also the deltas could be a little smaller in the common
>> case.
>> > > The
>> > > > > main
>> > > > > >> > > downside is that it introduces a semantics inconsistency
>> > between
>> > > > the
>> > > > > >> > first
>> > > > > >> > > message and the rest. Not ideal, but maybe we can live with
>> > > that.
>> > > > > >> > >
>> > > > > >> > > Ismael
>> > > > > >> > >
>> > > > > >> >
>> > > > > >>
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>



-- 
Gwen Shapira
Product Manager | Confluent
650.450.2760 | @gwenshap
Follow us: Twitter | blog

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
A couple follow-ups from Ismael's previous comments.

1. We have removed the individual message CRC field from the message
format. Because this field can already change on the broker in some
situations, we feel it is probably not wise to let clients depend on it,
and removing it saves some space and the redundant verification on the
broker. We have also proposed to deprecate the checksum() APIs which are
exposed in RecordMetadata and ConsumerRecord.
2. We changed the message timestamp field to be variable length. The values
are computed relative to the timestamp of the first message in the set. We
considered my previous suggestion to make the first message timestamp
absolute with later messages relative to their previous timestamp, but
ultimately felt it was simpler to stick with the same pattern that is used
for relative offsets. To enable this, we added an initial timestamp field
to the message set.

Thanks,
Jason

On Mon, Feb 6, 2017 at 6:01 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hello,
>
> I have gone ahead and updated the KIP wiki with a summary of the changes to
> the RPC protocol. The KIP wiki should now have _all_ the public facing
> changes being proposed.
>
> The proposed changes were always in the Google doc, and now we are simply
> making good on our promise to copy them over to the wiki since the design
> is almost finalized.
>
> Thanks,
> Apurva
>
> On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Tom,
> >
> > Re; complexity. This is always a tradeoff with new features. The changes
> > we've made during the design and review process have greatly simplified
> the
> > implementation for clients, and especially for the consumer, but there is
> > nontrivial work needed here to support transactions on the producer. I'm
> > not sure how it could be otherwise and we've spent a ton of time thinking
> > about this. It's also worth mentioning that there's a relatively low bar
> to
> > support the idempotent producer while keeping the client thin (it
> requires
> > support for one new request type and some sequence bookkeeping).
> >
> > Ultimately, we have to decide whether the improved semantics are worth
> the
> > cost of the complexity. In my opinion, they are. The benefit of having
> > exactly-once processing in streaming applications is huge. And to manage
> > the complexity, we've intentionally used patterns that were already
> > familiar in the codebase (e.g. our approach to maintaining transaction
> > state through a coordinator is copied from how consumer offsets are
> > managed). We've moved as much work from the clients to the broker as
> > possible, and we have resisted at every turn complicating the client APIs
> > even where it may have simplified some internals.
> >
> > -Jason
> >
> > On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Hi Tom,
> > >
> > > I updated the KIP with a note our plans for performance testing:
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
> > > ExactlyOnceDeliveryandTransactionalMessaging-Performance
> > >
> > > Thanks for pointing that out.
> > >
> > > Regards,
> > > Apurva
> > >
> > >
> > >
> > >
> > >
> > > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com>
> > wrote:
> > >
> > > > I think the updated wiki page makes sense with respect to ACLs, there
> > > seems
> > > > to be little potential for abuse there (other than the noted and
> known
> > > > issues).
> > > >
> > > > I am going to note that this is a major complexity increase for
> Kafka,
> > > and
> > > > that I'm concerned about performance impact (the JVM is quite…
> pedantic
> > > > about method size, for example, and even adding conditionals to
> larger
> > > > methods could impact this). The KIP doesn't note plans for
> performance
> > > > testing.
> > > >
> > > > I'm also concerned about the impact on non-JVM client libraries -
> > > writing a
> > > > client for Kafka is already a very complicated undertaking, and this
> > adds
> > > > to that complexity significantly.
> > > >
> > > > However, the approach seems ok enough. It does also violate the
> "Kafka
> > > has
> > > > dumb brokers and smart clients" (which I realize is in direct
> > > contradiction
> > > > of the previous statement about client implementation being hard).
> I'd
> > > love
> > > > to see some discussion in either the doc or the wiki as to why much
> of
> > > this
> > > > transactional work isn't a client-side part of Kafka Streams.
> > > >
> > > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > One additional note on the authorization. The WriteTxnMarker API is
> > > > > restricted to inter-broker usage, so it requires Cluster
> > authorization
> > > > > (just like other inter-broker APIs). I've updated the document and
> > wiki
> > > > to
> > > > > reflect this.
> > > > >
> > > > > Also, I have renamed GroupCoordinatorRequest to
> > FindCoordinatorRequest
> > > > > since there is no group for transactional producers. Let me know if
> > > there
> > > > > are any concerns.
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <
> jason@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hi Tom,
> > > > > >
> > > > > > I said this in the voting thread, but can the authors include a
> > > section
> > > > > >> about new ACLs if there are going to be ACLs for
> TransactionalId.
> > > It's
> > > > > >> mentioned in the google doc, but I think new ACLs should be in a
> > KIP
> > > > > >> directly.
> > > > > >
> > > > > >
> > > > > > We've updated the wiki. Can you take a look and let us know if
> you
> > > have
> > > > > > additional concerns?
> > > > > >
> > > > > > Thanks,
> > > > > > Jason
> > > > > >
> > > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> > > > rajinisivaram@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > >> Hi Jason,
> > > > > >>
> > > > > >> Thank you for the responses. Agree that authorizing
> > > transactional.id
> > > > in
> > > > > >> the
> > > > > >> producer requests will be good enough for version 1. And making
> it
> > > > > tighter
> > > > > >> in future based on delegation tokens sounds good too.
> > > > > >>
> > > > > >> Regards,
> > > > > >>
> > > > > >> Rajini
> > > > > >>
> > > > > >>
> > > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
> > jason@confluent.io
> > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Hey Rajini,
> > > > > >> >
> > > > > >> > Thanks for the questions. Responses below:
> > > > > >> >
> > > > > >> >
> > > > > >> > > 1. Will the transaction coordinator check topic ACLs based
> on
> > > the
> > > > > >> > > requesting client's credentials? Access to transaction logs,
> > > > topics
> > > > > >> being
> > > > > >> > > added for transaction etc?
> > > > > >> >
> > > > > >> >
> > > > > >> > Good question. I think it makes sense to check topic Write
> > > > permission
> > > > > >> when
> > > > > >> > adding partitions to the transaction. I'll add this to the
> > > document.
> > > > > >> > Perhaps authorization to the transaction log itself, however,
> > can
> > > be
> > > > > >> > assumed from having access to the ProducerTransactionalId
> > > resource?
> > > > > This
> > > > > >> > would be similar to how access to __consumer_offsets is
> assumed
> > if
> > > > the
> > > > > >> > client has access to the Group resource.
> > > > > >> >
> > > > > >> > 2. If I create a transactional produce request (by hand, not
> > using
> > > > the
> > > > > >> > > producer API) with a random PID (random, hence unlikely to
> be
> > in
> > > > > use),
> > > > > >> > will
> > > > > >> > > the broker append a transactional message to the logs,
> > > preventing
> > > > > LSO
> > > > > >> > from
> > > > > >> > > moving forward? What validation will broker do for PIDs?
> > > > > >> >
> > > > > >> >
> > > > > >> > Yes, that is correct. Validation of the TransactionalId to PID
> > > > binding
> > > > > >> is a
> > > > > >> > known gap in the current proposal, and is discussed in the
> > design
> > > > > >> document.
> > > > > >> > Now that I'm thinking about it a bit more, I think there is a
> > good
> > > > > case
> > > > > >> for
> > > > > >> > including the TransactionalId in the ProduceRequest (I think
> Jun
> > > > > >> suggested
> > > > > >> > this previously). Verifying it does not ensure that the
> included
> > > PID
> > > > > is
> > > > > >> > correct, but it does ensure that the client is authorized to
> use
> > > > > >> > transactions. If the client wanted to do an "endless
> transaction
> > > > > >> attack,"
> > > > > >> > having Write access to the topic and an authorized
> > transactionalID
> > > > is
> > > > > >> all
> > > > > >> > they would need anyway even if we could authorize the PID
> > itself.
> > > > This
> > > > > >> > seems like a worthwhile improvement.
> > > > > >> >
> > > > > >> > For future work, my half-baked idea to authorize the PID
> binding
> > > is
> > > > to
> > > > > >> > leverage the delegation work in KIP-48. When the PID is
> > generated,
> > > > we
> > > > > >> can
> > > > > >> > give the producer a token which is then used in produce
> requests
> > > > (say
> > > > > an
> > > > > >> > hmac covering the TransactionalId and PID).
> > > > > >> >
> > > > > >> >
> > > > > >> > > 3. Will every broker check that a client sending
> transactional
> > > > > produce
> > > > > >> > > requests at least has write access to transaction log topic
> > > since
> > > > it
> > > > > >> is
> > > > > >> > not
> > > > > >> > > validating transactional.id (for every produce request)?
> > > > > >> >
> > > > > >> >  4. I understand that brokers cannot authorize the
> transactional
> > > id
> > > > > for
> > > > > >> > each
> > > > > >> > > produce request since requests contain only the PID. But
> since
> > > > there
> > > > > >> is a
> > > > > >> > > one-to-one mapping between PID and transactional.id, and a
> > > > > >> connection is
> > > > > >> > > never expected to change its transactional.id, perhaps it
> is
> > > > > >> feasible to
> > > > > >> > > add authorization and cache the results in the Session?
> > Perhaps
> > > > not
> > > > > >> for
> > > > > >> > > version 1, but feels like it will be good to close the
> > security
> > > > gap
> > > > > >> here.
> > > > > >> > > Obviously it would be simpler if transactional.id was in
> the
> > > > > produce
> > > > > >> > > request if the overhead was acceptable.
> > > > > >> >
> > > > > >> >
> > > > > >> > I think my response above addresses both of these. We should
> > > include
> > > > > the
> > > > > >> > TransactionalId in the ProduceRequest. Of course it need not
> be
> > > > > >> included in
> > > > > >> > the message format, so I'm not too concerned about the
> > additional
> > > > > >> overhead
> > > > > >> > it adds.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Jason
> > > > > >> >
> > > > > >> >
> > > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <
> ismael@juma.me.uk>
> > > > > wrote:
> > > > > >> >
> > > > > >> > > Comments inline.
> > > > > >> > >
> > > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> > > > jason@confluent.io
> > > > > >
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Took me a while to remember why we didn't do this. The
> > > timestamp
> > > > > >> that
> > > > > >> > is
> > > > > >> > > > included at the message set level is the max timestamp of
> > all
> > > > > >> messages
> > > > > >> > in
> > > > > >> > > > the message set as is the case in the current message
> format
> > > (I
> > > > > will
> > > > > >> > > update
> > > > > >> > > > the document to make this explicit). We could make the
> > message
> > > > > >> > timestamps
> > > > > >> > > > relative to the max timestamp, but that makes
> serialization
> > a
> > > > bit
> > > > > >> > awkward
> > > > > >> > > > since the timestamps are not assumed to be increasing
> > > > sequentially
> > > > > >> or
> > > > > >> > > > monotonically. Once the messages in the message set had
> been
> > > > > >> > determined,
> > > > > >> > > we
> > > > > >> > > > would need to go back and adjust the relative timestamps.
> > > > > >> > > >
> > > > > >> > >
> > > > > >> > > Yes, I thought this would be a bit tricky and hence why I
> > > > mentioned
> > > > > >> the
> > > > > >> > > option of adding a new field at the message set level for
> the
> > > > first
> > > > > >> > > timestamp even though that's not ideal either.
> > > > > >> > >
> > > > > >> > > Here's one idea. We let the timestamps in the messages be
> > > varints,
> > > > > >> but we
> > > > > >> > > > make their values be relative to the timestamp of the
> > previous
> > > > > >> message,
> > > > > >> > > > with the timestamp of the first message being absolute.
> For
> > > > > >> example, if
> > > > > >> > > we
> > > > > >> > > > had timestamps 500, 501, 499, then we would write 500 for
> > the
> > > > > first
> > > > > >> > > > message, 1 for the next, and -2 for the final message.
> Would
> > > > that
> > > > > >> work?
> > > > > >> > > Let
> > > > > >> > > > me think a bit about it and see if there are any problems.
> > > > > >> > > >
> > > > > >> > >
> > > > > >> > > It's an interesting idea. Comparing to the option of having
> > the
> > > > > first
> > > > > >> > > timestamp in the message set, It's a little more space
> > efficient
> > > > as
> > > > > we
> > > > > >> > > don't have both a full timestamp in the message set _and_ a
> > > varint
> > > > > in
> > > > > >> the
> > > > > >> > > first message (which would always be 0, so we avoid the
> extra
> > > > byte)
> > > > > >> and
> > > > > >> > > also the deltas could be a little smaller in the common
> case.
> > > The
> > > > > main
> > > > > >> > > downside is that it introduces a semantics inconsistency
> > between
> > > > the
> > > > > >> > first
> > > > > >> > > message and the rest. Not ideal, but maybe we can live with
> > > that.
> > > > > >> > >
> > > > > >> > > Ismael
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hello,

I have gone ahead and updated the KIP wiki with a summary of the changes to
the RPC protocol. The KIP wiki should now have _all_ the public facing
changes being proposed.

The proposed changes were always in the Google doc, and now we are simply
making good on our promise to copy them over to the wiki since the design
is almost finalized.

Thanks,
Apurva

On Mon, Feb 6, 2017 at 4:02 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Tom,
>
> Re; complexity. This is always a tradeoff with new features. The changes
> we've made during the design and review process have greatly simplified the
> implementation for clients, and especially for the consumer, but there is
> nontrivial work needed here to support transactions on the producer. I'm
> not sure how it could be otherwise and we've spent a ton of time thinking
> about this. It's also worth mentioning that there's a relatively low bar to
> support the idempotent producer while keeping the client thin (it requires
> support for one new request type and some sequence bookkeeping).
>
> Ultimately, we have to decide whether the improved semantics are worth the
> cost of the complexity. In my opinion, they are. The benefit of having
> exactly-once processing in streaming applications is huge. And to manage
> the complexity, we've intentionally used patterns that were already
> familiar in the codebase (e.g. our approach to maintaining transaction
> state through a coordinator is copied from how consumer offsets are
> managed). We've moved as much work from the clients to the broker as
> possible, and we have resisted at every turn complicating the client APIs
> even where it may have simplified some internals.
>
> -Jason
>
> On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Tom,
> >
> > I updated the KIP with a note our plans for performance testing:
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
> > ExactlyOnceDeliveryandTransactionalMessaging-Performance
> >
> > Thanks for pointing that out.
> >
> > Regards,
> > Apurva
> >
> >
> >
> >
> >
> > On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com>
> wrote:
> >
> > > I think the updated wiki page makes sense with respect to ACLs, there
> > seems
> > > to be little potential for abuse there (other than the noted and known
> > > issues).
> > >
> > > I am going to note that this is a major complexity increase for Kafka,
> > and
> > > that I'm concerned about performance impact (the JVM is quite… pedantic
> > > about method size, for example, and even adding conditionals to larger
> > > methods could impact this). The KIP doesn't note plans for performance
> > > testing.
> > >
> > > I'm also concerned about the impact on non-JVM client libraries -
> > writing a
> > > client for Kafka is already a very complicated undertaking, and this
> adds
> > > to that complexity significantly.
> > >
> > > However, the approach seems ok enough. It does also violate the "Kafka
> > has
> > > dumb brokers and smart clients" (which I realize is in direct
> > contradiction
> > > of the previous statement about client implementation being hard). I'd
> > love
> > > to see some discussion in either the doc or the wiki as to why much of
> > this
> > > transactional work isn't a client-side part of Kafka Streams.
> > >
> > > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > One additional note on the authorization. The WriteTxnMarker API is
> > > > restricted to inter-broker usage, so it requires Cluster
> authorization
> > > > (just like other inter-broker APIs). I've updated the document and
> wiki
> > > to
> > > > reflect this.
> > > >
> > > > Also, I have renamed GroupCoordinatorRequest to
> FindCoordinatorRequest
> > > > since there is no group for transactional producers. Let me know if
> > there
> > > > are any concerns.
> > > >
> > > > -Jason
> > > >
> > > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Tom,
> > > > >
> > > > > I said this in the voting thread, but can the authors include a
> > section
> > > > >> about new ACLs if there are going to be ACLs for TransactionalId.
> > It's
> > > > >> mentioned in the google doc, but I think new ACLs should be in a
> KIP
> > > > >> directly.
> > > > >
> > > > >
> > > > > We've updated the wiki. Can you take a look and let us know if you
> > have
> > > > > additional concerns?
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> > > rajinisivaram@gmail.com>
> > > > > wrote:
> > > > >
> > > > >> Hi Jason,
> > > > >>
> > > > >> Thank you for the responses. Agree that authorizing
> > transactional.id
> > > in
> > > > >> the
> > > > >> producer requests will be good enough for version 1. And making it
> > > > tighter
> > > > >> in future based on delegation tokens sounds good too.
> > > > >>
> > > > >> Regards,
> > > > >>
> > > > >> Rajini
> > > > >>
> > > > >>
> > > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > > >> wrote:
> > > > >>
> > > > >> > Hey Rajini,
> > > > >> >
> > > > >> > Thanks for the questions. Responses below:
> > > > >> >
> > > > >> >
> > > > >> > > 1. Will the transaction coordinator check topic ACLs based on
> > the
> > > > >> > > requesting client's credentials? Access to transaction logs,
> > > topics
> > > > >> being
> > > > >> > > added for transaction etc?
> > > > >> >
> > > > >> >
> > > > >> > Good question. I think it makes sense to check topic Write
> > > permission
> > > > >> when
> > > > >> > adding partitions to the transaction. I'll add this to the
> > document.
> > > > >> > Perhaps authorization to the transaction log itself, however,
> can
> > be
> > > > >> > assumed from having access to the ProducerTransactionalId
> > resource?
> > > > This
> > > > >> > would be similar to how access to __consumer_offsets is assumed
> if
> > > the
> > > > >> > client has access to the Group resource.
> > > > >> >
> > > > >> > 2. If I create a transactional produce request (by hand, not
> using
> > > the
> > > > >> > > producer API) with a random PID (random, hence unlikely to be
> in
> > > > use),
> > > > >> > will
> > > > >> > > the broker append a transactional message to the logs,
> > preventing
> > > > LSO
> > > > >> > from
> > > > >> > > moving forward? What validation will broker do for PIDs?
> > > > >> >
> > > > >> >
> > > > >> > Yes, that is correct. Validation of the TransactionalId to PID
> > > binding
> > > > >> is a
> > > > >> > known gap in the current proposal, and is discussed in the
> design
> > > > >> document.
> > > > >> > Now that I'm thinking about it a bit more, I think there is a
> good
> > > > case
> > > > >> for
> > > > >> > including the TransactionalId in the ProduceRequest (I think Jun
> > > > >> suggested
> > > > >> > this previously). Verifying it does not ensure that the included
> > PID
> > > > is
> > > > >> > correct, but it does ensure that the client is authorized to use
> > > > >> > transactions. If the client wanted to do an "endless transaction
> > > > >> attack,"
> > > > >> > having Write access to the topic and an authorized
> transactionalID
> > > is
> > > > >> all
> > > > >> > they would need anyway even if we could authorize the PID
> itself.
> > > This
> > > > >> > seems like a worthwhile improvement.
> > > > >> >
> > > > >> > For future work, my half-baked idea to authorize the PID binding
> > is
> > > to
> > > > >> > leverage the delegation work in KIP-48. When the PID is
> generated,
> > > we
> > > > >> can
> > > > >> > give the producer a token which is then used in produce requests
> > > (say
> > > > an
> > > > >> > hmac covering the TransactionalId and PID).
> > > > >> >
> > > > >> >
> > > > >> > > 3. Will every broker check that a client sending transactional
> > > > produce
> > > > >> > > requests at least has write access to transaction log topic
> > since
> > > it
> > > > >> is
> > > > >> > not
> > > > >> > > validating transactional.id (for every produce request)?
> > > > >> >
> > > > >> >  4. I understand that brokers cannot authorize the transactional
> > id
> > > > for
> > > > >> > each
> > > > >> > > produce request since requests contain only the PID. But since
> > > there
> > > > >> is a
> > > > >> > > one-to-one mapping between PID and transactional.id, and a
> > > > >> connection is
> > > > >> > > never expected to change its transactional.id, perhaps it is
> > > > >> feasible to
> > > > >> > > add authorization and cache the results in the Session?
> Perhaps
> > > not
> > > > >> for
> > > > >> > > version 1, but feels like it will be good to close the
> security
> > > gap
> > > > >> here.
> > > > >> > > Obviously it would be simpler if transactional.id was in the
> > > > produce
> > > > >> > > request if the overhead was acceptable.
> > > > >> >
> > > > >> >
> > > > >> > I think my response above addresses both of these. We should
> > include
> > > > the
> > > > >> > TransactionalId in the ProduceRequest. Of course it need not be
> > > > >> included in
> > > > >> > the message format, so I'm not too concerned about the
> additional
> > > > >> overhead
> > > > >> > it adds.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Jason
> > > > >> >
> > > > >> >
> > > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk>
> > > > wrote:
> > > > >> >
> > > > >> > > Comments inline.
> > > > >> > >
> > > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> > > jason@confluent.io
> > > > >
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Took me a while to remember why we didn't do this. The
> > timestamp
> > > > >> that
> > > > >> > is
> > > > >> > > > included at the message set level is the max timestamp of
> all
> > > > >> messages
> > > > >> > in
> > > > >> > > > the message set as is the case in the current message format
> > (I
> > > > will
> > > > >> > > update
> > > > >> > > > the document to make this explicit). We could make the
> message
> > > > >> > timestamps
> > > > >> > > > relative to the max timestamp, but that makes serialization
> a
> > > bit
> > > > >> > awkward
> > > > >> > > > since the timestamps are not assumed to be increasing
> > > sequentially
> > > > >> or
> > > > >> > > > monotonically. Once the messages in the message set had been
> > > > >> > determined,
> > > > >> > > we
> > > > >> > > > would need to go back and adjust the relative timestamps.
> > > > >> > > >
> > > > >> > >
> > > > >> > > Yes, I thought this would be a bit tricky and hence why I
> > > mentioned
> > > > >> the
> > > > >> > > option of adding a new field at the message set level for the
> > > first
> > > > >> > > timestamp even though that's not ideal either.
> > > > >> > >
> > > > >> > > Here's one idea. We let the timestamps in the messages be
> > varints,
> > > > >> but we
> > > > >> > > > make their values be relative to the timestamp of the
> previous
> > > > >> message,
> > > > >> > > > with the timestamp of the first message being absolute. For
> > > > >> example, if
> > > > >> > > we
> > > > >> > > > had timestamps 500, 501, 499, then we would write 500 for
> the
> > > > first
> > > > >> > > > message, 1 for the next, and -2 for the final message. Would
> > > that
> > > > >> work?
> > > > >> > > Let
> > > > >> > > > me think a bit about it and see if there are any problems.
> > > > >> > > >
> > > > >> > >
> > > > >> > > It's an interesting idea. Comparing to the option of having
> the
> > > > first
> > > > >> > > timestamp in the message set, It's a little more space
> efficient
> > > as
> > > > we
> > > > >> > > don't have both a full timestamp in the message set _and_ a
> > varint
> > > > in
> > > > >> the
> > > > >> > > first message (which would always be 0, so we avoid the extra
> > > byte)
> > > > >> and
> > > > >> > > also the deltas could be a little smaller in the common case.
> > The
> > > > main
> > > > >> > > downside is that it introduces a semantics inconsistency
> between
> > > the
> > > > >> > first
> > > > >> > > message and the rest. Not ideal, but maybe we can live with
> > that.
> > > > >> > >
> > > > >> > > Ismael
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Tom,

Re; complexity. This is always a tradeoff with new features. The changes
we've made during the design and review process have greatly simplified the
implementation for clients, and especially for the consumer, but there is
nontrivial work needed here to support transactions on the producer. I'm
not sure how it could be otherwise and we've spent a ton of time thinking
about this. It's also worth mentioning that there's a relatively low bar to
support the idempotent producer while keeping the client thin (it requires
support for one new request type and some sequence bookkeeping).

Ultimately, we have to decide whether the improved semantics are worth the
cost of the complexity. In my opinion, they are. The benefit of having
exactly-once processing in streaming applications is huge. And to manage
the complexity, we've intentionally used patterns that were already
familiar in the codebase (e.g. our approach to maintaining transaction
state through a coordinator is copied from how consumer offsets are
managed). We've moved as much work from the clients to the broker as
possible, and we have resisted at every turn complicating the client APIs
even where it may have simplified some internals.

-Jason

On Mon, Feb 6, 2017 at 2:55 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Tom,
>
> I updated the KIP with a note our plans for performance testing:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-
> ExactlyOnceDeliveryandTransactionalMessaging-Performance
>
> Thanks for pointing that out.
>
> Regards,
> Apurva
>
>
>
>
>
> On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com> wrote:
>
> > I think the updated wiki page makes sense with respect to ACLs, there
> seems
> > to be little potential for abuse there (other than the noted and known
> > issues).
> >
> > I am going to note that this is a major complexity increase for Kafka,
> and
> > that I'm concerned about performance impact (the JVM is quite… pedantic
> > about method size, for example, and even adding conditionals to larger
> > methods could impact this). The KIP doesn't note plans for performance
> > testing.
> >
> > I'm also concerned about the impact on non-JVM client libraries -
> writing a
> > client for Kafka is already a very complicated undertaking, and this adds
> > to that complexity significantly.
> >
> > However, the approach seems ok enough. It does also violate the "Kafka
> has
> > dumb brokers and smart clients" (which I realize is in direct
> contradiction
> > of the previous statement about client implementation being hard). I'd
> love
> > to see some discussion in either the doc or the wiki as to why much of
> this
> > transactional work isn't a client-side part of Kafka Streams.
> >
> > On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > One additional note on the authorization. The WriteTxnMarker API is
> > > restricted to inter-broker usage, so it requires Cluster authorization
> > > (just like other inter-broker APIs). I've updated the document and wiki
> > to
> > > reflect this.
> > >
> > > Also, I have renamed GroupCoordinatorRequest to FindCoordinatorRequest
> > > since there is no group for transactional producers. Let me know if
> there
> > > are any concerns.
> > >
> > > -Jason
> > >
> > > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hi Tom,
> > > >
> > > > I said this in the voting thread, but can the authors include a
> section
> > > >> about new ACLs if there are going to be ACLs for TransactionalId.
> It's
> > > >> mentioned in the google doc, but I think new ACLs should be in a KIP
> > > >> directly.
> > > >
> > > >
> > > > We've updated the wiki. Can you take a look and let us know if you
> have
> > > > additional concerns?
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> > rajinisivaram@gmail.com>
> > > > wrote:
> > > >
> > > >> Hi Jason,
> > > >>
> > > >> Thank you for the responses. Agree that authorizing
> transactional.id
> > in
> > > >> the
> > > >> producer requests will be good enough for version 1. And making it
> > > tighter
> > > >> in future based on delegation tokens sounds good too.
> > > >>
> > > >> Regards,
> > > >>
> > > >> Rajini
> > > >>
> > > >>
> > > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <jason@confluent.io
> >
> > > >> wrote:
> > > >>
> > > >> > Hey Rajini,
> > > >> >
> > > >> > Thanks for the questions. Responses below:
> > > >> >
> > > >> >
> > > >> > > 1. Will the transaction coordinator check topic ACLs based on
> the
> > > >> > > requesting client's credentials? Access to transaction logs,
> > topics
> > > >> being
> > > >> > > added for transaction etc?
> > > >> >
> > > >> >
> > > >> > Good question. I think it makes sense to check topic Write
> > permission
> > > >> when
> > > >> > adding partitions to the transaction. I'll add this to the
> document.
> > > >> > Perhaps authorization to the transaction log itself, however, can
> be
> > > >> > assumed from having access to the ProducerTransactionalId
> resource?
> > > This
> > > >> > would be similar to how access to __consumer_offsets is assumed if
> > the
> > > >> > client has access to the Group resource.
> > > >> >
> > > >> > 2. If I create a transactional produce request (by hand, not using
> > the
> > > >> > > producer API) with a random PID (random, hence unlikely to be in
> > > use),
> > > >> > will
> > > >> > > the broker append a transactional message to the logs,
> preventing
> > > LSO
> > > >> > from
> > > >> > > moving forward? What validation will broker do for PIDs?
> > > >> >
> > > >> >
> > > >> > Yes, that is correct. Validation of the TransactionalId to PID
> > binding
> > > >> is a
> > > >> > known gap in the current proposal, and is discussed in the design
> > > >> document.
> > > >> > Now that I'm thinking about it a bit more, I think there is a good
> > > case
> > > >> for
> > > >> > including the TransactionalId in the ProduceRequest (I think Jun
> > > >> suggested
> > > >> > this previously). Verifying it does not ensure that the included
> PID
> > > is
> > > >> > correct, but it does ensure that the client is authorized to use
> > > >> > transactions. If the client wanted to do an "endless transaction
> > > >> attack,"
> > > >> > having Write access to the topic and an authorized transactionalID
> > is
> > > >> all
> > > >> > they would need anyway even if we could authorize the PID itself.
> > This
> > > >> > seems like a worthwhile improvement.
> > > >> >
> > > >> > For future work, my half-baked idea to authorize the PID binding
> is
> > to
> > > >> > leverage the delegation work in KIP-48. When the PID is generated,
> > we
> > > >> can
> > > >> > give the producer a token which is then used in produce requests
> > (say
> > > an
> > > >> > hmac covering the TransactionalId and PID).
> > > >> >
> > > >> >
> > > >> > > 3. Will every broker check that a client sending transactional
> > > produce
> > > >> > > requests at least has write access to transaction log topic
> since
> > it
> > > >> is
> > > >> > not
> > > >> > > validating transactional.id (for every produce request)?
> > > >> >
> > > >> >  4. I understand that brokers cannot authorize the transactional
> id
> > > for
> > > >> > each
> > > >> > > produce request since requests contain only the PID. But since
> > there
> > > >> is a
> > > >> > > one-to-one mapping between PID and transactional.id, and a
> > > >> connection is
> > > >> > > never expected to change its transactional.id, perhaps it is
> > > >> feasible to
> > > >> > > add authorization and cache the results in the Session? Perhaps
> > not
> > > >> for
> > > >> > > version 1, but feels like it will be good to close the security
> > gap
> > > >> here.
> > > >> > > Obviously it would be simpler if transactional.id was in the
> > > produce
> > > >> > > request if the overhead was acceptable.
> > > >> >
> > > >> >
> > > >> > I think my response above addresses both of these. We should
> include
> > > the
> > > >> > TransactionalId in the ProduceRequest. Of course it need not be
> > > >> included in
> > > >> > the message format, so I'm not too concerned about the additional
> > > >> overhead
> > > >> > it adds.
> > > >> >
> > > >> > Thanks,
> > > >> > Jason
> > > >> >
> > > >> >
> > > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk>
> > > wrote:
> > > >> >
> > > >> > > Comments inline.
> > > >> > >
> > > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> > jason@confluent.io
> > > >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Took me a while to remember why we didn't do this. The
> timestamp
> > > >> that
> > > >> > is
> > > >> > > > included at the message set level is the max timestamp of all
> > > >> messages
> > > >> > in
> > > >> > > > the message set as is the case in the current message format
> (I
> > > will
> > > >> > > update
> > > >> > > > the document to make this explicit). We could make the message
> > > >> > timestamps
> > > >> > > > relative to the max timestamp, but that makes serialization a
> > bit
> > > >> > awkward
> > > >> > > > since the timestamps are not assumed to be increasing
> > sequentially
> > > >> or
> > > >> > > > monotonically. Once the messages in the message set had been
> > > >> > determined,
> > > >> > > we
> > > >> > > > would need to go back and adjust the relative timestamps.
> > > >> > > >
> > > >> > >
> > > >> > > Yes, I thought this would be a bit tricky and hence why I
> > mentioned
> > > >> the
> > > >> > > option of adding a new field at the message set level for the
> > first
> > > >> > > timestamp even though that's not ideal either.
> > > >> > >
> > > >> > > Here's one idea. We let the timestamps in the messages be
> varints,
> > > >> but we
> > > >> > > > make their values be relative to the timestamp of the previous
> > > >> message,
> > > >> > > > with the timestamp of the first message being absolute. For
> > > >> example, if
> > > >> > > we
> > > >> > > > had timestamps 500, 501, 499, then we would write 500 for the
> > > first
> > > >> > > > message, 1 for the next, and -2 for the final message. Would
> > that
> > > >> work?
> > > >> > > Let
> > > >> > > > me think a bit about it and see if there are any problems.
> > > >> > > >
> > > >> > >
> > > >> > > It's an interesting idea. Comparing to the option of having the
> > > first
> > > >> > > timestamp in the message set, It's a little more space efficient
> > as
> > > we
> > > >> > > don't have both a full timestamp in the message set _and_ a
> varint
> > > in
> > > >> the
> > > >> > > first message (which would always be 0, so we avoid the extra
> > byte)
> > > >> and
> > > >> > > also the deltas could be a little smaller in the common case.
> The
> > > main
> > > >> > > downside is that it introduces a semantics inconsistency between
> > the
> > > >> > first
> > > >> > > message and the rest. Not ideal, but maybe we can live with
> that.
> > > >> > >
> > > >> > > Ismael
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Tom,

I updated the KIP with a note our plans for performance testing:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-Performance

Thanks for pointing that out.

Regards,
Apurva





On Mon, Feb 6, 2017 at 7:40 AM, Tom Crayford <tc...@heroku.com> wrote:

> I think the updated wiki page makes sense with respect to ACLs, there seems
> to be little potential for abuse there (other than the noted and known
> issues).
>
> I am going to note that this is a major complexity increase for Kafka, and
> that I'm concerned about performance impact (the JVM is quite… pedantic
> about method size, for example, and even adding conditionals to larger
> methods could impact this). The KIP doesn't note plans for performance
> testing.
>
> I'm also concerned about the impact on non-JVM client libraries - writing a
> client for Kafka is already a very complicated undertaking, and this adds
> to that complexity significantly.
>
> However, the approach seems ok enough. It does also violate the "Kafka has
> dumb brokers and smart clients" (which I realize is in direct contradiction
> of the previous statement about client implementation being hard). I'd love
> to see some discussion in either the doc or the wiki as to why much of this
> transactional work isn't a client-side part of Kafka Streams.
>
> On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > One additional note on the authorization. The WriteTxnMarker API is
> > restricted to inter-broker usage, so it requires Cluster authorization
> > (just like other inter-broker APIs). I've updated the document and wiki
> to
> > reflect this.
> >
> > Also, I have renamed GroupCoordinatorRequest to FindCoordinatorRequest
> > since there is no group for transactional producers. Let me know if there
> > are any concerns.
> >
> > -Jason
> >
> > On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hi Tom,
> > >
> > > I said this in the voting thread, but can the authors include a section
> > >> about new ACLs if there are going to be ACLs for TransactionalId. It's
> > >> mentioned in the google doc, but I think new ACLs should be in a KIP
> > >> directly.
> > >
> > >
> > > We've updated the wiki. Can you take a look and let us know if you have
> > > additional concerns?
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <
> rajinisivaram@gmail.com>
> > > wrote:
> > >
> > >> Hi Jason,
> > >>
> > >> Thank you for the responses. Agree that authorizing transactional.id
> in
> > >> the
> > >> producer requests will be good enough for version 1. And making it
> > tighter
> > >> in future based on delegation tokens sounds good too.
> > >>
> > >> Regards,
> > >>
> > >> Rajini
> > >>
> > >>
> > >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <ja...@confluent.io>
> > >> wrote:
> > >>
> > >> > Hey Rajini,
> > >> >
> > >> > Thanks for the questions. Responses below:
> > >> >
> > >> >
> > >> > > 1. Will the transaction coordinator check topic ACLs based on the
> > >> > > requesting client's credentials? Access to transaction logs,
> topics
> > >> being
> > >> > > added for transaction etc?
> > >> >
> > >> >
> > >> > Good question. I think it makes sense to check topic Write
> permission
> > >> when
> > >> > adding partitions to the transaction. I'll add this to the document.
> > >> > Perhaps authorization to the transaction log itself, however, can be
> > >> > assumed from having access to the ProducerTransactionalId resource?
> > This
> > >> > would be similar to how access to __consumer_offsets is assumed if
> the
> > >> > client has access to the Group resource.
> > >> >
> > >> > 2. If I create a transactional produce request (by hand, not using
> the
> > >> > > producer API) with a random PID (random, hence unlikely to be in
> > use),
> > >> > will
> > >> > > the broker append a transactional message to the logs, preventing
> > LSO
> > >> > from
> > >> > > moving forward? What validation will broker do for PIDs?
> > >> >
> > >> >
> > >> > Yes, that is correct. Validation of the TransactionalId to PID
> binding
> > >> is a
> > >> > known gap in the current proposal, and is discussed in the design
> > >> document.
> > >> > Now that I'm thinking about it a bit more, I think there is a good
> > case
> > >> for
> > >> > including the TransactionalId in the ProduceRequest (I think Jun
> > >> suggested
> > >> > this previously). Verifying it does not ensure that the included PID
> > is
> > >> > correct, but it does ensure that the client is authorized to use
> > >> > transactions. If the client wanted to do an "endless transaction
> > >> attack,"
> > >> > having Write access to the topic and an authorized transactionalID
> is
> > >> all
> > >> > they would need anyway even if we could authorize the PID itself.
> This
> > >> > seems like a worthwhile improvement.
> > >> >
> > >> > For future work, my half-baked idea to authorize the PID binding is
> to
> > >> > leverage the delegation work in KIP-48. When the PID is generated,
> we
> > >> can
> > >> > give the producer a token which is then used in produce requests
> (say
> > an
> > >> > hmac covering the TransactionalId and PID).
> > >> >
> > >> >
> > >> > > 3. Will every broker check that a client sending transactional
> > produce
> > >> > > requests at least has write access to transaction log topic since
> it
> > >> is
> > >> > not
> > >> > > validating transactional.id (for every produce request)?
> > >> >
> > >> >  4. I understand that brokers cannot authorize the transactional id
> > for
> > >> > each
> > >> > > produce request since requests contain only the PID. But since
> there
> > >> is a
> > >> > > one-to-one mapping between PID and transactional.id, and a
> > >> connection is
> > >> > > never expected to change its transactional.id, perhaps it is
> > >> feasible to
> > >> > > add authorization and cache the results in the Session? Perhaps
> not
> > >> for
> > >> > > version 1, but feels like it will be good to close the security
> gap
> > >> here.
> > >> > > Obviously it would be simpler if transactional.id was in the
> > produce
> > >> > > request if the overhead was acceptable.
> > >> >
> > >> >
> > >> > I think my response above addresses both of these. We should include
> > the
> > >> > TransactionalId in the ProduceRequest. Of course it need not be
> > >> included in
> > >> > the message format, so I'm not too concerned about the additional
> > >> overhead
> > >> > it adds.
> > >> >
> > >> > Thanks,
> > >> > Jason
> > >> >
> > >> >
> > >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk>
> > wrote:
> > >> >
> > >> > > Comments inline.
> > >> > >
> > >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > >> > > wrote:
> > >> > >
> > >> > > > Took me a while to remember why we didn't do this. The timestamp
> > >> that
> > >> > is
> > >> > > > included at the message set level is the max timestamp of all
> > >> messages
> > >> > in
> > >> > > > the message set as is the case in the current message format (I
> > will
> > >> > > update
> > >> > > > the document to make this explicit). We could make the message
> > >> > timestamps
> > >> > > > relative to the max timestamp, but that makes serialization a
> bit
> > >> > awkward
> > >> > > > since the timestamps are not assumed to be increasing
> sequentially
> > >> or
> > >> > > > monotonically. Once the messages in the message set had been
> > >> > determined,
> > >> > > we
> > >> > > > would need to go back and adjust the relative timestamps.
> > >> > > >
> > >> > >
> > >> > > Yes, I thought this would be a bit tricky and hence why I
> mentioned
> > >> the
> > >> > > option of adding a new field at the message set level for the
> first
> > >> > > timestamp even though that's not ideal either.
> > >> > >
> > >> > > Here's one idea. We let the timestamps in the messages be varints,
> > >> but we
> > >> > > > make their values be relative to the timestamp of the previous
> > >> message,
> > >> > > > with the timestamp of the first message being absolute. For
> > >> example, if
> > >> > > we
> > >> > > > had timestamps 500, 501, 499, then we would write 500 for the
> > first
> > >> > > > message, 1 for the next, and -2 for the final message. Would
> that
> > >> work?
> > >> > > Let
> > >> > > > me think a bit about it and see if there are any problems.
> > >> > > >
> > >> > >
> > >> > > It's an interesting idea. Comparing to the option of having the
> > first
> > >> > > timestamp in the message set, It's a little more space efficient
> as
> > we
> > >> > > don't have both a full timestamp in the message set _and_ a varint
> > in
> > >> the
> > >> > > first message (which would always be 0, so we avoid the extra
> byte)
> > >> and
> > >> > > also the deltas could be a little smaller in the common case. The
> > main
> > >> > > downside is that it introduces a semantics inconsistency between
> the
> > >> > first
> > >> > > message and the rest. Not ideal, but maybe we can live with that.
> > >> > >
> > >> > > Ismael
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Tom Crayford <tc...@heroku.com>.
I think the updated wiki page makes sense with respect to ACLs, there seems
to be little potential for abuse there (other than the noted and known
issues).

I am going to note that this is a major complexity increase for Kafka, and
that I'm concerned about performance impact (the JVM is quite… pedantic
about method size, for example, and even adding conditionals to larger
methods could impact this). The KIP doesn't note plans for performance
testing.

I'm also concerned about the impact on non-JVM client libraries - writing a
client for Kafka is already a very complicated undertaking, and this adds
to that complexity significantly.

However, the approach seems ok enough. It does also violate the "Kafka has
dumb brokers and smart clients" (which I realize is in direct contradiction
of the previous statement about client implementation being hard). I'd love
to see some discussion in either the doc or the wiki as to why much of this
transactional work isn't a client-side part of Kafka Streams.

On Sat, Feb 4, 2017 at 3:38 AM, Jason Gustafson <ja...@confluent.io> wrote:

> One additional note on the authorization. The WriteTxnMarker API is
> restricted to inter-broker usage, so it requires Cluster authorization
> (just like other inter-broker APIs). I've updated the document and wiki to
> reflect this.
>
> Also, I have renamed GroupCoordinatorRequest to FindCoordinatorRequest
> since there is no group for transactional producers. Let me know if there
> are any concerns.
>
> -Jason
>
> On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hi Tom,
> >
> > I said this in the voting thread, but can the authors include a section
> >> about new ACLs if there are going to be ACLs for TransactionalId. It's
> >> mentioned in the google doc, but I think new ACLs should be in a KIP
> >> directly.
> >
> >
> > We've updated the wiki. Can you take a look and let us know if you have
> > additional concerns?
> >
> > Thanks,
> > Jason
> >
> > On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <ra...@gmail.com>
> > wrote:
> >
> >> Hi Jason,
> >>
> >> Thank you for the responses. Agree that authorizing transactional.id in
> >> the
> >> producer requests will be good enough for version 1. And making it
> tighter
> >> in future based on delegation tokens sounds good too.
> >>
> >> Regards,
> >>
> >> Rajini
> >>
> >>
> >> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >>
> >> > Hey Rajini,
> >> >
> >> > Thanks for the questions. Responses below:
> >> >
> >> >
> >> > > 1. Will the transaction coordinator check topic ACLs based on the
> >> > > requesting client's credentials? Access to transaction logs, topics
> >> being
> >> > > added for transaction etc?
> >> >
> >> >
> >> > Good question. I think it makes sense to check topic Write permission
> >> when
> >> > adding partitions to the transaction. I'll add this to the document.
> >> > Perhaps authorization to the transaction log itself, however, can be
> >> > assumed from having access to the ProducerTransactionalId resource?
> This
> >> > would be similar to how access to __consumer_offsets is assumed if the
> >> > client has access to the Group resource.
> >> >
> >> > 2. If I create a transactional produce request (by hand, not using the
> >> > > producer API) with a random PID (random, hence unlikely to be in
> use),
> >> > will
> >> > > the broker append a transactional message to the logs, preventing
> LSO
> >> > from
> >> > > moving forward? What validation will broker do for PIDs?
> >> >
> >> >
> >> > Yes, that is correct. Validation of the TransactionalId to PID binding
> >> is a
> >> > known gap in the current proposal, and is discussed in the design
> >> document.
> >> > Now that I'm thinking about it a bit more, I think there is a good
> case
> >> for
> >> > including the TransactionalId in the ProduceRequest (I think Jun
> >> suggested
> >> > this previously). Verifying it does not ensure that the included PID
> is
> >> > correct, but it does ensure that the client is authorized to use
> >> > transactions. If the client wanted to do an "endless transaction
> >> attack,"
> >> > having Write access to the topic and an authorized transactionalID is
> >> all
> >> > they would need anyway even if we could authorize the PID itself. This
> >> > seems like a worthwhile improvement.
> >> >
> >> > For future work, my half-baked idea to authorize the PID binding is to
> >> > leverage the delegation work in KIP-48. When the PID is generated, we
> >> can
> >> > give the producer a token which is then used in produce requests (say
> an
> >> > hmac covering the TransactionalId and PID).
> >> >
> >> >
> >> > > 3. Will every broker check that a client sending transactional
> produce
> >> > > requests at least has write access to transaction log topic since it
> >> is
> >> > not
> >> > > validating transactional.id (for every produce request)?
> >> >
> >> >  4. I understand that brokers cannot authorize the transactional id
> for
> >> > each
> >> > > produce request since requests contain only the PID. But since there
> >> is a
> >> > > one-to-one mapping between PID and transactional.id, and a
> >> connection is
> >> > > never expected to change its transactional.id, perhaps it is
> >> feasible to
> >> > > add authorization and cache the results in the Session? Perhaps not
> >> for
> >> > > version 1, but feels like it will be good to close the security gap
> >> here.
> >> > > Obviously it would be simpler if transactional.id was in the
> produce
> >> > > request if the overhead was acceptable.
> >> >
> >> >
> >> > I think my response above addresses both of these. We should include
> the
> >> > TransactionalId in the ProduceRequest. Of course it need not be
> >> included in
> >> > the message format, so I'm not too concerned about the additional
> >> overhead
> >> > it adds.
> >> >
> >> > Thanks,
> >> > Jason
> >> >
> >> >
> >> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk>
> wrote:
> >> >
> >> > > Comments inline.
> >> > >
> >> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <jason@confluent.io
> >
> >> > > wrote:
> >> > >
> >> > > > Took me a while to remember why we didn't do this. The timestamp
> >> that
> >> > is
> >> > > > included at the message set level is the max timestamp of all
> >> messages
> >> > in
> >> > > > the message set as is the case in the current message format (I
> will
> >> > > update
> >> > > > the document to make this explicit). We could make the message
> >> > timestamps
> >> > > > relative to the max timestamp, but that makes serialization a bit
> >> > awkward
> >> > > > since the timestamps are not assumed to be increasing sequentially
> >> or
> >> > > > monotonically. Once the messages in the message set had been
> >> > determined,
> >> > > we
> >> > > > would need to go back and adjust the relative timestamps.
> >> > > >
> >> > >
> >> > > Yes, I thought this would be a bit tricky and hence why I mentioned
> >> the
> >> > > option of adding a new field at the message set level for the first
> >> > > timestamp even though that's not ideal either.
> >> > >
> >> > > Here's one idea. We let the timestamps in the messages be varints,
> >> but we
> >> > > > make their values be relative to the timestamp of the previous
> >> message,
> >> > > > with the timestamp of the first message being absolute. For
> >> example, if
> >> > > we
> >> > > > had timestamps 500, 501, 499, then we would write 500 for the
> first
> >> > > > message, 1 for the next, and -2 for the final message. Would that
> >> work?
> >> > > Let
> >> > > > me think a bit about it and see if there are any problems.
> >> > > >
> >> > >
> >> > > It's an interesting idea. Comparing to the option of having the
> first
> >> > > timestamp in the message set, It's a little more space efficient as
> we
> >> > > don't have both a full timestamp in the message set _and_ a varint
> in
> >> the
> >> > > first message (which would always be 0, so we avoid the extra byte)
> >> and
> >> > > also the deltas could be a little smaller in the common case. The
> main
> >> > > downside is that it introduces a semantics inconsistency between the
> >> > first
> >> > > message and the rest. Not ideal, but maybe we can live with that.
> >> > >
> >> > > Ismael
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
One additional note on the authorization. The WriteTxnMarker API is
restricted to inter-broker usage, so it requires Cluster authorization
(just like other inter-broker APIs). I've updated the document and wiki to
reflect this.

Also, I have renamed GroupCoordinatorRequest to FindCoordinatorRequest
since there is no group for transactional producers. Let me know if there
are any concerns.

-Jason

On Fri, Feb 3, 2017 at 2:35 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hi Tom,
>
> I said this in the voting thread, but can the authors include a section
>> about new ACLs if there are going to be ACLs for TransactionalId. It's
>> mentioned in the google doc, but I think new ACLs should be in a KIP
>> directly.
>
>
> We've updated the wiki. Can you take a look and let us know if you have
> additional concerns?
>
> Thanks,
> Jason
>
> On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <ra...@gmail.com>
> wrote:
>
>> Hi Jason,
>>
>> Thank you for the responses. Agree that authorizing transactional.id in
>> the
>> producer requests will be good enough for version 1. And making it tighter
>> in future based on delegation tokens sounds good too.
>>
>> Regards,
>>
>> Rajini
>>
>>
>> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>>
>> > Hey Rajini,
>> >
>> > Thanks for the questions. Responses below:
>> >
>> >
>> > > 1. Will the transaction coordinator check topic ACLs based on the
>> > > requesting client's credentials? Access to transaction logs, topics
>> being
>> > > added for transaction etc?
>> >
>> >
>> > Good question. I think it makes sense to check topic Write permission
>> when
>> > adding partitions to the transaction. I'll add this to the document.
>> > Perhaps authorization to the transaction log itself, however, can be
>> > assumed from having access to the ProducerTransactionalId resource? This
>> > would be similar to how access to __consumer_offsets is assumed if the
>> > client has access to the Group resource.
>> >
>> > 2. If I create a transactional produce request (by hand, not using the
>> > > producer API) with a random PID (random, hence unlikely to be in use),
>> > will
>> > > the broker append a transactional message to the logs, preventing LSO
>> > from
>> > > moving forward? What validation will broker do for PIDs?
>> >
>> >
>> > Yes, that is correct. Validation of the TransactionalId to PID binding
>> is a
>> > known gap in the current proposal, and is discussed in the design
>> document.
>> > Now that I'm thinking about it a bit more, I think there is a good case
>> for
>> > including the TransactionalId in the ProduceRequest (I think Jun
>> suggested
>> > this previously). Verifying it does not ensure that the included PID is
>> > correct, but it does ensure that the client is authorized to use
>> > transactions. If the client wanted to do an "endless transaction
>> attack,"
>> > having Write access to the topic and an authorized transactionalID is
>> all
>> > they would need anyway even if we could authorize the PID itself. This
>> > seems like a worthwhile improvement.
>> >
>> > For future work, my half-baked idea to authorize the PID binding is to
>> > leverage the delegation work in KIP-48. When the PID is generated, we
>> can
>> > give the producer a token which is then used in produce requests (say an
>> > hmac covering the TransactionalId and PID).
>> >
>> >
>> > > 3. Will every broker check that a client sending transactional produce
>> > > requests at least has write access to transaction log topic since it
>> is
>> > not
>> > > validating transactional.id (for every produce request)?
>> >
>> >  4. I understand that brokers cannot authorize the transactional id for
>> > each
>> > > produce request since requests contain only the PID. But since there
>> is a
>> > > one-to-one mapping between PID and transactional.id, and a
>> connection is
>> > > never expected to change its transactional.id, perhaps it is
>> feasible to
>> > > add authorization and cache the results in the Session? Perhaps not
>> for
>> > > version 1, but feels like it will be good to close the security gap
>> here.
>> > > Obviously it would be simpler if transactional.id was in the produce
>> > > request if the overhead was acceptable.
>> >
>> >
>> > I think my response above addresses both of these. We should include the
>> > TransactionalId in the ProduceRequest. Of course it need not be
>> included in
>> > the message format, so I'm not too concerned about the additional
>> overhead
>> > it adds.
>> >
>> > Thanks,
>> > Jason
>> >
>> >
>> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk> wrote:
>> >
>> > > Comments inline.
>> > >
>> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io>
>> > > wrote:
>> > >
>> > > > Took me a while to remember why we didn't do this. The timestamp
>> that
>> > is
>> > > > included at the message set level is the max timestamp of all
>> messages
>> > in
>> > > > the message set as is the case in the current message format (I will
>> > > update
>> > > > the document to make this explicit). We could make the message
>> > timestamps
>> > > > relative to the max timestamp, but that makes serialization a bit
>> > awkward
>> > > > since the timestamps are not assumed to be increasing sequentially
>> or
>> > > > monotonically. Once the messages in the message set had been
>> > determined,
>> > > we
>> > > > would need to go back and adjust the relative timestamps.
>> > > >
>> > >
>> > > Yes, I thought this would be a bit tricky and hence why I mentioned
>> the
>> > > option of adding a new field at the message set level for the first
>> > > timestamp even though that's not ideal either.
>> > >
>> > > Here's one idea. We let the timestamps in the messages be varints,
>> but we
>> > > > make their values be relative to the timestamp of the previous
>> message,
>> > > > with the timestamp of the first message being absolute. For
>> example, if
>> > > we
>> > > > had timestamps 500, 501, 499, then we would write 500 for the first
>> > > > message, 1 for the next, and -2 for the final message. Would that
>> work?
>> > > Let
>> > > > me think a bit about it and see if there are any problems.
>> > > >
>> > >
>> > > It's an interesting idea. Comparing to the option of having the first
>> > > timestamp in the message set, It's a little more space efficient as we
>> > > don't have both a full timestamp in the message set _and_ a varint in
>> the
>> > > first message (which would always be 0, so we avoid the extra byte)
>> and
>> > > also the deltas could be a little smaller in the common case. The main
>> > > downside is that it introduces a semantics inconsistency between the
>> > first
>> > > message and the rest. Not ideal, but maybe we can live with that.
>> > >
>> > > Ismael
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Tom,

I said this in the voting thread, but can the authors include a section
> about new ACLs if there are going to be ACLs for TransactionalId. It's
> mentioned in the google doc, but I think new ACLs should be in a KIP
> directly.


We've updated the wiki. Can you take a look and let us know if you have
additional concerns?

Thanks,
Jason

On Fri, Feb 3, 2017 at 1:52 PM, Rajini Sivaram <ra...@gmail.com>
wrote:

> Hi Jason,
>
> Thank you for the responses. Agree that authorizing transactional.id in
> the
> producer requests will be good enough for version 1. And making it tighter
> in future based on delegation tokens sounds good too.
>
> Regards,
>
> Rajini
>
>
> On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Rajini,
> >
> > Thanks for the questions. Responses below:
> >
> >
> > > 1. Will the transaction coordinator check topic ACLs based on the
> > > requesting client's credentials? Access to transaction logs, topics
> being
> > > added for transaction etc?
> >
> >
> > Good question. I think it makes sense to check topic Write permission
> when
> > adding partitions to the transaction. I'll add this to the document.
> > Perhaps authorization to the transaction log itself, however, can be
> > assumed from having access to the ProducerTransactionalId resource? This
> > would be similar to how access to __consumer_offsets is assumed if the
> > client has access to the Group resource.
> >
> > 2. If I create a transactional produce request (by hand, not using the
> > > producer API) with a random PID (random, hence unlikely to be in use),
> > will
> > > the broker append a transactional message to the logs, preventing LSO
> > from
> > > moving forward? What validation will broker do for PIDs?
> >
> >
> > Yes, that is correct. Validation of the TransactionalId to PID binding
> is a
> > known gap in the current proposal, and is discussed in the design
> document.
> > Now that I'm thinking about it a bit more, I think there is a good case
> for
> > including the TransactionalId in the ProduceRequest (I think Jun
> suggested
> > this previously). Verifying it does not ensure that the included PID is
> > correct, but it does ensure that the client is authorized to use
> > transactions. If the client wanted to do an "endless transaction attack,"
> > having Write access to the topic and an authorized transactionalID is all
> > they would need anyway even if we could authorize the PID itself. This
> > seems like a worthwhile improvement.
> >
> > For future work, my half-baked idea to authorize the PID binding is to
> > leverage the delegation work in KIP-48. When the PID is generated, we can
> > give the producer a token which is then used in produce requests (say an
> > hmac covering the TransactionalId and PID).
> >
> >
> > > 3. Will every broker check that a client sending transactional produce
> > > requests at least has write access to transaction log topic since it is
> > not
> > > validating transactional.id (for every produce request)?
> >
> >  4. I understand that brokers cannot authorize the transactional id for
> > each
> > > produce request since requests contain only the PID. But since there
> is a
> > > one-to-one mapping between PID and transactional.id, and a connection
> is
> > > never expected to change its transactional.id, perhaps it is feasible
> to
> > > add authorization and cache the results in the Session? Perhaps not for
> > > version 1, but feels like it will be good to close the security gap
> here.
> > > Obviously it would be simpler if transactional.id was in the produce
> > > request if the overhead was acceptable.
> >
> >
> > I think my response above addresses both of these. We should include the
> > TransactionalId in the ProduceRequest. Of course it need not be included
> in
> > the message format, so I'm not too concerned about the additional
> overhead
> > it adds.
> >
> > Thanks,
> > Jason
> >
> >
> > On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Comments inline.
> > >
> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Took me a while to remember why we didn't do this. The timestamp that
> > is
> > > > included at the message set level is the max timestamp of all
> messages
> > in
> > > > the message set as is the case in the current message format (I will
> > > update
> > > > the document to make this explicit). We could make the message
> > timestamps
> > > > relative to the max timestamp, but that makes serialization a bit
> > awkward
> > > > since the timestamps are not assumed to be increasing sequentially or
> > > > monotonically. Once the messages in the message set had been
> > determined,
> > > we
> > > > would need to go back and adjust the relative timestamps.
> > > >
> > >
> > > Yes, I thought this would be a bit tricky and hence why I mentioned the
> > > option of adding a new field at the message set level for the first
> > > timestamp even though that's not ideal either.
> > >
> > > Here's one idea. We let the timestamps in the messages be varints, but
> we
> > > > make their values be relative to the timestamp of the previous
> message,
> > > > with the timestamp of the first message being absolute. For example,
> if
> > > we
> > > > had timestamps 500, 501, 499, then we would write 500 for the first
> > > > message, 1 for the next, and -2 for the final message. Would that
> work?
> > > Let
> > > > me think a bit about it and see if there are any problems.
> > > >
> > >
> > > It's an interesting idea. Comparing to the option of having the first
> > > timestamp in the message set, It's a little more space efficient as we
> > > don't have both a full timestamp in the message set _and_ a varint in
> the
> > > first message (which would always be 0, so we avoid the extra byte) and
> > > also the deltas could be a little smaller in the common case. The main
> > > downside is that it introduces a semantics inconsistency between the
> > first
> > > message and the rest. Not ideal, but maybe we can live with that.
> > >
> > > Ismael
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <ra...@gmail.com>.
Hi Jason,

Thank you for the responses. Agree that authorizing transactional.id in the
producer requests will be good enough for version 1. And making it tighter
in future based on delegation tokens sounds good too.

Regards,

Rajini


On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Rajini,
>
> Thanks for the questions. Responses below:
>
>
> > 1. Will the transaction coordinator check topic ACLs based on the
> > requesting client's credentials? Access to transaction logs, topics being
> > added for transaction etc?
>
>
> Good question. I think it makes sense to check topic Write permission when
> adding partitions to the transaction. I'll add this to the document.
> Perhaps authorization to the transaction log itself, however, can be
> assumed from having access to the ProducerTransactionalId resource? This
> would be similar to how access to __consumer_offsets is assumed if the
> client has access to the Group resource.
>
> 2. If I create a transactional produce request (by hand, not using the
> > producer API) with a random PID (random, hence unlikely to be in use),
> will
> > the broker append a transactional message to the logs, preventing LSO
> from
> > moving forward? What validation will broker do for PIDs?
>
>
> Yes, that is correct. Validation of the TransactionalId to PID binding is a
> known gap in the current proposal, and is discussed in the design document.
> Now that I'm thinking about it a bit more, I think there is a good case for
> including the TransactionalId in the ProduceRequest (I think Jun suggested
> this previously). Verifying it does not ensure that the included PID is
> correct, but it does ensure that the client is authorized to use
> transactions. If the client wanted to do an "endless transaction attack,"
> having Write access to the topic and an authorized transactionalID is all
> they would need anyway even if we could authorize the PID itself. This
> seems like a worthwhile improvement.
>
> For future work, my half-baked idea to authorize the PID binding is to
> leverage the delegation work in KIP-48. When the PID is generated, we can
> give the producer a token which is then used in produce requests (say an
> hmac covering the TransactionalId and PID).
>
>
> > 3. Will every broker check that a client sending transactional produce
> > requests at least has write access to transaction log topic since it is
> not
> > validating transactional.id (for every produce request)?
>
>  4. I understand that brokers cannot authorize the transactional id for
> each
> > produce request since requests contain only the PID. But since there is a
> > one-to-one mapping between PID and transactional.id, and a connection is
> > never expected to change its transactional.id, perhaps it is feasible to
> > add authorization and cache the results in the Session? Perhaps not for
> > version 1, but feels like it will be good to close the security gap here.
> > Obviously it would be simpler if transactional.id was in the produce
> > request if the overhead was acceptable.
>
>
> I think my response above addresses both of these. We should include the
> TransactionalId in the ProduceRequest. Of course it need not be included in
> the message format, so I'm not too concerned about the additional overhead
> it adds.
>
> Thanks,
> Jason
>
>
> On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Comments inline.
> >
> > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Took me a while to remember why we didn't do this. The timestamp that
> is
> > > included at the message set level is the max timestamp of all messages
> in
> > > the message set as is the case in the current message format (I will
> > update
> > > the document to make this explicit). We could make the message
> timestamps
> > > relative to the max timestamp, but that makes serialization a bit
> awkward
> > > since the timestamps are not assumed to be increasing sequentially or
> > > monotonically. Once the messages in the message set had been
> determined,
> > we
> > > would need to go back and adjust the relative timestamps.
> > >
> >
> > Yes, I thought this would be a bit tricky and hence why I mentioned the
> > option of adding a new field at the message set level for the first
> > timestamp even though that's not ideal either.
> >
> > Here's one idea. We let the timestamps in the messages be varints, but we
> > > make their values be relative to the timestamp of the previous message,
> > > with the timestamp of the first message being absolute. For example, if
> > we
> > > had timestamps 500, 501, 499, then we would write 500 for the first
> > > message, 1 for the next, and -2 for the final message. Would that work?
> > Let
> > > me think a bit about it and see if there are any problems.
> > >
> >
> > It's an interesting idea. Comparing to the option of having the first
> > timestamp in the message set, It's a little more space efficient as we
> > don't have both a full timestamp in the message set _and_ a varint in the
> > first message (which would always be 0, so we avoid the extra byte) and
> > also the deltas could be a little smaller in the common case. The main
> > downside is that it introduces a semantics inconsistency between the
> first
> > message and the rest. Not ideal, but maybe we can live with that.
> >
> > Ismael
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Rajini,

Thanks for the questions. Responses below:


> 1. Will the transaction coordinator check topic ACLs based on the
> requesting client's credentials? Access to transaction logs, topics being
> added for transaction etc?


Good question. I think it makes sense to check topic Write permission when
adding partitions to the transaction. I'll add this to the document.
Perhaps authorization to the transaction log itself, however, can be
assumed from having access to the ProducerTransactionalId resource? This
would be similar to how access to __consumer_offsets is assumed if the
client has access to the Group resource.

2. If I create a transactional produce request (by hand, not using the
> producer API) with a random PID (random, hence unlikely to be in use), will
> the broker append a transactional message to the logs, preventing LSO from
> moving forward? What validation will broker do for PIDs?


Yes, that is correct. Validation of the TransactionalId to PID binding is a
known gap in the current proposal, and is discussed in the design document.
Now that I'm thinking about it a bit more, I think there is a good case for
including the TransactionalId in the ProduceRequest (I think Jun suggested
this previously). Verifying it does not ensure that the included PID is
correct, but it does ensure that the client is authorized to use
transactions. If the client wanted to do an "endless transaction attack,"
having Write access to the topic and an authorized transactionalID is all
they would need anyway even if we could authorize the PID itself. This
seems like a worthwhile improvement.

For future work, my half-baked idea to authorize the PID binding is to
leverage the delegation work in KIP-48. When the PID is generated, we can
give the producer a token which is then used in produce requests (say an
hmac covering the TransactionalId and PID).


> 3. Will every broker check that a client sending transactional produce
> requests at least has write access to transaction log topic since it is not
> validating transactional.id (for every produce request)?

 4. I understand that brokers cannot authorize the transactional id for each
> produce request since requests contain only the PID. But since there is a
> one-to-one mapping between PID and transactional.id, and a connection is
> never expected to change its transactional.id, perhaps it is feasible to
> add authorization and cache the results in the Session? Perhaps not for
> version 1, but feels like it will be good to close the security gap here.
> Obviously it would be simpler if transactional.id was in the produce
> request if the overhead was acceptable.


I think my response above addresses both of these. We should include the
TransactionalId in the ProduceRequest. Of course it need not be included in
the message format, so I'm not too concerned about the additional overhead
it adds.

Thanks,
Jason


On Fri, Feb 3, 2017 at 6:52 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Comments inline.
>
> On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Took me a while to remember why we didn't do this. The timestamp that is
> > included at the message set level is the max timestamp of all messages in
> > the message set as is the case in the current message format (I will
> update
> > the document to make this explicit). We could make the message timestamps
> > relative to the max timestamp, but that makes serialization a bit awkward
> > since the timestamps are not assumed to be increasing sequentially or
> > monotonically. Once the messages in the message set had been determined,
> we
> > would need to go back and adjust the relative timestamps.
> >
>
> Yes, I thought this would be a bit tricky and hence why I mentioned the
> option of adding a new field at the message set level for the first
> timestamp even though that's not ideal either.
>
> Here's one idea. We let the timestamps in the messages be varints, but we
> > make their values be relative to the timestamp of the previous message,
> > with the timestamp of the first message being absolute. For example, if
> we
> > had timestamps 500, 501, 499, then we would write 500 for the first
> > message, 1 for the next, and -2 for the final message. Would that work?
> Let
> > me think a bit about it and see if there are any problems.
> >
>
> It's an interesting idea. Comparing to the option of having the first
> timestamp in the message set, It's a little more space efficient as we
> don't have both a full timestamp in the message set _and_ a varint in the
> first message (which would always be 0, so we avoid the extra byte) and
> also the deltas could be a little smaller in the common case. The main
> downside is that it introduces a semantics inconsistency between the first
> message and the rest. Not ideal, but maybe we can live with that.
>
> Ismael
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Comments inline.

On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined, we
> would need to go back and adjust the relative timestamps.
>

Yes, I thought this would be a bit tricky and hence why I mentioned the
option of adding a new field at the message set level for the first
timestamp even though that's not ideal either.

Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work? Let
> me think a bit about it and see if there are any problems.
>

It's an interesting idea. Comparing to the option of having the first
timestamp in the message set, It's a little more space efficient as we
don't have both a full timestamp in the message set _and_ a varint in the
first message (which would always be 0, so we avoid the extra byte) and
also the deltas could be a little smaller in the common case. The main
downside is that it introduces a semantics inconsistency between the first
message and the rest. Not ideal, but maybe we can live with that.

Ismael

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <ra...@gmail.com>.
I have a few questions on security (sorry, only just catching up on the
updates).

1. Will the transaction coordinator check topic ACLs based on the
requesting client's credentials? Access to transaction logs, topics being
added for transaction etc?
2. If I create a transactional produce request (by hand, not using the
producer API) with a random PID (random, hence unlikely to be in use), will
the broker append a transactional message to the logs, preventing LSO from
moving forward? What validation will broker do for PIDs?
3. Will every broker check that a client sending transactional produce
requests at least has write access to transaction log topic since it is not
validating transactional.id (for every produce request)?
4. I understand that brokers cannot authorize the transactional id for each
produce request since requests contain only the PID. But since there is a
one-to-one mapping between PID and transactional.id, and a connection is
never expected to change its transactional.id, perhaps it is feasible to
add authorization and cache the results in the Session? Perhaps not for
version 1, but feels like it will be good to close the security gap here.
Obviously it would be simpler if transactional.id was in the produce
request if the overhead was acceptable.

Thank you,

Rajini


On Thu, Feb 2, 2017 at 8:37 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Yes, I'd also prefer the option where we only have a checksum at the
> message set level. I didn't suggest it due to the mentioned auditing use
> cases, but if they can be satisfied in some other way, then that would be
> great.
>
> Ismael
>
> On 2 Feb 2017 7:03 pm, "Jason Gustafson" <ja...@confluent.io> wrote:
>
> One more:
>
> 1. I did some benchmarking and CRC32C seems to be a massive win when using
> > the hardware instruction (particularly for messages larger than 65k), so
> > I'm keen on taking advantage of the message format version bump to add
> > support for it. I can write a separate KIP for this as it's not tied to
> > Exactly-once, but it would be good to include the code change in the same
> > PR that bumps the message format version. The benchmark and results can
> be
> > found in the following link:
> > https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>
>
> Yeah, makes sense. We can add this to this KIP or do it separately,
> whichever you prefer. I have also been very interested in removing the
> individual message CRCs. The main reason we haven't done so is because some
> auditing applications depend on them, but there are cases where it's
> already unsafe to depend on the message CRCs not changing on the broker
> (message conversion and the use of log append time can both result in new
> message-level crcs). So I'm wondering a bit about the use cases that
> require the message CRCs and how they handle this. Perhaps if it is not
> dependable anyway, we can remove it and safe some space and computation.
>
> -Jason
>
>
> On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Ismael,
> >
> > 2. The message timestamp field is 8 bytes. Did we consider storing the
> >> first timestamp in the message set and then storing deltas using varints
> >> in
> >> the messages like we do for offsets (the difference would be the usage
> of
> >> signed varints)? It seems like the deltas would be quite a bit smaller
> in
> >> the common case (potentially 0 for log append time, so we could even not
> >> store them at all using attributes like we do for key/value lengths). An
> >> alternative is using MaxTimestamp that is already present in the message
> >> set and computing deltas from that, but that seems more complicated. In
> >> any
> >> case, details aside, was this idea considered and rejected or is it
> worth
> >> exploring further?
> >
> >
> > Took me a while to remember why we didn't do this. The timestamp that is
> > included at the message set level is the max timestamp of all messages in
> > the message set as is the case in the current message format (I will
> update
> > the document to make this explicit). We could make the message timestamps
> > relative to the max timestamp, but that makes serialization a bit awkward
> > since the timestamps are not assumed to be increasing sequentially or
> > monotonically. Once the messages in the message set had been determined,
> we
> > would need to go back and adjust the relative timestamps.
> >
> > Here's one idea. We let the timestamps in the messages be varints, but we
> > make their values be relative to the timestamp of the previous message,
> > with the timestamp of the first message being absolute. For example, if
> we
> > had timestamps 500, 501, 499, then we would write 500 for the first
> > message, 1 for the next, and -2 for the final message. Would that work?
> Let
> > me think a bit about it and see if there are any problems.
> >
> > -Jason
> >
> > On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> >> Good point Tom. We will update the KIP with the ACLs section and also
> the
> >> message format changes.
> >>
> >> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> >> >
> >> > I said this in the voting thread, but can the authors include a
> section
> >> > about new ACLs if there are going to be ACLs for TransactionalId. It's
> >> > mentioned in the google doc, but I think new ACLs should be in a KIP
> >> > directly.
> >> >
> >> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk>
> wrote:
> >> >>
> >> >> Thanks for the responses and updates to the document, Guozhang and
> >> Jason.
> >> >> They look good. One follow-up and one additional comment:
> >> >>
> >> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
> >> using
> >> >> the hardware instruction (particularly for messages larger than 65k),
> >> so
> >> >> I'm keen on taking advantage of the message format version bump to
> add
> >> >> support for it. I can write a separate KIP for this as it's not tied
> to
> >> >> Exactly-once, but it would be good to include the code change in the
> >> same
> >> >> PR that bumps the message format version. The benchmark and results
> >> can be
> >> >> found in the following link:
> >> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
> >> >>
> >> >> 2. The message timestamp field is 8 bytes. Did we consider storing
> the
> >> >> first timestamp in the message set and then storing deltas using
> >> varints in
> >> >> the messages like we do for offsets (the difference would be the
> usage
> >> of
> >> >> signed varints)? It seems like the deltas would be quite a bit
> smaller
> >> in
> >> >> the common case (potentially 0 for log append time, so we could even
> >> not
> >> >> store them at all using attributes like we do for key/value lengths).
> >> An
> >> >> alternative is using MaxTimestamp that is already present in the
> >> message
> >> >> set and computing deltas from that, but that seems more complicated.
> >> In any
> >> >> case, details aside, was this idea considered and rejected or is it
> >> worth
> >> >> exploring further?
> >> >>
> >> >> Ismael
> >> >>
> >> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> >> >> wrote:
> >> >>
> >> >>> Ismael,
> >> >>>
> >> >>> Thanks for the comments. A few responses below:
> >> >>>
> >> >>>
> >> >>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >> introduces
> >> >>> a
> >> >>>> compatibility issue with regards to existing third-party
> authorizers.
> >> >> It
> >> >>>> would be good to highlight this in the migration/compatibility
> >> section.
> >> >>>
> >> >>>
> >> >>> Ack. I added a note in the migration section.
> >> >>>
> >> >>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>> considered
> >> >>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>> 0.10.2.0)?
> >> >>>
> >> >>>
> >> >>> Thanks, I added a few notes about client compatibility to the
> >> migration
> >> >>> section. I covered the main issues that come to mind, but let me
> know
> >> if
> >> >>> you think of others.
> >> >>>
> >> >>> 7. It seems like there is a bit of inconsistency when it comes to
> >> naming
> >> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >> conventions.
> >> >>>> There are a few other examples like that and it would be good to
> >> clean
> >> >>> them
> >> >>>> up.
> >> >>>
> >> >>>
> >> >>> Let's go with InitPidRequest for consistency.  Haha,
> "InitPIdRequest"
> >> >> seems
> >> >>> like a compromise which satisfies no one.
> >> >>>
> >> >>>
> >> >>> -Jason
> >> >>>
> >> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> >> >> wrote:
> >> >>>
> >> >>>> Ismael, thanks for your feedbacks. Replied inline.
> >> >>>>
> >> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
> >> wrote:
> >> >>>>>
> >> >>>>> Hi all,
> >> >>>>>
> >> >>>>> A few comments follow:
> >> >>>>>
> >> >>>>> 1. The document states "inter-broker communications will be
> >> increased
> >> >>> by
> >> >>>> M
> >> >>>>> * N * P round trips per sec. We need to conduct some system
> >> >> performance
> >> >>>>> test to make sure this additional inter-broker traffic would not
> >> >>> largely
> >> >>>>> impact the broker cluster". Has this testing been done? And if
> not,
> >> >> are
> >> >>>> we
> >> >>>>> planning to do it soon? It seems important to validate this sooner
> >> >>> rather
> >> >>>>> than later. This applies more generally too, it would be great to
> >> >>>>> understand how the new message format affects the producer with
> >> small
> >> >>>>> messages, for example.
> >> >>>>>
> >> >>>>>
> >> >>>> Yes we are conducting the perf tests with the message format
> changes
> >> in
> >> >>> the
> >> >>>> first stage; then the inter-broker communication with minimal
> >> >> transaction
> >> >>>> coordinator implementations in the second stage.
> >> >>>>
> >> >>>>
> >> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >>> introduces
> >> >>>> a
> >> >>>>> compatibility issue with regards to existing third-party
> >> authorizers.
> >> >>> It
> >> >>>>> would be good to highlight this in the migration/compatibility
> >> >> section.
> >> >>>>>
> >> >>>>> 3. I was happy to see that default values for the new configs have
> >> >> been
> >> >>>>> added to the document since I last checked it. It would be good to
> >> >>>> explain
> >> >>>>> the motivation for the choices.
> >> >>>>>
> >> >>>>>
> >> >>>> Updated doc.
> >> >>>>
> >> >>>>
> >> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>>> considered
> >> >>>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>>> 0.10.2.0)?
> >> >>>>>
> >> >>>>> 5. transactional.id sounds good
> >> >>>>>
> >> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have
> we
> >> >>>>> considered mitigating the performance cost by using the more
> >> >> performant
> >> >>>>> CRC32c in the new message format version?
> >> >>>>>
> >> >>>>>
> >> >>>> We have not discussed about this before. But I think it should be
> >> >> doable
> >> >>> as
> >> >>>> long as we can include the additional conversion logic in the
> >> migration
> >> >>>> plan.
> >> >>>>
> >> >>>>
> >> >>>>> Nits:
> >> >>>>>
> >> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
> >> >>> naming
> >> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >>> conventions.
> >> >>>>> There are a few other examples like that and it would be good to
> >> >> clean
> >> >>>> them
> >> >>>>> up.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree with the inconsistency issue. About the name itself though,
> >> >>> should
> >> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
> >> since
> >> >>> we
> >> >>>> need to capitalize "I" right?
> >> >>>>
> >> >>>>
> >> >>>>> 8. The document states "The first four fields of a message set in
> >> >> this
> >> >>>>> format must to be the same as the existing format because any
> fields
> >> >>>> before
> >> >>>>> the magic byte cannot be changed in order to provide a path for
> >> >>> upgrades
> >> >>>>> following a similar approach as was used in KIP-32". This makes
> >> >> things
> >> >>>>> easier, but it seems to me that the only strict requirement is
> that
> >> >> the
> >> >>>>> magic byte remains in the same offset and with the same size.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree theoretically it is not required, but I think in practice
> it
> >> is
> >> >>>> actually better to make it more restrict: the three fields before
> >> magic
> >> >>>> byte are offset, length, and crc. Among them, crc needs to be
> before
> >> >>> magic
> >> >>>> byte if it wants to cover the magic byte fields; length would
> better
> >> be
> >> >>>> before the magic byte as well for pre-allocate memory to
> >> >> deser/decompress
> >> >>>> the message set, and the only field that does not matter too much
> to
> >> be
> >> >>>> after magic byte is offset, but in KIP-98 we will use it as the
> base
> >> >>> offset
> >> >>>> for message set and some validation checks can be optimized to not
> >> scan
> >> >>>> through the whole message with this field in front of the format.
> >> >>>>
> >> >>>>
> >> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> >>>>> wrote:
> >> >>>>>
> >> >>>>>> Hello Folks,
> >> >>>>>>
> >> >>>>>> We have addressed all the comments collected so far, and would
> like
> >> >>> to
> >> >>>>>> propose a voting thread this Wednesday. If you have any further
> >> >>>> comments
> >> >>>>> on
> >> >>>>>> this KIP, please feel free to continue sending them on this
> thread
> >> >>>> before
> >> >>>>>> that.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> Guozhang
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> >> >> jason@confluent.io
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>> +1 for transactional.id.
> >> >>>>>>>
> >> >>>>>>> -Jason
> >> >>>>>>>
> >> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> >> >> wangguoz@gmail.com
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>>> If I have to choose between app.id and
> >> >> transactional.instance.id
> >> >>> ,
> >> >>>>> I'd
> >> >>>>>>>> choose the latter.
> >> >>>>>>>>
> >> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
> >> >>> even
> >> >>>>>>> better.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> Guozhang
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> >> >>>> apurva@confluent.io>
> >> >>>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
> >> >>> has
> >> >>>>>> caused
> >> >>>>>>>>> some
> >> >>>>>>>>>> confusion. We're considering the following renaming:
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> >> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
> >> >>>>>>>> transactional.id
> >> >>>>>>>>> )
> >> >>>>>>>>>
> >> >>>>>>>>> This makes it clear that this id just needs to be set when
> >> >> the
> >> >>>>>>>> application
> >> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
> >> >>> in
> >> >>>>> the
> >> >>>>>>>>> context of how this id is used, viz. to maintain transactions
> >> >>>>> across
> >> >>>>>>>>> producer sessions.
> >> >>>>>>>>>
> >> >>>>>>>>> Thanks,
> >> >>>>>>>>> Apurva
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> --
> >> >>>>>>>> -- Guozhang
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> --
> >> >>>>>> -- Guozhang
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> --
> >> >>>> -- Guozhang
> >> >>>>
> >> >>>
> >> >>
> >>
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Yes, I'd also prefer the option where we only have a checksum at the
message set level. I didn't suggest it due to the mentioned auditing use
cases, but if they can be satisfied in some other way, then that would be
great.

Ismael

On 2 Feb 2017 7:03 pm, "Jason Gustafson" <ja...@confluent.io> wrote:

One more:

1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.


Yeah, makes sense. We can add this to this KIP or do it separately,
whichever you prefer. I have also been very interested in removing the
individual message CRCs. The main reason we haven't done so is because some
auditing applications depend on them, but there are cases where it's
already unsafe to depend on the message CRCs not changing on the broker
(message conversion and the use of log append time can both result in new
message-level crcs). So I'm wondering a bit about the use cases that
require the message CRCs and how they handle this. Perhaps if it is not
dependable anyway, we can remove it and safe some space and computation.

-Jason


On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Ismael,
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints
>> in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In
>> any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>
>
> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will
update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined,
we
> would need to go back and adjust the relative timestamps.
>
> Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work?
Let
> me think a bit about it and see if there are any problems.
>
> -Jason
>
> On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Good point Tom. We will update the KIP with the ACLs section and also the
>> message format changes.
>>
>> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
>> >
>> > I said this in the voting thread, but can the authors include a section
>> > about new ACLs if there are going to be ACLs for TransactionalId. It's
>> > mentioned in the google doc, but I think new ACLs should be in a KIP
>> > directly.
>> >
>> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> >>
>> >> Thanks for the responses and updates to the document, Guozhang and
>> Jason.
>> >> They look good. One follow-up and one additional comment:
>> >>
>> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
>> using
>> >> the hardware instruction (particularly for messages larger than 65k),
>> so
>> >> I'm keen on taking advantage of the message format version bump to add
>> >> support for it. I can write a separate KIP for this as it's not tied
to
>> >> Exactly-once, but it would be good to include the code change in the
>> same
>> >> PR that bumps the message format version. The benchmark and results
>> can be
>> >> found in the following link:
>> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> >>
>> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> >> first timestamp in the message set and then storing deltas using
>> varints in
>> >> the messages like we do for offsets (the difference would be the usage
>> of
>> >> signed varints)? It seems like the deltas would be quite a bit smaller
>> in
>> >> the common case (potentially 0 for log append time, so we could even
>> not
>> >> store them at all using attributes like we do for key/value lengths).
>> An
>> >> alternative is using MaxTimestamp that is already present in the
>> message
>> >> set and computing deltas from that, but that seems more complicated.
>> In any
>> >> case, details aside, was this idea considered and rejected or is it
>> worth
>> >> exploring further?
>> >>
>> >> Ismael
>> >>
>> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >>
>> >>> Ismael,
>> >>>
>> >>> Thanks for the comments. A few responses below:
>> >>>
>> >>>
>> >>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >> introduces
>> >>> a
>> >>>> compatibility issue with regards to existing third-party
authorizers.
>> >> It
>> >>>> would be good to highlight this in the migration/compatibility
>> section.
>> >>>
>> >>>
>> >>> Ack. I added a note in the migration section.
>> >>>
>> >>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>> considered
>> >>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>> 0.10.2.0)?
>> >>>
>> >>>
>> >>> Thanks, I added a few notes about client compatibility to the
>> migration
>> >>> section. I covered the main issues that come to mind, but let me know
>> if
>> >>> you think of others.
>> >>>
>> >>> 7. It seems like there is a bit of inconsistency when it comes to
>> naming
>> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >> conventions.
>> >>>> There are a few other examples like that and it would be good to
>> clean
>> >>> them
>> >>>> up.
>> >>>
>> >>>
>> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> >> seems
>> >>> like a compromise which satisfies no one.
>> >>>
>> >>>
>> >>> -Jason
>> >>>
>> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>
>> >>>> Ismael, thanks for your feedbacks. Replied inline.
>> >>>>
>> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
>> wrote:
>> >>>>>
>> >>>>> Hi all,
>> >>>>>
>> >>>>> A few comments follow:
>> >>>>>
>> >>>>> 1. The document states "inter-broker communications will be
>> increased
>> >>> by
>> >>>> M
>> >>>>> * N * P round trips per sec. We need to conduct some system
>> >> performance
>> >>>>> test to make sure this additional inter-broker traffic would not
>> >>> largely
>> >>>>> impact the broker cluster". Has this testing been done? And if not,
>> >> are
>> >>>> we
>> >>>>> planning to do it soon? It seems important to validate this sooner
>> >>> rather
>> >>>>> than later. This applies more generally too, it would be great to
>> >>>>> understand how the new message format affects the producer with
>> small
>> >>>>> messages, for example.
>> >>>>>
>> >>>>>
>> >>>> Yes we are conducting the perf tests with the message format changes
>> in
>> >>> the
>> >>>> first stage; then the inter-broker communication with minimal
>> >> transaction
>> >>>> coordinator implementations in the second stage.
>> >>>>
>> >>>>
>> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >>> introduces
>> >>>> a
>> >>>>> compatibility issue with regards to existing third-party
>> authorizers.
>> >>> It
>> >>>>> would be good to highlight this in the migration/compatibility
>> >> section.
>> >>>>>
>> >>>>> 3. I was happy to see that default values for the new configs have
>> >> been
>> >>>>> added to the document since I last checked it. It would be good to
>> >>>> explain
>> >>>>> the motivation for the choices.
>> >>>>>
>> >>>>>
>> >>>> Updated doc.
>> >>>>
>> >>>>
>> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>>> considered
>> >>>>> if there's any additional work required due to KIP-97 (introduced
in
>> >>>>> 0.10.2.0)?
>> >>>>>
>> >>>>> 5. transactional.id sounds good
>> >>>>>
>> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>> >>>>> considered mitigating the performance cost by using the more
>> >> performant
>> >>>>> CRC32c in the new message format version?
>> >>>>>
>> >>>>>
>> >>>> We have not discussed about this before. But I think it should be
>> >> doable
>> >>> as
>> >>>> long as we can include the additional conversion logic in the
>> migration
>> >>>> plan.
>> >>>>
>> >>>>
>> >>>>> Nits:
>> >>>>>
>> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
>> >>> naming
>> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >>> conventions.
>> >>>>> There are a few other examples like that and it would be good to
>> >> clean
>> >>>> them
>> >>>>> up.
>> >>>>>
>> >>>>>
>> >>>> I agree with the inconsistency issue. About the name itself though,
>> >>> should
>> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
>> since
>> >>> we
>> >>>> need to capitalize "I" right?
>> >>>>
>> >>>>
>> >>>>> 8. The document states "The first four fields of a message set in
>> >> this
>> >>>>> format must to be the same as the existing format because any
fields
>> >>>> before
>> >>>>> the magic byte cannot be changed in order to provide a path for
>> >>> upgrades
>> >>>>> following a similar approach as was used in KIP-32". This makes
>> >> things
>> >>>>> easier, but it seems to me that the only strict requirement is that
>> >> the
>> >>>>> magic byte remains in the same offset and with the same size.
>> >>>>>
>> >>>>>
>> >>>> I agree theoretically it is not required, but I think in practice it
>> is
>> >>>> actually better to make it more restrict: the three fields before
>> magic
>> >>>> byte are offset, length, and crc. Among them, crc needs to be before
>> >>> magic
>> >>>> byte if it wants to cover the magic byte fields; length would better
>> be
>> >>>> before the magic byte as well for pre-allocate memory to
>> >> deser/decompress
>> >>>> the message set, and the only field that does not matter too much to
>> be
>> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
>> >>> offset
>> >>>> for message set and some validation checks can be optimized to not
>> scan
>> >>>> through the whole message with this field in front of the format.
>> >>>>
>> >>>>
>> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Hello Folks,
>> >>>>>>
>> >>>>>> We have addressed all the comments collected so far, and would
like
>> >>> to
>> >>>>>> propose a voting thread this Wednesday. If you have any further
>> >>>> comments
>> >>>>> on
>> >>>>>> this KIP, please feel free to continue sending them on this thread
>> >>>> before
>> >>>>>> that.
>> >>>>>>
>> >>>>>>
>> >>>>>> Guozhang
>> >>>>>>
>> >>>>>>
>> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> >> jason@confluent.io
>> >>>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>> +1 for transactional.id.
>> >>>>>>>
>> >>>>>>> -Jason
>> >>>>>>>
>> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> >> wangguoz@gmail.com
>> >>>>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> If I have to choose between app.id and
>> >> transactional.instance.id
>> >>> ,
>> >>>>> I'd
>> >>>>>>>> choose the latter.
>> >>>>>>>>
>> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>> >>> even
>> >>>>>>> better.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>> >>>> apurva@confluent.io>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>> >>> has
>> >>>>>> caused
>> >>>>>>>>> some
>> >>>>>>>>>> confusion. We're considering the following renaming:
>> >>>>>>>>>>
>> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>> >>>>>>>> transactional.id
>> >>>>>>>>> )
>> >>>>>>>>>
>> >>>>>>>>> This makes it clear that this id just needs to be set when
>> >> the
>> >>>>>>>> application
>> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
>> >>> in
>> >>>>> the
>> >>>>>>>>> context of how this id is used, viz. to maintain transactions
>> >>>>> across
>> >>>>>>>>> producer sessions.
>> >>>>>>>>>
>> >>>>>>>>> Thanks,
>> >>>>>>>>> Apurva
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> --
>> >>>>>>>> -- Guozhang
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>>> -- Guozhang
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> --
>> >>>> -- Guozhang
>> >>>>
>> >>>
>> >>
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
One more:

1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.


Yeah, makes sense. We can add this to this KIP or do it separately,
whichever you prefer. I have also been very interested in removing the
individual message CRCs. The main reason we haven't done so is because some
auditing applications depend on them, but there are cases where it's
already unsafe to depend on the message CRCs not changing on the broker
(message conversion and the use of log append time can both result in new
message-level crcs). So I'm wondering a bit about the use cases that
require the message CRCs and how they handle this. Perhaps if it is not
dependable anyway, we can remove it and safe some space and computation.

-Jason


On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Ismael,
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints
>> in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In
>> any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>
>
> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined, we
> would need to go back and adjust the relative timestamps.
>
> Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work? Let
> me think a bit about it and see if there are any problems.
>
> -Jason
>
> On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Good point Tom. We will update the KIP with the ACLs section and also the
>> message format changes.
>>
>> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
>> >
>> > I said this in the voting thread, but can the authors include a section
>> > about new ACLs if there are going to be ACLs for TransactionalId. It's
>> > mentioned in the google doc, but I think new ACLs should be in a KIP
>> > directly.
>> >
>> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> >>
>> >> Thanks for the responses and updates to the document, Guozhang and
>> Jason.
>> >> They look good. One follow-up and one additional comment:
>> >>
>> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
>> using
>> >> the hardware instruction (particularly for messages larger than 65k),
>> so
>> >> I'm keen on taking advantage of the message format version bump to add
>> >> support for it. I can write a separate KIP for this as it's not tied to
>> >> Exactly-once, but it would be good to include the code change in the
>> same
>> >> PR that bumps the message format version. The benchmark and results
>> can be
>> >> found in the following link:
>> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> >>
>> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> >> first timestamp in the message set and then storing deltas using
>> varints in
>> >> the messages like we do for offsets (the difference would be the usage
>> of
>> >> signed varints)? It seems like the deltas would be quite a bit smaller
>> in
>> >> the common case (potentially 0 for log append time, so we could even
>> not
>> >> store them at all using attributes like we do for key/value lengths).
>> An
>> >> alternative is using MaxTimestamp that is already present in the
>> message
>> >> set and computing deltas from that, but that seems more complicated.
>> In any
>> >> case, details aside, was this idea considered and rejected or is it
>> worth
>> >> exploring further?
>> >>
>> >> Ismael
>> >>
>> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >>
>> >>> Ismael,
>> >>>
>> >>> Thanks for the comments. A few responses below:
>> >>>
>> >>>
>> >>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >> introduces
>> >>> a
>> >>>> compatibility issue with regards to existing third-party authorizers.
>> >> It
>> >>>> would be good to highlight this in the migration/compatibility
>> section.
>> >>>
>> >>>
>> >>> Ack. I added a note in the migration section.
>> >>>
>> >>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>> considered
>> >>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>> 0.10.2.0)?
>> >>>
>> >>>
>> >>> Thanks, I added a few notes about client compatibility to the
>> migration
>> >>> section. I covered the main issues that come to mind, but let me know
>> if
>> >>> you think of others.
>> >>>
>> >>> 7. It seems like there is a bit of inconsistency when it comes to
>> naming
>> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >> conventions.
>> >>>> There are a few other examples like that and it would be good to
>> clean
>> >>> them
>> >>>> up.
>> >>>
>> >>>
>> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> >> seems
>> >>> like a compromise which satisfies no one.
>> >>>
>> >>>
>> >>> -Jason
>> >>>
>> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>
>> >>>> Ismael, thanks for your feedbacks. Replied inline.
>> >>>>
>> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
>> wrote:
>> >>>>>
>> >>>>> Hi all,
>> >>>>>
>> >>>>> A few comments follow:
>> >>>>>
>> >>>>> 1. The document states "inter-broker communications will be
>> increased
>> >>> by
>> >>>> M
>> >>>>> * N * P round trips per sec. We need to conduct some system
>> >> performance
>> >>>>> test to make sure this additional inter-broker traffic would not
>> >>> largely
>> >>>>> impact the broker cluster". Has this testing been done? And if not,
>> >> are
>> >>>> we
>> >>>>> planning to do it soon? It seems important to validate this sooner
>> >>> rather
>> >>>>> than later. This applies more generally too, it would be great to
>> >>>>> understand how the new message format affects the producer with
>> small
>> >>>>> messages, for example.
>> >>>>>
>> >>>>>
>> >>>> Yes we are conducting the perf tests with the message format changes
>> in
>> >>> the
>> >>>> first stage; then the inter-broker communication with minimal
>> >> transaction
>> >>>> coordinator implementations in the second stage.
>> >>>>
>> >>>>
>> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >>> introduces
>> >>>> a
>> >>>>> compatibility issue with regards to existing third-party
>> authorizers.
>> >>> It
>> >>>>> would be good to highlight this in the migration/compatibility
>> >> section.
>> >>>>>
>> >>>>> 3. I was happy to see that default values for the new configs have
>> >> been
>> >>>>> added to the document since I last checked it. It would be good to
>> >>>> explain
>> >>>>> the motivation for the choices.
>> >>>>>
>> >>>>>
>> >>>> Updated doc.
>> >>>>
>> >>>>
>> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>>> considered
>> >>>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>>> 0.10.2.0)?
>> >>>>>
>> >>>>> 5. transactional.id sounds good
>> >>>>>
>> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>> >>>>> considered mitigating the performance cost by using the more
>> >> performant
>> >>>>> CRC32c in the new message format version?
>> >>>>>
>> >>>>>
>> >>>> We have not discussed about this before. But I think it should be
>> >> doable
>> >>> as
>> >>>> long as we can include the additional conversion logic in the
>> migration
>> >>>> plan.
>> >>>>
>> >>>>
>> >>>>> Nits:
>> >>>>>
>> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
>> >>> naming
>> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >>> conventions.
>> >>>>> There are a few other examples like that and it would be good to
>> >> clean
>> >>>> them
>> >>>>> up.
>> >>>>>
>> >>>>>
>> >>>> I agree with the inconsistency issue. About the name itself though,
>> >>> should
>> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
>> since
>> >>> we
>> >>>> need to capitalize "I" right?
>> >>>>
>> >>>>
>> >>>>> 8. The document states "The first four fields of a message set in
>> >> this
>> >>>>> format must to be the same as the existing format because any fields
>> >>>> before
>> >>>>> the magic byte cannot be changed in order to provide a path for
>> >>> upgrades
>> >>>>> following a similar approach as was used in KIP-32". This makes
>> >> things
>> >>>>> easier, but it seems to me that the only strict requirement is that
>> >> the
>> >>>>> magic byte remains in the same offset and with the same size.
>> >>>>>
>> >>>>>
>> >>>> I agree theoretically it is not required, but I think in practice it
>> is
>> >>>> actually better to make it more restrict: the three fields before
>> magic
>> >>>> byte are offset, length, and crc. Among them, crc needs to be before
>> >>> magic
>> >>>> byte if it wants to cover the magic byte fields; length would better
>> be
>> >>>> before the magic byte as well for pre-allocate memory to
>> >> deser/decompress
>> >>>> the message set, and the only field that does not matter too much to
>> be
>> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
>> >>> offset
>> >>>> for message set and some validation checks can be optimized to not
>> scan
>> >>>> through the whole message with this field in front of the format.
>> >>>>
>> >>>>
>> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Hello Folks,
>> >>>>>>
>> >>>>>> We have addressed all the comments collected so far, and would like
>> >>> to
>> >>>>>> propose a voting thread this Wednesday. If you have any further
>> >>>> comments
>> >>>>> on
>> >>>>>> this KIP, please feel free to continue sending them on this thread
>> >>>> before
>> >>>>>> that.
>> >>>>>>
>> >>>>>>
>> >>>>>> Guozhang
>> >>>>>>
>> >>>>>>
>> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> >> jason@confluent.io
>> >>>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>> +1 for transactional.id.
>> >>>>>>>
>> >>>>>>> -Jason
>> >>>>>>>
>> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> >> wangguoz@gmail.com
>> >>>>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> If I have to choose between app.id and
>> >> transactional.instance.id
>> >>> ,
>> >>>>> I'd
>> >>>>>>>> choose the latter.
>> >>>>>>>>
>> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>> >>> even
>> >>>>>>> better.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>> >>>> apurva@confluent.io>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>> >>> has
>> >>>>>> caused
>> >>>>>>>>> some
>> >>>>>>>>>> confusion. We're considering the following renaming:
>> >>>>>>>>>>
>> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>> >>>>>>>> transactional.id
>> >>>>>>>>> )
>> >>>>>>>>>
>> >>>>>>>>> This makes it clear that this id just needs to be set when
>> >> the
>> >>>>>>>> application
>> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
>> >>> in
>> >>>>> the
>> >>>>>>>>> context of how this id is used, viz. to maintain transactions
>> >>>>> across
>> >>>>>>>>> producer sessions.
>> >>>>>>>>>
>> >>>>>>>>> Thanks,
>> >>>>>>>>> Apurva
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> --
>> >>>>>>>> -- Guozhang
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>>> -- Guozhang
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> --
>> >>>> -- Guozhang
>> >>>>
>> >>>
>> >>
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Ismael,

2. The message timestamp field is 8 bytes. Did we consider storing the
> first timestamp in the message set and then storing deltas using varints in
> the messages like we do for offsets (the difference would be the usage of
> signed varints)? It seems like the deltas would be quite a bit smaller in
> the common case (potentially 0 for log append time, so we could even not
> store them at all using attributes like we do for key/value lengths). An
> alternative is using MaxTimestamp that is already present in the message
> set and computing deltas from that, but that seems more complicated. In any
> case, details aside, was this idea considered and rejected or is it worth
> exploring further?


Took me a while to remember why we didn't do this. The timestamp that is
included at the message set level is the max timestamp of all messages in
the message set as is the case in the current message format (I will update
the document to make this explicit). We could make the message timestamps
relative to the max timestamp, but that makes serialization a bit awkward
since the timestamps are not assumed to be increasing sequentially or
monotonically. Once the messages in the message set had been determined, we
would need to go back and adjust the relative timestamps.

Here's one idea. We let the timestamps in the messages be varints, but we
make their values be relative to the timestamp of the previous message,
with the timestamp of the first message being absolute. For example, if we
had timestamps 500, 501, 499, then we would write 500 for the first
message, 1 for the next, and -2 for the final message. Would that work? Let
me think a bit about it and see if there are any problems.

-Jason

On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:

> Good point Tom. We will update the KIP with the ACLs section and also the
> message format changes.
>
> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> >
> > I said this in the voting thread, but can the authors include a section
> > about new ACLs if there are going to be ACLs for TransactionalId. It's
> > mentioned in the google doc, but I think new ACLs should be in a KIP
> > directly.
> >
> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
> >>
> >> Thanks for the responses and updates to the document, Guozhang and
> Jason.
> >> They look good. One follow-up and one additional comment:
> >>
> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
> using
> >> the hardware instruction (particularly for messages larger than 65k), so
> >> I'm keen on taking advantage of the message format version bump to add
> >> support for it. I can write a separate KIP for this as it's not tied to
> >> Exactly-once, but it would be good to include the code change in the
> same
> >> PR that bumps the message format version. The benchmark and results can
> be
> >> found in the following link:
> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
> >>
> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
> >> first timestamp in the message set and then storing deltas using
> varints in
> >> the messages like we do for offsets (the difference would be the usage
> of
> >> signed varints)? It seems like the deltas would be quite a bit smaller
> in
> >> the common case (potentially 0 for log append time, so we could even not
> >> store them at all using attributes like we do for key/value lengths). An
> >> alternative is using MaxTimestamp that is already present in the message
> >> set and computing deltas from that, but that seems more complicated. In
> any
> >> case, details aside, was this idea considered and rejected or is it
> worth
> >> exploring further?
> >>
> >> Ismael
> >>
> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >>
> >>> Ismael,
> >>>
> >>> Thanks for the comments. A few responses below:
> >>>
> >>>
> >>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> introduces
> >>> a
> >>>> compatibility issue with regards to existing third-party authorizers.
> >> It
> >>>> would be good to highlight this in the migration/compatibility
> section.
> >>>
> >>>
> >>> Ack. I added a note in the migration section.
> >>>
> >>> 4. The Migration plan is relatively brief at the moment. Have we
> >>> considered
> >>>> if there's any additional work required due to KIP-97 (introduced in
> >>>> 0.10.2.0)?
> >>>
> >>>
> >>> Thanks, I added a few notes about client compatibility to the migration
> >>> section. I covered the main issues that come to mind, but let me know
> if
> >>> you think of others.
> >>>
> >>> 7. It seems like there is a bit of inconsistency when it comes to
> naming
> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> conventions.
> >>>> There are a few other examples like that and it would be good to clean
> >>> them
> >>>> up.
> >>>
> >>>
> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
> >> seems
> >>> like a compromise which satisfies no one.
> >>>
> >>>
> >>> -Jason
> >>>
> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> Ismael, thanks for your feedbacks. Replied inline.
> >>>>
> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
> wrote:
> >>>>>
> >>>>> Hi all,
> >>>>>
> >>>>> A few comments follow:
> >>>>>
> >>>>> 1. The document states "inter-broker communications will be increased
> >>> by
> >>>> M
> >>>>> * N * P round trips per sec. We need to conduct some system
> >> performance
> >>>>> test to make sure this additional inter-broker traffic would not
> >>> largely
> >>>>> impact the broker cluster". Has this testing been done? And if not,
> >> are
> >>>> we
> >>>>> planning to do it soon? It seems important to validate this sooner
> >>> rather
> >>>>> than later. This applies more generally too, it would be great to
> >>>>> understand how the new message format affects the producer with small
> >>>>> messages, for example.
> >>>>>
> >>>>>
> >>>> Yes we are conducting the perf tests with the message format changes
> in
> >>> the
> >>>> first stage; then the inter-broker communication with minimal
> >> transaction
> >>>> coordinator implementations in the second stage.
> >>>>
> >>>>
> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
> >>> introduces
> >>>> a
> >>>>> compatibility issue with regards to existing third-party authorizers.
> >>> It
> >>>>> would be good to highlight this in the migration/compatibility
> >> section.
> >>>>>
> >>>>> 3. I was happy to see that default values for the new configs have
> >> been
> >>>>> added to the document since I last checked it. It would be good to
> >>>> explain
> >>>>> the motivation for the choices.
> >>>>>
> >>>>>
> >>>> Updated doc.
> >>>>
> >>>>
> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
> >>>> considered
> >>>>> if there's any additional work required due to KIP-97 (introduced in
> >>>>> 0.10.2.0)?
> >>>>>
> >>>>> 5. transactional.id sounds good
> >>>>>
> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
> >>>>> considered mitigating the performance cost by using the more
> >> performant
> >>>>> CRC32c in the new message format version?
> >>>>>
> >>>>>
> >>>> We have not discussed about this before. But I think it should be
> >> doable
> >>> as
> >>>> long as we can include the additional conversion logic in the
> migration
> >>>> plan.
> >>>>
> >>>>
> >>>>> Nits:
> >>>>>
> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
> >>> naming
> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >>> conventions.
> >>>>> There are a few other examples like that and it would be good to
> >> clean
> >>>> them
> >>>>> up.
> >>>>>
> >>>>>
> >>>> I agree with the inconsistency issue. About the name itself though,
> >>> should
> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
> since
> >>> we
> >>>> need to capitalize "I" right?
> >>>>
> >>>>
> >>>>> 8. The document states "The first four fields of a message set in
> >> this
> >>>>> format must to be the same as the existing format because any fields
> >>>> before
> >>>>> the magic byte cannot be changed in order to provide a path for
> >>> upgrades
> >>>>> following a similar approach as was used in KIP-32". This makes
> >> things
> >>>>> easier, but it seems to me that the only strict requirement is that
> >> the
> >>>>> magic byte remains in the same offset and with the same size.
> >>>>>
> >>>>>
> >>>> I agree theoretically it is not required, but I think in practice it
> is
> >>>> actually better to make it more restrict: the three fields before
> magic
> >>>> byte are offset, length, and crc. Among them, crc needs to be before
> >>> magic
> >>>> byte if it wants to cover the magic byte fields; length would better
> be
> >>>> before the magic byte as well for pre-allocate memory to
> >> deser/decompress
> >>>> the message set, and the only field that does not matter too much to
> be
> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
> >>> offset
> >>>> for message set and some validation checks can be optimized to not
> scan
> >>>> through the whole message with this field in front of the format.
> >>>>
> >>>>
> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hello Folks,
> >>>>>>
> >>>>>> We have addressed all the comments collected so far, and would like
> >>> to
> >>>>>> propose a voting thread this Wednesday. If you have any further
> >>>> comments
> >>>>> on
> >>>>>> this KIP, please feel free to continue sending them on this thread
> >>>> before
> >>>>>> that.
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>>
> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> >> jason@confluent.io
> >>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> +1 for transactional.id.
> >>>>>>>
> >>>>>>> -Jason
> >>>>>>>
> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> >> wangguoz@gmail.com
> >>>>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> If I have to choose between app.id and
> >> transactional.instance.id
> >>> ,
> >>>>> I'd
> >>>>>>>> choose the latter.
> >>>>>>>>
> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
> >>> even
> >>>>>>> better.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> >>>> apurva@confluent.io>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
> >>> has
> >>>>>> caused
> >>>>>>>>> some
> >>>>>>>>>> confusion. We're considering the following renaming:
> >>>>>>>>>>
> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
> >>>>>>>> transactional.id
> >>>>>>>>> )
> >>>>>>>>>
> >>>>>>>>> This makes it clear that this id just needs to be set when
> >> the
> >>>>>>>> application
> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
> >>> in
> >>>>> the
> >>>>>>>>> context of how this id is used, viz. to maintain transactions
> >>>>> across
> >>>>>>>>> producer sessions.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Apurva
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> --
> >>>>>>>> -- Guozhang
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> -- Guozhang
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>
> >>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Good point Tom. We will update the KIP with the ACLs section and also the message format changes. 

> On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> 
> I said this in the voting thread, but can the authors include a section
> about new ACLs if there are going to be ACLs for TransactionalId. It's
> mentioned in the google doc, but I think new ACLs should be in a KIP
> directly.
> 
>> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> 
>> Thanks for the responses and updates to the document, Guozhang and Jason.
>> They look good. One follow-up and one additional comment:
>> 
>> 1. I did some benchmarking and CRC32C seems to be a massive win when using
>> the hardware instruction (particularly for messages larger than 65k), so
>> I'm keen on taking advantage of the message format version bump to add
>> support for it. I can write a separate KIP for this as it's not tied to
>> Exactly-once, but it would be good to include the code change in the same
>> PR that bumps the message format version. The benchmark and results can be
>> found in the following link:
>> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> 
>> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>> 
>> Ismael
>> 
>> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>> 
>>> Ismael,
>>> 
>>> Thanks for the comments. A few responses below:
>>> 
>>> 
>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> introduces
>>> a
>>>> compatibility issue with regards to existing third-party authorizers.
>> It
>>>> would be good to highlight this in the migration/compatibility section.
>>> 
>>> 
>>> Ack. I added a note in the migration section.
>>> 
>>> 4. The Migration plan is relatively brief at the moment. Have we
>>> considered
>>>> if there's any additional work required due to KIP-97 (introduced in
>>>> 0.10.2.0)?
>>> 
>>> 
>>> Thanks, I added a few notes about client compatibility to the migration
>>> section. I covered the main issues that come to mind, but let me know if
>>> you think of others.
>>> 
>>> 7. It seems like there is a bit of inconsistency when it comes to naming
>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> conventions.
>>>> There are a few other examples like that and it would be good to clean
>>> them
>>>> up.
>>> 
>>> 
>>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> seems
>>> like a compromise which satisfies no one.
>>> 
>>> 
>>> -Jason
>>> 
>>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>> 
>>>> Ismael, thanks for your feedbacks. Replied inline.
>>>> 
>>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
>>>>> 
>>>>> Hi all,
>>>>> 
>>>>> A few comments follow:
>>>>> 
>>>>> 1. The document states "inter-broker communications will be increased
>>> by
>>>> M
>>>>> * N * P round trips per sec. We need to conduct some system
>> performance
>>>>> test to make sure this additional inter-broker traffic would not
>>> largely
>>>>> impact the broker cluster". Has this testing been done? And if not,
>> are
>>>> we
>>>>> planning to do it soon? It seems important to validate this sooner
>>> rather
>>>>> than later. This applies more generally too, it would be great to
>>>>> understand how the new message format affects the producer with small
>>>>> messages, for example.
>>>>> 
>>>>> 
>>>> Yes we are conducting the perf tests with the message format changes in
>>> the
>>>> first stage; then the inter-broker communication with minimal
>> transaction
>>>> coordinator implementations in the second stage.
>>>> 
>>>> 
>>>>> 2. `ProducerAppId` is a new authorization resource type. This
>>> introduces
>>>> a
>>>>> compatibility issue with regards to existing third-party authorizers.
>>> It
>>>>> would be good to highlight this in the migration/compatibility
>> section.
>>>>> 
>>>>> 3. I was happy to see that default values for the new configs have
>> been
>>>>> added to the document since I last checked it. It would be good to
>>>> explain
>>>>> the motivation for the choices.
>>>>> 
>>>>> 
>>>> Updated doc.
>>>> 
>>>> 
>>>>> 4. The Migration plan is relatively brief at the moment. Have we
>>>> considered
>>>>> if there's any additional work required due to KIP-97 (introduced in
>>>>> 0.10.2.0)?
>>>>> 
>>>>> 5. transactional.id sounds good
>>>>> 
>>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>>>>> considered mitigating the performance cost by using the more
>> performant
>>>>> CRC32c in the new message format version?
>>>>> 
>>>>> 
>>>> We have not discussed about this before. But I think it should be
>> doable
>>> as
>>>> long as we can include the additional conversion logic in the migration
>>>> plan.
>>>> 
>>>> 
>>>>> Nits:
>>>>> 
>>>>> 7. It seems like there is a bit of inconsistency when it comes to
>>> naming
>>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>>> conventions.
>>>>> There are a few other examples like that and it would be good to
>> clean
>>>> them
>>>>> up.
>>>>> 
>>>>> 
>>>> I agree with the inconsistency issue. About the name itself though,
>>> should
>>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
>>> we
>>>> need to capitalize "I" right?
>>>> 
>>>> 
>>>>> 8. The document states "The first four fields of a message set in
>> this
>>>>> format must to be the same as the existing format because any fields
>>>> before
>>>>> the magic byte cannot be changed in order to provide a path for
>>> upgrades
>>>>> following a similar approach as was used in KIP-32". This makes
>> things
>>>>> easier, but it seems to me that the only strict requirement is that
>> the
>>>>> magic byte remains in the same offset and with the same size.
>>>>> 
>>>>> 
>>>> I agree theoretically it is not required, but I think in practice it is
>>>> actually better to make it more restrict: the three fields before magic
>>>> byte are offset, length, and crc. Among them, crc needs to be before
>>> magic
>>>> byte if it wants to cover the magic byte fields; length would better be
>>>> before the magic byte as well for pre-allocate memory to
>> deser/decompress
>>>> the message set, and the only field that does not matter too much to be
>>>> after magic byte is offset, but in KIP-98 we will use it as the base
>>> offset
>>>> for message set and some validation checks can be optimized to not scan
>>>> through the whole message with this field in front of the format.
>>>> 
>>>> 
>>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>> 
>>>>>> Hello Folks,
>>>>>> 
>>>>>> We have addressed all the comments collected so far, and would like
>>> to
>>>>>> propose a voting thread this Wednesday. If you have any further
>>>> comments
>>>>> on
>>>>>> this KIP, please feel free to continue sending them on this thread
>>>> before
>>>>>> that.
>>>>>> 
>>>>>> 
>>>>>> Guozhang
>>>>>> 
>>>>>> 
>>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> jason@confluent.io
>>>> 
>>>>>> wrote:
>>>>>> 
>>>>>>> +1 for transactional.id.
>>>>>>> 
>>>>>>> -Jason
>>>>>>> 
>>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> wangguoz@gmail.com
>>>> 
>>>>>> wrote:
>>>>>>> 
>>>>>>>> If I have to choose between app.id and
>> transactional.instance.id
>>> ,
>>>>> I'd
>>>>>>>> choose the latter.
>>>>>>>> 
>>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>>> even
>>>>>>> better.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Guozhang
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>>>> apurva@confluent.io>
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>>> has
>>>>>> caused
>>>>>>>>> some
>>>>>>>>>> confusion. We're considering the following renaming:
>>>>>>>>>> 
>>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>>>>>>>>>> 2. PID -> IPID (internal producer ID)
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>>>>>>>> transactional.id
>>>>>>>>> )
>>>>>>>>> 
>>>>>>>>> This makes it clear that this id just needs to be set when
>> the
>>>>>>>> application
>>>>>>>>> wishes to use transactions. I also think it is more intuitive
>>> in
>>>>> the
>>>>>>>>> context of how this id is used, viz. to maintain transactions
>>>>> across
>>>>>>>>> producer sessions.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> Apurva
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> --
>>>>>>>> -- Guozhang
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> -- Guozhang
>>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> --
>>>> -- Guozhang
>>>> 
>>> 
>> 

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Tom Crayford <tc...@heroku.com>.
I said this in the voting thread, but can the authors include a section
about new ACLs if there are going to be ACLs for TransactionalId. It's
mentioned in the google doc, but I think new ACLs should be in a KIP
directly.

On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Thanks for the responses and updates to the document, Guozhang and Jason.
> They look good. One follow-up and one additional comment:
>
> 1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
> first timestamp in the message set and then storing deltas using varints in
> the messages like we do for offsets (the difference would be the usage of
> signed varints)? It seems like the deltas would be quite a bit smaller in
> the common case (potentially 0 for log append time, so we could even not
> store them at all using attributes like we do for key/value lengths). An
> alternative is using MaxTimestamp that is already present in the message
> set and computing deltas from that, but that seems more complicated. In any
> case, details aside, was this idea considered and rejected or is it worth
> exploring further?
>
> Ismael
>
> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Ismael,
> >
> > Thanks for the comments. A few responses below:
> >
> >
> > > 2. `ProducerAppId` is a new authorization resource type. This
> introduces
> > a
> > > compatibility issue with regards to existing third-party authorizers.
> It
> > > would be good to highlight this in the migration/compatibility section.
> >
> >
> > Ack. I added a note in the migration section.
> >
> >  4. The Migration plan is relatively brief at the moment. Have we
> > considered
> > > if there's any additional work required due to KIP-97 (introduced in
> > > 0.10.2.0)?
> >
> >
> > Thanks, I added a few notes about client compatibility to the migration
> > section. I covered the main issues that come to mind, but let me know if
> > you think of others.
> >
> > 7. It seems like there is a bit of inconsistency when it comes to naming
> > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > and `InvalidPidMapping`. The latter two match Kafka's naming
> conventions.
> > > There are a few other examples like that and it would be good to clean
> > them
> > > up.
> >
> >
> > Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
> seems
> > like a compromise which satisfies no one.
> >
> >
> > -Jason
> >
> > On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Ismael, thanks for your feedbacks. Replied inline.
> > >
> > > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > Hi all,
> > > >
> > > > A few comments follow:
> > > >
> > > > 1. The document states "inter-broker communications will be increased
> > by
> > > M
> > > > * N * P round trips per sec. We need to conduct some system
> performance
> > > > test to make sure this additional inter-broker traffic would not
> > largely
> > > > impact the broker cluster". Has this testing been done? And if not,
> are
> > > we
> > > > planning to do it soon? It seems important to validate this sooner
> > rather
> > > > than later. This applies more generally too, it would be great to
> > > > understand how the new message format affects the producer with small
> > > > messages, for example.
> > > >
> > > >
> > > Yes we are conducting the perf tests with the message format changes in
> > the
> > > first stage; then the inter-broker communication with minimal
> transaction
> > > coordinator implementations in the second stage.
> > >
> > >
> > > > 2. `ProducerAppId` is a new authorization resource type. This
> > introduces
> > > a
> > > > compatibility issue with regards to existing third-party authorizers.
> > It
> > > > would be good to highlight this in the migration/compatibility
> section.
> > > >
> > > > 3. I was happy to see that default values for the new configs have
> been
> > > > added to the document since I last checked it. It would be good to
> > > explain
> > > > the motivation for the choices.
> > > >
> > > >
> > > Updated doc.
> > >
> > >
> > > > 4. The Migration plan is relatively brief at the moment. Have we
> > > considered
> > > > if there's any additional work required due to KIP-97 (introduced in
> > > > 0.10.2.0)?
> > > >
> > > > 5. transactional.id sounds good
> > > >
> > > > 6. Since we are keeping per message CRCs for auditing apps, have we
> > > > considered mitigating the performance cost by using the more
> performant
> > > > CRC32c in the new message format version?
> > > >
> > > >
> > > We have not discussed about this before. But I think it should be
> doable
> > as
> > > long as we can include the additional conversion logic in the migration
> > > plan.
> > >
> > >
> > > > Nits:
> > > >
> > > > 7. It seems like there is a bit of inconsistency when it comes to
> > naming
> > > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > > and `InvalidPidMapping`. The latter two match Kafka's naming
> > conventions.
> > > > There are a few other examples like that and it would be good to
> clean
> > > them
> > > > up.
> > > >
> > > >
> > > I agree with the inconsistency issue. About the name itself though,
> > should
> > > it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
> > we
> > > need to capitalize "I" right?
> > >
> > >
> > > > 8. The document states "The first four fields of a message set in
> this
> > > > format must to be the same as the existing format because any fields
> > > before
> > > > the magic byte cannot be changed in order to provide a path for
> > upgrades
> > > > following a similar approach as was used in KIP-32". This makes
> things
> > > > easier, but it seems to me that the only strict requirement is that
> the
> > > > magic byte remains in the same offset and with the same size.
> > > >
> > > >
> > > I agree theoretically it is not required, but I think in practice it is
> > > actually better to make it more restrict: the three fields before magic
> > > byte are offset, length, and crc. Among them, crc needs to be before
> > magic
> > > byte if it wants to cover the magic byte fields; length would better be
> > > before the magic byte as well for pre-allocate memory to
> deser/decompress
> > > the message set, and the only field that does not matter too much to be
> > > after magic byte is offset, but in KIP-98 we will use it as the base
> > offset
> > > for message set and some validation checks can be optimized to not scan
> > > through the whole message with this field in front of the format.
> > >
> > >
> > > > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Folks,
> > > > >
> > > > > We have addressed all the comments collected so far, and would like
> > to
> > > > > propose a voting thread this Wednesday. If you have any further
> > > comments
> > > > on
> > > > > this KIP, please feel free to continue sending them on this thread
> > > before
> > > > > that.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > > > wrote:
> > > > >
> > > > > > +1 for transactional.id.
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > If I have to choose between app.id and
> transactional.instance.id
> > ,
> > > > I'd
> > > > > > > choose the latter.
> > > > > > >
> > > > > > > Renaming transactional.instance.id to transactional.id sounds
> > even
> > > > > > better.
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> > > apurva@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > > Bumping one suggestion from Apurva above. The name "AppID"
> > has
> > > > > caused
> > > > > > > > some
> > > > > > > > > confusion. We're considering the following renaming:
> > > > > > > > >
> > > > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > > > transactional.id
> > > > > > > > )
> > > > > > > >
> > > > > > > > This makes it clear that this id just needs to be set when
> the
> > > > > > > application
> > > > > > > > wishes to use transactions. I also think it is more intuitive
> > in
> > > > the
> > > > > > > > context of how this id is used, viz. to maintain transactions
> > > > across
> > > > > > > > producer sessions.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Apurva
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks for the responses and updates to the document, Guozhang and Jason.
They look good. One follow-up and one additional comment:

1. I did some benchmarking and CRC32C seems to be a massive win when using
the hardware instruction (particularly for messages larger than 65k), so
I'm keen on taking advantage of the message format version bump to add
support for it. I can write a separate KIP for this as it's not tied to
Exactly-once, but it would be good to include the code change in the same
PR that bumps the message format version. The benchmark and results can be
found in the following link:
https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.

2. The message timestamp field is 8 bytes. Did we consider storing the
first timestamp in the message set and then storing deltas using varints in
the messages like we do for offsets (the difference would be the usage of
signed varints)? It seems like the deltas would be quite a bit smaller in
the common case (potentially 0 for log append time, so we could even not
store them at all using attributes like we do for key/value lengths). An
alternative is using MaxTimestamp that is already present in the message
set and computing deltas from that, but that seems more complicated. In any
case, details aside, was this idea considered and rejected or is it worth
exploring further?

Ismael

On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Ismael,
>
> Thanks for the comments. A few responses below:
>
>
> > 2. `ProducerAppId` is a new authorization resource type. This introduces
> a
> > compatibility issue with regards to existing third-party authorizers. It
> > would be good to highlight this in the migration/compatibility section.
>
>
> Ack. I added a note in the migration section.
>
>  4. The Migration plan is relatively brief at the moment. Have we
> considered
> > if there's any additional work required due to KIP-97 (introduced in
> > 0.10.2.0)?
>
>
> Thanks, I added a few notes about client compatibility to the migration
> section. I covered the main issues that come to mind, but let me know if
> you think of others.
>
> 7. It seems like there is a bit of inconsistency when it comes to naming
> > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> > There are a few other examples like that and it would be good to clean
> them
> > up.
>
>
> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest" seems
> like a compromise which satisfies no one.
>
>
> -Jason
>
> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Ismael, thanks for your feedbacks. Replied inline.
> >
> > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Hi all,
> > >
> > > A few comments follow:
> > >
> > > 1. The document states "inter-broker communications will be increased
> by
> > M
> > > * N * P round trips per sec. We need to conduct some system performance
> > > test to make sure this additional inter-broker traffic would not
> largely
> > > impact the broker cluster". Has this testing been done? And if not, are
> > we
> > > planning to do it soon? It seems important to validate this sooner
> rather
> > > than later. This applies more generally too, it would be great to
> > > understand how the new message format affects the producer with small
> > > messages, for example.
> > >
> > >
> > Yes we are conducting the perf tests with the message format changes in
> the
> > first stage; then the inter-broker communication with minimal transaction
> > coordinator implementations in the second stage.
> >
> >
> > > 2. `ProducerAppId` is a new authorization resource type. This
> introduces
> > a
> > > compatibility issue with regards to existing third-party authorizers.
> It
> > > would be good to highlight this in the migration/compatibility section.
> > >
> > > 3. I was happy to see that default values for the new configs have been
> > > added to the document since I last checked it. It would be good to
> > explain
> > > the motivation for the choices.
> > >
> > >
> > Updated doc.
> >
> >
> > > 4. The Migration plan is relatively brief at the moment. Have we
> > considered
> > > if there's any additional work required due to KIP-97 (introduced in
> > > 0.10.2.0)?
> > >
> > > 5. transactional.id sounds good
> > >
> > > 6. Since we are keeping per message CRCs for auditing apps, have we
> > > considered mitigating the performance cost by using the more performant
> > > CRC32c in the new message format version?
> > >
> > >
> > We have not discussed about this before. But I think it should be doable
> as
> > long as we can include the additional conversion logic in the migration
> > plan.
> >
> >
> > > Nits:
> > >
> > > 7. It seems like there is a bit of inconsistency when it comes to
> naming
> > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > and `InvalidPidMapping`. The latter two match Kafka's naming
> conventions.
> > > There are a few other examples like that and it would be good to clean
> > them
> > > up.
> > >
> > >
> > I agree with the inconsistency issue. About the name itself though,
> should
> > it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
> we
> > need to capitalize "I" right?
> >
> >
> > > 8. The document states "The first four fields of a message set in this
> > > format must to be the same as the existing format because any fields
> > before
> > > the magic byte cannot be changed in order to provide a path for
> upgrades
> > > following a similar approach as was used in KIP-32". This makes things
> > > easier, but it seems to me that the only strict requirement is that the
> > > magic byte remains in the same offset and with the same size.
> > >
> > >
> > I agree theoretically it is not required, but I think in practice it is
> > actually better to make it more restrict: the three fields before magic
> > byte are offset, length, and crc. Among them, crc needs to be before
> magic
> > byte if it wants to cover the magic byte fields; length would better be
> > before the magic byte as well for pre-allocate memory to deser/decompress
> > the message set, and the only field that does not matter too much to be
> > after magic byte is offset, but in KIP-98 we will use it as the base
> offset
> > for message set and some validation checks can be optimized to not scan
> > through the whole message with this field in front of the format.
> >
> >
> > > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > >
> > > > Hello Folks,
> > > >
> > > > We have addressed all the comments collected so far, and would like
> to
> > > > propose a voting thread this Wednesday. If you have any further
> > comments
> > > on
> > > > this KIP, please feel free to continue sending them on this thread
> > before
> > > > that.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <jason@confluent.io
> >
> > > > wrote:
> > > >
> > > > > +1 for transactional.id.
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > If I have to choose between app.id and transactional.instance.id
> ,
> > > I'd
> > > > > > choose the latter.
> > > > > >
> > > > > > Renaming transactional.instance.id to transactional.id sounds
> even
> > > > > better.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> > apurva@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > > Bumping one suggestion from Apurva above. The name "AppID"
> has
> > > > caused
> > > > > > > some
> > > > > > > > confusion. We're considering the following renaming:
> > > > > > > >
> > > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > > transactional.id
> > > > > > > )
> > > > > > >
> > > > > > > This makes it clear that this id just needs to be set when the
> > > > > > application
> > > > > > > wishes to use transactions. I also think it is more intuitive
> in
> > > the
> > > > > > > context of how this id is used, viz. to maintain transactions
> > > across
> > > > > > > producer sessions.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Apurva
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Ismael,

Thanks for the comments. A few responses below:


> 2. `ProducerAppId` is a new authorization resource type. This introduces a
> compatibility issue with regards to existing third-party authorizers. It
> would be good to highlight this in the migration/compatibility section.


Ack. I added a note in the migration section.

 4. The Migration plan is relatively brief at the moment. Have we considered
> if there's any additional work required due to KIP-97 (introduced in
> 0.10.2.0)?


Thanks, I added a few notes about client compatibility to the migration
section. I covered the main issues that come to mind, but let me know if
you think of others.

7. It seems like there is a bit of inconsistency when it comes to naming
> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> There are a few other examples like that and it would be good to clean them
> up.


Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest" seems
like a compromise which satisfies no one.


-Jason

On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Ismael, thanks for your feedbacks. Replied inline.
>
> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hi all,
> >
> > A few comments follow:
> >
> > 1. The document states "inter-broker communications will be increased by
> M
> > * N * P round trips per sec. We need to conduct some system performance
> > test to make sure this additional inter-broker traffic would not largely
> > impact the broker cluster". Has this testing been done? And if not, are
> we
> > planning to do it soon? It seems important to validate this sooner rather
> > than later. This applies more generally too, it would be great to
> > understand how the new message format affects the producer with small
> > messages, for example.
> >
> >
> Yes we are conducting the perf tests with the message format changes in the
> first stage; then the inter-broker communication with minimal transaction
> coordinator implementations in the second stage.
>
>
> > 2. `ProducerAppId` is a new authorization resource type. This introduces
> a
> > compatibility issue with regards to existing third-party authorizers. It
> > would be good to highlight this in the migration/compatibility section.
> >
> > 3. I was happy to see that default values for the new configs have been
> > added to the document since I last checked it. It would be good to
> explain
> > the motivation for the choices.
> >
> >
> Updated doc.
>
>
> > 4. The Migration plan is relatively brief at the moment. Have we
> considered
> > if there's any additional work required due to KIP-97 (introduced in
> > 0.10.2.0)?
> >
> > 5. transactional.id sounds good
> >
> > 6. Since we are keeping per message CRCs for auditing apps, have we
> > considered mitigating the performance cost by using the more performant
> > CRC32c in the new message format version?
> >
> >
> We have not discussed about this before. But I think it should be doable as
> long as we can include the additional conversion logic in the migration
> plan.
>
>
> > Nits:
> >
> > 7. It seems like there is a bit of inconsistency when it comes to naming
> > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> > There are a few other examples like that and it would be good to clean
> them
> > up.
> >
> >
> I agree with the inconsistency issue. About the name itself though, should
> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since we
> need to capitalize "I" right?
>
>
> > 8. The document states "The first four fields of a message set in this
> > format must to be the same as the existing format because any fields
> before
> > the magic byte cannot be changed in order to provide a path for upgrades
> > following a similar approach as was used in KIP-32". This makes things
> > easier, but it seems to me that the only strict requirement is that the
> > magic byte remains in the same offset and with the same size.
> >
> >
> I agree theoretically it is not required, but I think in practice it is
> actually better to make it more restrict: the three fields before magic
> byte are offset, length, and crc. Among them, crc needs to be before magic
> byte if it wants to cover the magic byte fields; length would better be
> before the magic byte as well for pre-allocate memory to deser/decompress
> the message set, and the only field that does not matter too much to be
> after magic byte is offset, but in KIP-98 we will use it as the base offset
> for message set and some validation checks can be optimized to not scan
> through the whole message with this field in front of the format.
>
>
> > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> >
> > > Hello Folks,
> > >
> > > We have addressed all the comments collected so far, and would like to
> > > propose a voting thread this Wednesday. If you have any further
> comments
> > on
> > > this KIP, please feel free to continue sending them on this thread
> before
> > > that.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > +1 for transactional.id.
> > > >
> > > > -Jason
> > > >
> > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > If I have to choose between app.id and transactional.instance.id,
> > I'd
> > > > > choose the latter.
> > > > >
> > > > > Renaming transactional.instance.id to transactional.id sounds even
> > > > better.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> apurva@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> > > caused
> > > > > > some
> > > > > > > confusion. We're considering the following renaming:
> > > > > > >
> > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > >
> > > > > >
> > > > > >
> > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > transactional.id
> > > > > > )
> > > > > >
> > > > > > This makes it clear that this id just needs to be set when the
> > > > > application
> > > > > > wishes to use transactions. I also think it is more intuitive in
> > the
> > > > > > context of how this id is used, viz. to maintain transactions
> > across
> > > > > > producer sessions.
> > > > > >
> > > > > > Thanks,
> > > > > > Apurva
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Ismael, thanks for your feedbacks. Replied inline.

On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi all,
>
> A few comments follow:
>
> 1. The document states "inter-broker communications will be increased by M
> * N * P round trips per sec. We need to conduct some system performance
> test to make sure this additional inter-broker traffic would not largely
> impact the broker cluster". Has this testing been done? And if not, are we
> planning to do it soon? It seems important to validate this sooner rather
> than later. This applies more generally too, it would be great to
> understand how the new message format affects the producer with small
> messages, for example.
>
>
Yes we are conducting the perf tests with the message format changes in the
first stage; then the inter-broker communication with minimal transaction
coordinator implementations in the second stage.


> 2. `ProducerAppId` is a new authorization resource type. This introduces a
> compatibility issue with regards to existing third-party authorizers. It
> would be good to highlight this in the migration/compatibility section.
>
> 3. I was happy to see that default values for the new configs have been
> added to the document since I last checked it. It would be good to explain
> the motivation for the choices.
>
>
Updated doc.


> 4. The Migration plan is relatively brief at the moment. Have we considered
> if there's any additional work required due to KIP-97 (introduced in
> 0.10.2.0)?
>
> 5. transactional.id sounds good
>
> 6. Since we are keeping per message CRCs for auditing apps, have we
> considered mitigating the performance cost by using the more performant
> CRC32c in the new message format version?
>
>
We have not discussed about this before. But I think it should be doable as
long as we can include the additional conversion logic in the migration
plan.


> Nits:
>
> 7. It seems like there is a bit of inconsistency when it comes to naming
> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> There are a few other examples like that and it would be good to clean them
> up.
>
>
I agree with the inconsistency issue. About the name itself though, should
it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since we
need to capitalize "I" right?


> 8. The document states "The first four fields of a message set in this
> format must to be the same as the existing format because any fields before
> the magic byte cannot be changed in order to provide a path for upgrades
> following a similar approach as was used in KIP-32". This makes things
> easier, but it seems to me that the only strict requirement is that the
> magic byte remains in the same offset and with the same size.
>
>
I agree theoretically it is not required, but I think in practice it is
actually better to make it more restrict: the three fields before magic
byte are offset, length, and crc. Among them, crc needs to be before magic
byte if it wants to cover the magic byte fields; length would better be
before the magic byte as well for pre-allocate memory to deser/decompress
the message set, and the only field that does not matter too much to be
after magic byte is offset, but in KIP-98 we will use it as the base offset
for message set and some validation checks can be optimized to not scan
through the whole message with this field in front of the format.


> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > Hello Folks,
> >
> > We have addressed all the comments collected so far, and would like to
> > propose a voting thread this Wednesday. If you have any further comments
> on
> > this KIP, please feel free to continue sending them on this thread before
> > that.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > +1 for transactional.id.
> > >
> > > -Jason
> > >
> > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > If I have to choose between app.id and transactional.instance.id,
> I'd
> > > > choose the latter.
> > > >
> > > > Renaming transactional.instance.id to transactional.id sounds even
> > > better.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > > > wrote:
> > > >
> > > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> > caused
> > > > > some
> > > > > > confusion. We're considering the following renaming:
> > > > > >
> > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > 2. PID -> IPID (internal producer ID)
> > > > > >
> > > > >
> > > > >
> > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > transactional.id
> > > > > )
> > > > >
> > > > > This makes it clear that this id just needs to be set when the
> > > > application
> > > > > wishes to use transactions. I also think it is more intuitive in
> the
> > > > > context of how this id is used, viz. to maintain transactions
> across
> > > > > producer sessions.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Hi all,

A few comments follow:

1. The document states "inter-broker communications will be increased by M
* N * P round trips per sec. We need to conduct some system performance
test to make sure this additional inter-broker traffic would not largely
impact the broker cluster". Has this testing been done? And if not, are we
planning to do it soon? It seems important to validate this sooner rather
than later. This applies more generally too, it would be great to
understand how the new message format affects the producer with small
messages, for example.

2. `ProducerAppId` is a new authorization resource type. This introduces a
compatibility issue with regards to existing third-party authorizers. It
would be good to highlight this in the migration/compatibility section.

3. I was happy to see that default values for the new configs have been
added to the document since I last checked it. It would be good to explain
the motivation for the choices.

4. The Migration plan is relatively brief at the moment. Have we considered
if there's any additional work required due to KIP-97 (introduced in
0.10.2.0)?

5. transactional.id sounds good

6. Since we are keeping per message CRCs for auditing apps, have we
considered mitigating the performance cost by using the more performant
CRC32c in the new message format version?

Nits:

7. It seems like there is a bit of inconsistency when it comes to naming
convention. For example, we have `InitPIDRequest`, `PidSnapshot`
and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
There are a few other examples like that and it would be good to clean them
up.

8. The document states "The first four fields of a message set in this
format must to be the same as the existing format because any fields before
the magic byte cannot be changed in order to provide a path for upgrades
following a similar approach as was used in KIP-32". This makes things
easier, but it seems to me that the only strict requirement is that the
magic byte remains in the same offset and with the same size.


On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hello Folks,
>
> We have addressed all the comments collected so far, and would like to
> propose a voting thread this Wednesday. If you have any further comments on
> this KIP, please feel free to continue sending them on this thread before
> that.
>
>
> Guozhang
>
>
> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > +1 for transactional.id.
> >
> > -Jason
> >
> > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > If I have to choose between app.id and transactional.instance.id, I'd
> > > choose the latter.
> > >
> > > Renaming transactional.instance.id to transactional.id sounds even
> > better.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > >
> > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> caused
> > > > some
> > > > > confusion. We're considering the following renaming:
> > > > >
> > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > 2. PID -> IPID (internal producer ID)
> > > > >
> > > >
> > > >
> > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > transactional.id
> > > > )
> > > >
> > > > This makes it clear that this id just needs to be set when the
> > > application
> > > > wishes to use transactions. I also think it is more intuitive in the
> > > > context of how this id is used, viz. to maintain transactions across
> > > > producer sessions.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Folks,

We have addressed all the comments collected so far, and would like to
propose a voting thread this Wednesday. If you have any further comments on
this KIP, please feel free to continue sending them on this thread before
that.


Guozhang


On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io> wrote:

> +1 for transactional.id.
>
> -Jason
>
> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > If I have to choose between app.id and transactional.instance.id, I'd
> > choose the latter.
> >
> > Renaming transactional.instance.id to transactional.id sounds even
> better.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > > some
> > > > confusion. We're considering the following renaming:
> > > >
> > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > 2. PID -> IPID (internal producer ID)
> > > >
> > >
> > >
> > > How about AppId -> TransactionalId (transaction.app.id ->
> > transactional.id
> > > )
> > >
> > > This makes it clear that this id just needs to be set when the
> > application
> > > wishes to use transactions. I also think it is more intuitive in the
> > > context of how this id is used, viz. to maintain transactions across
> > > producer sessions.
> > >
> > > Thanks,
> > > Apurva
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Folks,

We have addressed all the comments collected so far, and would like to
propose a voting thread this Wednesday. If you have any further comments on
this KIP, please feel free to continue sending them on this thread before
that.


Guozhang


On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io> wrote:

> +1 for transactional.id.
>
> -Jason
>
> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > If I have to choose between app.id and transactional.instance.id, I'd
> > choose the latter.
> >
> > Renaming transactional.instance.id to transactional.id sounds even
> better.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > > some
> > > > confusion. We're considering the following renaming:
> > > >
> > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > 2. PID -> IPID (internal producer ID)
> > > >
> > >
> > >
> > > How about AppId -> TransactionalId (transaction.app.id ->
> > transactional.id
> > > )
> > >
> > > This makes it clear that this id just needs to be set when the
> > application
> > > wishes to use transactions. I also think it is more intuitive in the
> > > context of how this id is used, viz. to maintain transactions across
> > > producer sessions.
> > >
> > > Thanks,
> > > Apurva
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
+1 for transactional.id.

-Jason

On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:

> If I have to choose between app.id and transactional.instance.id, I'd
> choose the latter.
>
> Renaming transactional.instance.id to transactional.id sounds even better.
>
>
> Guozhang
>
>
> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
>
> > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > some
> > > confusion. We're considering the following renaming:
> > >
> > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > 2. PID -> IPID (internal producer ID)
> > >
> >
> >
> > How about AppId -> TransactionalId (transaction.app.id ->
> transactional.id
> > )
> >
> > This makes it clear that this id just needs to be set when the
> application
> > wishes to use transactions. I also think it is more intuitive in the
> > context of how this id is used, viz. to maintain transactions across
> > producer sessions.
> >
> > Thanks,
> > Apurva
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
If I have to choose between app.id and transactional.instance.id, I'd
choose the latter.

Renaming transactional.instance.id to transactional.id sounds even better.


Guozhang


On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io> wrote:

> > Bumping one suggestion from Apurva above. The name "AppID" has caused
> some
> > confusion. We're considering the following renaming:
> >
> > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > 2. PID -> IPID (internal producer ID)
> >
>
>
> How about AppId -> TransactionalId (transaction.app.id -> transactional.id
> )
>
> This makes it clear that this id just needs to be set when the application
> wishes to use transactions. I also think it is more intuitive in the
> context of how this id is used, viz. to maintain transactions across
> producer sessions.
>
> Thanks,
> Apurva
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
> Bumping one suggestion from Apurva above. The name "AppID" has caused some
> confusion. We're considering the following renaming:
>
> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> 2. PID -> IPID (internal producer ID)
>


How about AppId -> TransactionalId (transaction.app.id -> transactional.id)

This makes it clear that this id just needs to be set when the application
wishes to use transactions. I also think it is more intuitive in the
context of how this id is used, viz. to maintain transactions across
producer sessions.

Thanks,
Apurva

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Bumping one suggestion from Apurva above. The name "AppID" has caused some
confusion. We're considering the following renaming:

1. AppID -> ProducerId (transaction.app.id -> producer.id)
2. PID -> IPID (internal producer ID)

The main drawback of using ProducerId is that it fails to express the fact
that it is tied to the use of transactions (it's actually slightly
preferable to not set it if you're not using transactions). It's a bit more
verbose, but I was also considering something like "
transactional.instance.id"

Feedback appreciated.

-Jason

On Mon, Jan 30, 2017 at 11:17 AM, Apurva Mehta <ap...@confluent.io> wrote:

> >   102. When there is a correlated hard failure (e.g., power outage), it's
> >>   possible that an existing commit/abort marker is lost in all replicas.
> >>  This  may not be fixed by the transaction coordinator automatically and
> >> the
> >>   consumer may get stuck on that incomplete transaction forever. Not
> sure
> >>   what's the best way to address this. Perhaps, one way is to run a tool
> >> to
> >>   add an abort maker for all pids in all affected partitions.
> >
> >
>
> There can be two types of tools, one for diagnosing the issue and another
> > for fixing the issue. I think having at least a diagnostic tool in the
> > first version could be helpful. For example, the tool can report things
> > like which producer id is preventing the LSO from being advanced. That
> way,
> > at least the users can try to fix this themselves.
> >
>
>
> That sounds reasonable. Will add a work item to track this so that such a
> tool is available in the first version.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
>   102. When there is a correlated hard failure (e.g., power outage), it's
>>   possible that an existing commit/abort marker is lost in all replicas.
>>  This  may not be fixed by the transaction coordinator automatically and
>> the
>>   consumer may get stuck on that incomplete transaction forever. Not sure
>>   what's the best way to address this. Perhaps, one way is to run a tool
>> to
>>   add an abort maker for all pids in all affected partitions.
>
>

There can be two types of tools, one for diagnosing the issue and another
> for fixing the issue. I think having at least a diagnostic tool in the
> first version could be helpful. For example, the tool can report things
> like which producer id is preventing the LSO from being advanced. That way,
> at least the users can try to fix this themselves.
>


That sounds reasonable. Will add a work item to track this so that such a
tool is available in the first version.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jun Rao <ju...@confluent.io>.
Hi, Apurva,

Thanks for the update. My replies are inlined below.

On Wed, Jan 25, 2017 at 5:15 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Jun,
>
> Thanks again for the comments. More responses follow:
>
>
> > 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> to
> > the new version, but the broker message format is still the old one. If a
> > new producer uses the transaction feature, should the producer get an
> error
> > in this case? A tricky case can be that the leader broker is on the new
> > message format, but the follower broker is still on the old message
> format.
> > In this case, the transactional info will be lost in the follower due to
> > down conversion. Should we failed the transactional requests when the
> > followers are still on the old message format?
> >
>
> This will only be an issue if applications are written to use transactions
> and are deployed with the new client before all the brokers are upgraded to
> the new message format.
>
> There are a variety of engineering solutions to this problem, one of which
> is for each broker to register itself as 'transaction ready' in zookeeper
> when it is on the right version of the message format. Once the controller
> detects that all brokers in the cluster are transaction ready, it will
> signal to each broker via the UpdateMetadataRequest that the cluster is
> ready for transactions. Any transactional requests received by brokers
> before this point will be rejected.
>
> A simpler way to solve this problem is through organizational policy: a
> cluster should not be advertised to application developers as 'transaction
> ready' until all brokers are on the new message format.
>
> I think the non-engineering solution is reasonable, and as such would
> prefer to not include engineering solutions in V1. It could be future work
> if necessary.
>
> We can make the problems that arise out of premature use of transactions
> clear in the release notes so that operators can take the necessary
> precautions. Is that reasonable?
>
>
> Yes, in the first version, we can just document the impact in the upgrade
doc.



>
>
> > 102. When there is a correlated hard failure (e.g., power outage), it's
> > possible that an existing commit/abort marker is lost in all replicas.
> This
> > may not be fixed by the transaction coordinator automatically and the
> > consumer may get stuck on that incomplete transaction forever. Not sure
> > what's the best way to address this. Perhaps, one way is to run a tool to
> > add an abort maker for all pids in all affected partitions.
> >
> >
> This is a good point. With the abort index proposal, if a correlated hard
> failure causes us to lose the markers everywhere, the LSO on the broker
> would not advance and consumers would block (but not buffer). This would be
> a noticeable situation.
>
> A simple tool may make use of internal functions to effectively do a
> 'initPID', 'beginTransaction', 'AddTopicPartitiontoTransaction',
> 'commitTransaction'. This would ensure that the markers are rewritten to
> all partitions by the transaction coordinator, but would also fence the
> existing producer with the same AppId.
>
> To make this workable, we need to make sure that the transaction
> coordinator adds a sufficient logging so that we know the AppID -> PID
> mapping as well as the partitions participating in each transaction. The
> broker should also probably log information so that we know which
> unfinished transaction (ie. which PID) is preventing the LSO from moving
> forward. Both these things will make it fairly easy to configure the tool.
>
> Of course, it is possible for the producer to continue onto another
> transaction before the tool is run, in which case the data will be corrupt
> since the second transaction will include messages from the first. But this
> is no worse than Kafka's existing durability semantics which this proposal
> relies on.
>
> I think such a tool can be a follow up work, and I have added it to the
> 'future work' section of the document.
>
>
> There can be two types of tools, one for diagnosing the issue and another
for fixing the issue. I think having at least a diagnostic tool in the
first version could be helpful. For example, the tool can report things
like which producer id is preventing the LSO from being advanced. That way,
at least the users can try to fix this themselves.


>
> > 103. Currently, there is no check for producer liveness. This means that
> if
> > a producer has not been sending transactional requests for a long time,
> its
> > appId will be expired by the coordinator. Have we considered having
> > producers sending heartbeatRequest just like the consumer to keep it
> alive?
> >
> >
> In the current proposal, a producer whose AppId has expired is a Zombie
> which will get a Fatal 'ProducerFencedException' when it tries to make any
> new transactional requests. A bounce of the producer will reinitialize it,
> at which point it can continue.
>
> As such, while the proposed behavior is not ideal, I think that a heartbeat
> thread would be a nice to have that may not be worth putting into V1. I
> have made a note to add this in the 'Future Work' section of the document.
>
>
>
Yes, as long as we have a path to add heartbeat in the future, this is fine.


>
> > 104. The logic for handling follower truncation can be a bit tricker now.
> > The truncation may rewind the sequence number for some pids. The question
> > is how to quickly recover the last sequence number of those pids. Do we
> > plan to reload from a PID snapshot and scan forward?
> >
> >
> Yes, this is exactly what we intend to do.
>
>
> > 115. Message format:
> > 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> > since it's possible for a producer to be restarted 10s thousands of
> times?
> > 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> > producer to publish more than 2 billion messages in a session.
> >
>
> Yes, both the epoch and the sequence number will have a circular space and
> wrap around.
>
> In the pure idempotent producer case, ie. where there is no AppId, each
> producer session is guaranteed to have a unique PID and the leader for the
> partition validates the incoming sequence number per PID before committing
> to the log. So wrapping is safe.
>
> In the transactional case, the transaction coordinator assigns the epoch
> and returns the current epoch as part of the InitPIDRequest. All
> transaction requests (Begin, AddTPToTransaction, Prepare) will be accepted
> only if their epoch exactly matches the epoch recorded by the coordinator.
> Wrap around is safe except in some degenerate cases like very long lived
> zombie producers, described below.
>
> Say we have a producer at epoch 20, who suddenly becomes a zombie. Assume
> other producers with the same id are bounced for 65536 sessions (since
> epoch are 2 bytes). Now after this period, the zombie producer comes back
> online and would have the same PID/epoch as the current producer. In this
> case, it is possible for both to produce transactions and for this
> situation to go undetected.
>
> We can solve this problem by including 'session ids' for producers and
> validating that as well. But since the degenerate case is so rare, we think
> that any such session id can be part of future work if it becomes a real
> problem.
>
> I have updated the doc to explicitly mention that the epoch and sequence
> numbers may wrap.
>
>
>
Sounds good.


>
> > 119. InitPIDRequest
> > <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8/edit#heading=h.z99xar1h2enr>
> > : Should we write the completion of open transactions before append the
> pid
> > with bumped up epoch to the transaction log?
> >
> >
> Done.
>
>
> > 120. transaction.app.id: An app may have multiple concurrent instances.
> > Perhaps we should name it transaction.instance.id or just instance.id?
> >
> >
> This has been the most debated point. How about just `producer.id`?
>
>
The only thing is that this will be set in the producer config so the
"producer" part seems redundant.


>
>
> > 121. The ordering is important with idempotent producer, which means
> > that max.in.flight.requests.per.connection should be set to 1. Do we
> want
> > to enforce this?
> >
>
> This makes sense. Doc has been updated.
>
>
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com>
> wrote:
> >
> > > @jun - good proposal. i was willing to concede that read-uncommitted
> was
> > > impossible under my proposal but if LSO/NSO is introduced is becomes
> > > possible.
> > >
> > >
> > > On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Just to follow up on Radai's idea of pushing the buffering logic to
> the
> > > > broker. It may be possible to do this efficiently if we assume
> aborted
> > > > transactions are rare. The following is a draft proposal. For each
> > > > partition, the broker maintains the last stable offset (LSO) as
> > described
> > > > in the document, and only exposes messages up to this point if the
> > reader
> > > > is in the read-committed mode. When a new stable offset (NSO) is
> > > > determined, if there is no aborted message in this window, the broker
> > > > simply advances the LSO to the NSO. If there is at least one aborted
> > > > message, the broker first replaces the current log segment with new
> log
> > > > segments excluding the aborted messages and then advances the LSO. To
> > > make
> > > > the replacement efficient, we can replace the current log segment
> with
> > 3
> > > > new segments: (1) a new "shadow" log segment that simply references
> the
> > > > portion of the current log segment from the beginning to the LSO,
> (2) a
> > > log
> > > > segment created by copying only committed messages between the LSO
> and
> > > the
> > > > NSO, (3) a new "shadow" log segment that references the portion of
> the
> > > > current log segment from the NSO (open ended). Note that only (2)
> > > involves
> > > > real data copying. If aborted transactions are rare, this overhead
> will
> > > be
> > > > insignificant. Assuming that applications typically don't abort
> > > > transactions, transactions will only be aborted by transaction
> > > coordinators
> > > > during hard failure of the producers, which should be rare.
> > > >
> > > > This way, the consumer library's logic will be simplified. We can
> still
> > > > expose uncommitted messages to readers in the read-uncommitted mode
> and
> > > > therefore leave the door open for speculative reader in the future.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Joel,
> > > > >
> > > > > The alternatives are embedded in the 'discussion' sections which
> are
> > > > spread
> > > > > throughout the google doc.
> > > > >
> > > > > Admittedly, we have not covered high level alternatives like those
> > > which
> > > > > have been brought up in this thread. In particular, having a
> separate
> > > log
> > > > > for transactional mesages and also having multiple producers
> > > participate
> > > > in
> > > > > a single transaction.
> > > > >
> > > > > This is an omission which we will correct.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > >
> > > > > > >
> > > > > > > @Joel,
> > > > > > >
> > > > > > > I read over your wiki, and apart from the introduction of the
> > > notion
> > > > of
> > > > > > > journal partitions --whose pros and cons are already being
> > > > discussed--
> > > > > > you
> > > > > > > also introduce the notion of a 'producer group' which enables
> > > > multiple
> > > > > > > producers to participate in a single transaction. This is
> > > completely
> > > > > > > opposite of the model in the KIP where a transaction is defined
> > by
> > > a
> > > > > > > producer id, and hence there is a 1-1 mapping between producers
> > and
> > > > > > > transactions. Further, each producer can have exactly one
> > in-flight
> > > > > > > transaction at a time in the KIP.
> > > > > > >
> > > > > >
> > > > > > Hi Apurva - yes I did notice those differences among other things
> > :)
> > > > > BTW, I
> > > > > > haven't yet gone through the google-doc carefully but on a skim
> it
> > > does
> > > > > not
> > > > > > seem to contain any rejected alternatives as the wiki states.
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Jun,

Thanks again for the comments. More responses follow:


> 101. Compatibility during upgrade: Suppose that the brokers are upgraded to
> the new version, but the broker message format is still the old one. If a
> new producer uses the transaction feature, should the producer get an error
> in this case? A tricky case can be that the leader broker is on the new
> message format, but the follower broker is still on the old message format.
> In this case, the transactional info will be lost in the follower due to
> down conversion. Should we failed the transactional requests when the
> followers are still on the old message format?
>

This will only be an issue if applications are written to use transactions
and are deployed with the new client before all the brokers are upgraded to
the new message format.

There are a variety of engineering solutions to this problem, one of which
is for each broker to register itself as 'transaction ready' in zookeeper
when it is on the right version of the message format. Once the controller
detects that all brokers in the cluster are transaction ready, it will
signal to each broker via the UpdateMetadataRequest that the cluster is
ready for transactions. Any transactional requests received by brokers
before this point will be rejected.

A simpler way to solve this problem is through organizational policy: a
cluster should not be advertised to application developers as 'transaction
ready' until all brokers are on the new message format.

I think the non-engineering solution is reasonable, and as such would
prefer to not include engineering solutions in V1. It could be future work
if necessary.

We can make the problems that arise out of premature use of transactions
clear in the release notes so that operators can take the necessary
precautions. Is that reasonable?




> 102. When there is a correlated hard failure (e.g., power outage), it's
> possible that an existing commit/abort marker is lost in all replicas. This
> may not be fixed by the transaction coordinator automatically and the
> consumer may get stuck on that incomplete transaction forever. Not sure
> what's the best way to address this. Perhaps, one way is to run a tool to
> add an abort maker for all pids in all affected partitions.
>
>
This is a good point. With the abort index proposal, if a correlated hard
failure causes us to lose the markers everywhere, the LSO on the broker
would not advance and consumers would block (but not buffer). This would be
a noticeable situation.

A simple tool may make use of internal functions to effectively do a
'initPID', 'beginTransaction', 'AddTopicPartitiontoTransaction',
'commitTransaction'. This would ensure that the markers are rewritten to
all partitions by the transaction coordinator, but would also fence the
existing producer with the same AppId.

To make this workable, we need to make sure that the transaction
coordinator adds a sufficient logging so that we know the AppID -> PID
mapping as well as the partitions participating in each transaction. The
broker should also probably log information so that we know which
unfinished transaction (ie. which PID) is preventing the LSO from moving
forward. Both these things will make it fairly easy to configure the tool.

Of course, it is possible for the producer to continue onto another
transaction before the tool is run, in which case the data will be corrupt
since the second transaction will include messages from the first. But this
is no worse than Kafka's existing durability semantics which this proposal
relies on.

I think such a tool can be a follow up work, and I have added it to the
'future work' section of the document.



> 103. Currently, there is no check for producer liveness. This means that if
> a producer has not been sending transactional requests for a long time, its
> appId will be expired by the coordinator. Have we considered having
> producers sending heartbeatRequest just like the consumer to keep it alive?
>
>
In the current proposal, a producer whose AppId has expired is a Zombie
which will get a Fatal 'ProducerFencedException' when it tries to make any
new transactional requests. A bounce of the producer will reinitialize it,
at which point it can continue.

As such, while the proposed behavior is not ideal, I think that a heartbeat
thread would be a nice to have that may not be worth putting into V1. I
have made a note to add this in the 'Future Work' section of the document.



> 104. The logic for handling follower truncation can be a bit tricker now.
> The truncation may rewind the sequence number for some pids. The question
> is how to quickly recover the last sequence number of those pids. Do we
> plan to reload from a PID snapshot and scan forward?
>
>
Yes, this is exactly what we intend to do.


> 115. Message format:
> 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> since it's possible for a producer to be restarted 10s thousands of times?
> 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> producer to publish more than 2 billion messages in a session.
>

Yes, both the epoch and the sequence number will have a circular space and
wrap around.

In the pure idempotent producer case, ie. where there is no AppId, each
producer session is guaranteed to have a unique PID and the leader for the
partition validates the incoming sequence number per PID before committing
to the log. So wrapping is safe.

In the transactional case, the transaction coordinator assigns the epoch
and returns the current epoch as part of the InitPIDRequest. All
transaction requests (Begin, AddTPToTransaction, Prepare) will be accepted
only if their epoch exactly matches the epoch recorded by the coordinator.
Wrap around is safe except in some degenerate cases like very long lived
zombie producers, described below.

Say we have a producer at epoch 20, who suddenly becomes a zombie. Assume
other producers with the same id are bounced for 65536 sessions (since
epoch are 2 bytes). Now after this period, the zombie producer comes back
online and would have the same PID/epoch as the current producer. In this
case, it is possible for both to produce transactions and for this
situation to go undetected.

We can solve this problem by including 'session ids' for producers and
validating that as well. But since the degenerate case is so rare, we think
that any such session id can be part of future work if it becomes a real
problem.

I have updated the doc to explicitly mention that the epoch and sequence
numbers may wrap.



> 119. InitPIDRequest
> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#heading=h.z99xar1h2enr>
> : Should we write the completion of open transactions before append the pid
> with bumped up epoch to the transaction log?
>
>
Done.


> 120. transaction.app.id: An app may have multiple concurrent instances.
> Perhaps we should name it transaction.instance.id or just instance.id?
>
>
This has been the most debated point. How about just `producer.id`?



> 121. The ordering is important with idempotent producer, which means
> that max.in.flight.requests.per.connection should be set to 1. Do we want
> to enforce this?
>

This makes sense. Doc has been updated.


>
> Thanks,
>
> Jun
>
>
> On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com> wrote:
>
> > @jun - good proposal. i was willing to concede that read-uncommitted was
> > impossible under my proposal but if LSO/NSO is introduced is becomes
> > possible.
> >
> >
> > On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Just to follow up on Radai's idea of pushing the buffering logic to the
> > > broker. It may be possible to do this efficiently if we assume aborted
> > > transactions are rare. The following is a draft proposal. For each
> > > partition, the broker maintains the last stable offset (LSO) as
> described
> > > in the document, and only exposes messages up to this point if the
> reader
> > > is in the read-committed mode. When a new stable offset (NSO) is
> > > determined, if there is no aborted message in this window, the broker
> > > simply advances the LSO to the NSO. If there is at least one aborted
> > > message, the broker first replaces the current log segment with new log
> > > segments excluding the aborted messages and then advances the LSO. To
> > make
> > > the replacement efficient, we can replace the current log segment with
> 3
> > > new segments: (1) a new "shadow" log segment that simply references the
> > > portion of the current log segment from the beginning to the LSO, (2) a
> > log
> > > segment created by copying only committed messages between the LSO and
> > the
> > > NSO, (3) a new "shadow" log segment that references the portion of the
> > > current log segment from the NSO (open ended). Note that only (2)
> > involves
> > > real data copying. If aborted transactions are rare, this overhead will
> > be
> > > insignificant. Assuming that applications typically don't abort
> > > transactions, transactions will only be aborted by transaction
> > coordinators
> > > during hard failure of the producers, which should be rare.
> > >
> > > This way, the consumer library's logic will be simplified. We can still
> > > expose uncommitted messages to readers in the read-uncommitted mode and
> > > therefore leave the door open for speculative reader in the future.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > >
> > > > Hi Joel,
> > > >
> > > > The alternatives are embedded in the 'discussion' sections which are
> > > spread
> > > > throughout the google doc.
> > > >
> > > > Admittedly, we have not covered high level alternatives like those
> > which
> > > > have been brought up in this thread. In particular, having a separate
> > log
> > > > for transactional mesages and also having multiple producers
> > participate
> > > in
> > > > a single transaction.
> > > >
> > > > This is an omission which we will correct.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > >
> > > > > >
> > > > > >
> > > > > > @Joel,
> > > > > >
> > > > > > I read over your wiki, and apart from the introduction of the
> > notion
> > > of
> > > > > > journal partitions --whose pros and cons are already being
> > > discussed--
> > > > > you
> > > > > > also introduce the notion of a 'producer group' which enables
> > > multiple
> > > > > > producers to participate in a single transaction. This is
> > completely
> > > > > > opposite of the model in the KIP where a transaction is defined
> by
> > a
> > > > > > producer id, and hence there is a 1-1 mapping between producers
> and
> > > > > > transactions. Further, each producer can have exactly one
> in-flight
> > > > > > transaction at a time in the KIP.
> > > > > >
> > > > >
> > > > > Hi Apurva - yes I did notice those differences among other things
> :)
> > > > BTW, I
> > > > > haven't yet gone through the google-doc carefully but on a skim it
> > does
> > > > not
> > > > > seem to contain any rejected alternatives as the wiki states.
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jun Rao <ju...@confluent.io>.
Thanks for the proposal. A few more detailed comments.

100. Security: It seems that if an app is mistakenly configured with the
appId of an existing producer, it can take over the pid and prevent the
existing app from publishing. So, I am wondering if it makes sense to add
ACLs at the TransactionResource level just like we do for
ConsumerGroupResource. So, a user can only do transactions under a
particular appId if he/she has the write permission to the TransactionResource
associated with the appId.

101. Compatibility during upgrade: Suppose that the brokers are upgraded to
the new version, but the broker message format is still the old one. If a
new producer uses the transaction feature, should the producer get an error
in this case? A tricky case can be that the leader broker is on the new
message format, but the follower broker is still on the old message format.
In this case, the transactional info will be lost in the follower due to
down conversion. Should we failed the transactional requests when the
followers are still on the old message format?

102. When there is a correlated hard failure (e.g., power outage), it's
possible that an existing commit/abort marker is lost in all replicas. This
may not be fixed by the transaction coordinator automatically and the
consumer may get stuck on that incomplete transaction forever. Not sure
what's the best way to address this. Perhaps, one way is to run a tool to
add an abort maker for all pids in all affected partitions.

103. Currently, there is no check for producer liveness. This means that if
a producer has not been sending transactional requests for a long time, its
appId will be expired by the coordinator. Have we considered having
producers sending heartbeatRequest just like the consumer to keep it alive?

104. The logic for handling follower truncation can be a bit tricker now.
The truncation may rewind the sequence number for some pids. The question
is how to quickly recover the last sequence number of those pids. Do we
plan to reload from a PID snapshot and scan forward?

105. When the transaction coordinator changes (due to leadership changes),
it's possible for both the old and the new coordinator sending requests to
a broker at the same time (for a short period of time). I am wondering if
we need to add logic to fence off the old coordinator. One way to do that
is to include the leaderEpoch of the partition associated with the
coordinator in the coordinator to broker request and control messages.

106. Compacted topics.
106.1. When all messages in a transaction are removed, we could remove the
commit/abort marker for that transaction too. However, we have to be a bit
careful. If the marker is removed too quickly, it's possible for a consumer
to see a message in that transaction, but not to see the marker, and
therefore will be stuck in that transaction forever. We have a similar
issue when dealing with tombstones. The solution is to preserve the
tombstone for at least a preconfigured amount of time after the cleaning
has passed the tombstone. Then, as long as a consumer can finish reading to
the cleaning point within the configured amount of time, it's guaranteed
not to miss the tombstone after it has seen a non-tombstone message on the
same key. I am wondering if we should do something similar here.
106.2. "To address this problem, we propose to preserve the last epoch and
sequence number written by each producer for a fixed amount of time as an
empty message set. This is allowed by the new message format we are
proposing in this document. The time to preserve the sequence number will
be governed by the log retention settings. " Could you be a bit more
specific on what retention time will be used since by default, there is no
retention time for compacted (but not delete) topic?
106.3 "As for control messages, if the broker does not have any
corresponding transaction cached with the PID when encountering a control
message, that message can be safely removed."
Do controlled messages have keys? If not, do we need to relax the
constraint that messages in a compacted topic must have keys?

107. Could you include the default values for the newly introduced configs?

108. Could you describe the format of the PID snapshot file?

109. Could you describe when Producer.send() will receive an UnrecognizedM
essageException?

110. Transaction log:
110.1 "Key => Version AppID Version" It seems that Version should really be
Type?
110.2 "Value => Version Epoch Status ExpirationTime [Topic Partition]"
Should we store [Topic [Partition]] instead?
110.3 To expire an AppId, do we need to insert a tombstone with the expired
AppID as the key to physically remove the existing AppID entries in the
transaction log?

111. Transaction coordinator startup: "Verify that there is already an
entry with the PID in the AppID map. If there is not, raise an exception."
For completed transactions, it seems that it's possible that their
AppId->pid has been compacted out. But that shouldn't trigger an exception?

112. Control message: Will control messages be used for timestamp indexing?
If so, what timestamp will we use if the timestamp type is creation time?

113. Zombie producer:
"If the zombie has an ongoing transaction with its old PID while its AppID
is being expired by the coordinator, by the time the zombie is about to
commit the transaction it needs to talk to coordinator again and will be
notified its PID is unrecognized and hence need to re-register its AppID
with the InitPIDRequest. At this time, if there is already another
registered producer with the same AppID, then this request will be rejected
with the fatal ProducerFenced error code."
Is that right? According the the coordinator request handling logic, it
seems that the InitPIDRequest will bump up the epoch of the pid and succeed?

114.The section on Discussion on Pro-active Transaction Timeout: "If there
is no other instance with the same PID yet, or it has not started a
transaction, or it has not appended a message to some of the partitions,
then the zombie can continue appending messages to those partitions after
the abort marker whose epoch has not been incremented yet, but its commitTxn
call will fail."
Is that correct, in earlier discussion, it seems that if a transaction is
timed out by the coordinator, the coordinator will bump up epoch and write
the abort marker to those inserted partitions.

115. Message format:
115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
since it's possible for a producer to be restarted 10s thousands of times?
115.2 Sequence number int32. Does it wrap after max? It's possible for a
producer to publish more than 2 billion messages in a session.
115.3 "Null-value bit is 1: skip the key-length (since it can now be
calculated) and value fields." It seems that it's unnatural for the format
of key to depend on value. It seems it's easier to just skip value in this
case?

116. ProducerRequest: The existing format doesn't have "MessageSetSize" at
the partition level.

117. UpdateTxnRequest: Could you explain the format of Marker?

118. TxnOffsetCommitRequest: How is retention time determined? Do we need a
new config in producer or just default it to -1 as the consumer?

119. InitPIDRequest
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.z99xar1h2enr>
: Should we write the completion of open transactions before append the pid
with bumped up epoch to the transaction log?

120. transaction.app.id: An app may have multiple concurrent instances.
Perhaps we should name it transaction.instance.id or just instance.id?

121. The ordering is important with idempotent producer, which means
that max.in.flight.requests.per.connection should be set to 1. Do we want
to enforce this?

Thanks,

Jun


On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com> wrote:

> @jun - good proposal. i was willing to concede that read-uncommitted was
> impossible under my proposal but if LSO/NSO is introduced is becomes
> possible.
>
>
> On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Just to follow up on Radai's idea of pushing the buffering logic to the
> > broker. It may be possible to do this efficiently if we assume aborted
> > transactions are rare. The following is a draft proposal. For each
> > partition, the broker maintains the last stable offset (LSO) as described
> > in the document, and only exposes messages up to this point if the reader
> > is in the read-committed mode. When a new stable offset (NSO) is
> > determined, if there is no aborted message in this window, the broker
> > simply advances the LSO to the NSO. If there is at least one aborted
> > message, the broker first replaces the current log segment with new log
> > segments excluding the aborted messages and then advances the LSO. To
> make
> > the replacement efficient, we can replace the current log segment with 3
> > new segments: (1) a new "shadow" log segment that simply references the
> > portion of the current log segment from the beginning to the LSO, (2) a
> log
> > segment created by copying only committed messages between the LSO and
> the
> > NSO, (3) a new "shadow" log segment that references the portion of the
> > current log segment from the NSO (open ended). Note that only (2)
> involves
> > real data copying. If aborted transactions are rare, this overhead will
> be
> > insignificant. Assuming that applications typically don't abort
> > transactions, transactions will only be aborted by transaction
> coordinators
> > during hard failure of the producers, which should be rare.
> >
> > This way, the consumer library's logic will be simplified. We can still
> > expose uncommitted messages to readers in the read-uncommitted mode and
> > therefore leave the door open for speculative reader in the future.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > Hi Joel,
> > >
> > > The alternatives are embedded in the 'discussion' sections which are
> > spread
> > > throughout the google doc.
> > >
> > > Admittedly, we have not covered high level alternatives like those
> which
> > > have been brought up in this thread. In particular, having a separate
> log
> > > for transactional mesages and also having multiple producers
> participate
> > in
> > > a single transaction.
> > >
> > > This is an omission which we will correct.
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > >
> > > > >
> > > > >
> > > > > @Joel,
> > > > >
> > > > > I read over your wiki, and apart from the introduction of the
> notion
> > of
> > > > > journal partitions --whose pros and cons are already being
> > discussed--
> > > > you
> > > > > also introduce the notion of a 'producer group' which enables
> > multiple
> > > > > producers to participate in a single transaction. This is
> completely
> > > > > opposite of the model in the KIP where a transaction is defined by
> a
> > > > > producer id, and hence there is a 1-1 mapping between producers and
> > > > > transactions. Further, each producer can have exactly one in-flight
> > > > > transaction at a time in the KIP.
> > > > >
> > > >
> > > > Hi Apurva - yes I did notice those differences among other things :)
> > > BTW, I
> > > > haven't yet gone through the google-doc carefully but on a skim it
> does
> > > not
> > > > seem to contain any rejected alternatives as the wiki states.
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
@jun - good proposal. i was willing to concede that read-uncommitted was
impossible under my proposal but if LSO/NSO is introduced is becomes
possible.


On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao <ju...@confluent.io> wrote:

> Just to follow up on Radai's idea of pushing the buffering logic to the
> broker. It may be possible to do this efficiently if we assume aborted
> transactions are rare. The following is a draft proposal. For each
> partition, the broker maintains the last stable offset (LSO) as described
> in the document, and only exposes messages up to this point if the reader
> is in the read-committed mode. When a new stable offset (NSO) is
> determined, if there is no aborted message in this window, the broker
> simply advances the LSO to the NSO. If there is at least one aborted
> message, the broker first replaces the current log segment with new log
> segments excluding the aborted messages and then advances the LSO. To make
> the replacement efficient, we can replace the current log segment with 3
> new segments: (1) a new "shadow" log segment that simply references the
> portion of the current log segment from the beginning to the LSO, (2) a log
> segment created by copying only committed messages between the LSO and the
> NSO, (3) a new "shadow" log segment that references the portion of the
> current log segment from the NSO (open ended). Note that only (2) involves
> real data copying. If aborted transactions are rare, this overhead will be
> insignificant. Assuming that applications typically don't abort
> transactions, transactions will only be aborted by transaction coordinators
> during hard failure of the producers, which should be rare.
>
> This way, the consumer library's logic will be simplified. We can still
> expose uncommitted messages to readers in the read-uncommitted mode and
> therefore leave the door open for speculative reader in the future.
>
> Thanks,
>
> Jun
>
>
> On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
>
> > Hi Joel,
> >
> > The alternatives are embedded in the 'discussion' sections which are
> spread
> > throughout the google doc.
> >
> > Admittedly, we have not covered high level alternatives like those which
> > have been brought up in this thread. In particular, having a separate log
> > for transactional mesages and also having multiple producers participate
> in
> > a single transaction.
> >
> > This is an omission which we will correct.
> >
> > Thanks,
> > Apurva
> >
> > On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> >
> > > >
> > > >
> > > > @Joel,
> > > >
> > > > I read over your wiki, and apart from the introduction of the notion
> of
> > > > journal partitions --whose pros and cons are already being
> discussed--
> > > you
> > > > also introduce the notion of a 'producer group' which enables
> multiple
> > > > producers to participate in a single transaction. This is completely
> > > > opposite of the model in the KIP where a transaction is defined by a
> > > > producer id, and hence there is a 1-1 mapping between producers and
> > > > transactions. Further, each producer can have exactly one in-flight
> > > > transaction at a time in the KIP.
> > > >
> > >
> > > Hi Apurva - yes I did notice those differences among other things :)
> > BTW, I
> > > haven't yet gone through the google-doc carefully but on a skim it does
> > not
> > > seem to contain any rejected alternatives as the wiki states.
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jun Rao <ju...@confluent.io>.
Just to follow up on Radai's idea of pushing the buffering logic to the
broker. It may be possible to do this efficiently if we assume aborted
transactions are rare. The following is a draft proposal. For each
partition, the broker maintains the last stable offset (LSO) as described
in the document, and only exposes messages up to this point if the reader
is in the read-committed mode. When a new stable offset (NSO) is
determined, if there is no aborted message in this window, the broker
simply advances the LSO to the NSO. If there is at least one aborted
message, the broker first replaces the current log segment with new log
segments excluding the aborted messages and then advances the LSO. To make
the replacement efficient, we can replace the current log segment with 3
new segments: (1) a new "shadow" log segment that simply references the
portion of the current log segment from the beginning to the LSO, (2) a log
segment created by copying only committed messages between the LSO and the
NSO, (3) a new "shadow" log segment that references the portion of the
current log segment from the NSO (open ended). Note that only (2) involves
real data copying. If aborted transactions are rare, this overhead will be
insignificant. Assuming that applications typically don't abort
transactions, transactions will only be aborted by transaction coordinators
during hard failure of the producers, which should be rare.

This way, the consumer library's logic will be simplified. We can still
expose uncommitted messages to readers in the read-uncommitted mode and
therefore leave the door open for speculative reader in the future.

Thanks,

Jun


On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Joel,
>
> The alternatives are embedded in the 'discussion' sections which are spread
> throughout the google doc.
>
> Admittedly, we have not covered high level alternatives like those which
> have been brought up in this thread. In particular, having a separate log
> for transactional mesages and also having multiple producers participate in
> a single transaction.
>
> This is an omission which we will correct.
>
> Thanks,
> Apurva
>
> On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > >
> > >
> > > @Joel,
> > >
> > > I read over your wiki, and apart from the introduction of the notion of
> > > journal partitions --whose pros and cons are already being discussed--
> > you
> > > also introduce the notion of a 'producer group' which enables multiple
> > > producers to participate in a single transaction. This is completely
> > > opposite of the model in the KIP where a transaction is defined by a
> > > producer id, and hence there is a 1-1 mapping between producers and
> > > transactions. Further, each producer can have exactly one in-flight
> > > transaction at a time in the KIP.
> > >
> >
> > Hi Apurva - yes I did notice those differences among other things :)
> BTW, I
> > haven't yet gone through the google-doc carefully but on a skim it does
> not
> > seem to contain any rejected alternatives as the wiki states.
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Joel,

The alternatives are embedded in the 'discussion' sections which are spread
throughout the google doc.

Admittedly, we have not covered high level alternatives like those which
have been brought up in this thread. In particular, having a separate log
for transactional mesages and also having multiple producers participate in
a single transaction.

This is an omission which we will correct.

Thanks,
Apurva

On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com> wrote:

> >
> >
> > @Joel,
> >
> > I read over your wiki, and apart from the introduction of the notion of
> > journal partitions --whose pros and cons are already being discussed--
> you
> > also introduce the notion of a 'producer group' which enables multiple
> > producers to participate in a single transaction. This is completely
> > opposite of the model in the KIP where a transaction is defined by a
> > producer id, and hence there is a 1-1 mapping between producers and
> > transactions. Further, each producer can have exactly one in-flight
> > transaction at a time in the KIP.
> >
>
> Hi Apurva - yes I did notice those differences among other things :) BTW, I
> haven't yet gone through the google-doc carefully but on a skim it does not
> seem to contain any rejected alternatives as the wiki states.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Joel Koshy <jj...@gmail.com>.
>
>
> @Joel,
>
> I read over your wiki, and apart from the introduction of the notion of
> journal partitions --whose pros and cons are already being discussed-- you
> also introduce the notion of a 'producer group' which enables multiple
> producers to participate in a single transaction. This is completely
> opposite of the model in the KIP where a transaction is defined by a
> producer id, and hence there is a 1-1 mapping between producers and
> transactions. Further, each producer can have exactly one in-flight
> transaction at a time in the KIP.
>

Hi Apurva - yes I did notice those differences among other things :) BTW, I
haven't yet gone through the google-doc carefully but on a skim it does not
seem to contain any rejected alternatives as the wiki states.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
when the leader decides to commit a TX (of X msgs, known at this point), it
writes an "intent to append X msgs" msg (control?) followed by the X msgs
(at this point it is the leader and therefor point of sync, so this can be
done with no "foreign" msgs in between).
if there's a crash/change of leadership the new leader can just roll back
(remove what partial contents it had) if it sees the "intent" msg but dosnt
see X msgs belonging to the TX after it. the watermark does not advance
into the middle of a TX - so nothing is visible to any consumer until the
whole thing is committed and replicated (or crashes and rolled back). which
means i dont think TX storage needs replication, and atomicity to consumers
is retained.

I cant argue with the latency argument, but:

1. if TXs can be done in-mem maybe TX per-msg isnt that expensive?
2. I think a logical clock approach (with broker-side dedup based on the
clock) could provide the same exactly once semantics without requiring
transactions at all?

however, I concede that as you describe it (long running TXs where commits
are actually "checkpoint"s spaced to optimize overhead vs RPO/RTO) you
would require read uncommitted to minimize latency.

On Tue, Dec 20, 2016 at 1:24 PM, Apurva Mehta <ap...@confluent.io> wrote:

> durably at the moment we enter the pre-commit phase. If we
> don't have durable persistence of these messages, we can't have idempotent
> and atomic copying into the main  log, and your proposal to date does not
> show otherwise.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
@Radai, regarding the replication for inflight transactional messages.

I think Jay and Joel have addressed the need for transactional messages to
be persisted durably at the moment we enter the pre-commit phase. If we
don't have durable persistence of these messages, we can't have idempotent
and atomic copying into the main  log, and your proposal to date does not
show otherwise.

Additionally, I would like to point out that both the proposed solutions
for the copy operation in the transaction journal approach are pretty
invasive changes at the core of the kafka log manager layer and below: you
either have to 'splice in' segments. Or else you have to guarantee that set
of messages will be copied from one log to another idempotently and
atomically even in the case of failures, which means reliably keeping track
of messages already copied, reliably knowing from where to resume the copy,
etc.

The proposal in the KIP does not require major changes to Kafka at the Log
manager level and below: every partition involved in the transaction
(including the transaction log) is just another partition, so we inherit
all the durability guarantees for these partitions.

I don't think significantly complicating the log manager level is a deal
breaker, but I would like to point out the costs of the two log approach
from an implementation perspective.

@Joel,

I read over your wiki, and apart from the introduction of the notion of
journal partitions --whose pros and cons are already being discussed-- you
also introduce the notion of a 'producer group' which enables multiple
producers to participate in a single transaction. This is completely
opposite of the model in the KIP where a transaction is defined by a
producer id, and hence there is a 1-1 mapping between producers and
transactions. Further, each producer can have exactly one in-flight
transaction at a time in the KIP.

The motivation for the model in the KIP is the streams use-case, where a
1-1 mapping between producers and transactions is natural. I am curious
about the use cases you have in mind for a many-to-one mapping between
producers and transactions.

@all,

As Jay and Sriram have alluded to, the current proposal is geared toward
enabling transactions for streaming applications. However, the details of
these use-cases and the features they need are missing from the KIP. In
particular, enabling deep stream topologies with low end-to-end processing
time necessitates speculative execution, and is one of the driving factors
behind the present proposal. We will update the document with these
details.

Regards,
Apurva


On Tue, Dec 20, 2016 at 11:28 AM, Jay Kreps <ja...@confluent.io> wrote:

> I don't think the simple approach of writing to a local store (in memory or
> on disk) and then copying out to the destination topics would work but
> there could well be more sophisticated things that would. As you say, it is
> fine for the data to be un-replicated while you are accumulating the
> transaction, because you can always just abort the transaction if that node
> fails, but once you decided to commit and begin the process of copying out
> data you must guarantee you eventually will copy out the full transaction.
> If you have a non-durable store on one broker, and that broker crashes in
> the middle of copying out the transaction to the destination brokers, if it
> is possible that some of the writes have already succeeded, and the others
> are now lost, then you would violate atomicity.
>
> This is similar in classic two-phase commit protocols: a post-condition of
> a successful prepare commit is a promise that the transaction will
> eventually be successfully committed if requested so full durability is
> required in the pre-commit phase.
>
> But the flaw in the simple approach doesn't mean there isn't some less
> obvious solution that hasn't been thought of yet.
>
> For latency, yeah you're exactly right. We're assuming the latency of
> transactions can be pushed down to almost the duration of the transaction
> and obviously it can't be less than that. Let me try to flesh out the
> motivation for caring about latency (I think Sriram touched on this):
>
>    - We're primarily motivated by uses that fit a generalized notion of
>    correct, stateful stream processing. That is you consume/process/produce
>    potentially with associated local state in the processing. This fits KS
> and
>    Samza, but potentially a whole world of things that do transformation of
>    data. I think this is a really general notion of stream processing as a
>    kind of "protocol" and the proposed semantics give a kind of "closure"
> to
>    Kafka's producer and consumer protocols so they can be correctly
> chained.
>    - These use cases end up being a kind of DAG of transformations, often
>    even a fairly simple flow will have a depth of 5 stages and more
> realistic
>    flows can be more like 10.
>    - The transaction size is proportional to the efficiency since the
>    overhead of the transaction is fixed irrespective of the number of
>    messages. A transaction with two messages will be extremely inefficient,
>    but one with a few thousand should be much better. So you can't
> comfortably
>    make the transactions too small but yes you probably wouldn't need them
> to
>    be multisecond.
>    - The latency of the transactions stack up with the stages in the DAG in
>    a naive usage. Say you commit every 100ms, if you have 10 stages your
>    latency is going to be 1 second.
>    - This latency is definitely a concern in many domains. This is why we
>    are interested in having the option of supporting speculative execution.
>    For speculative execution you assume likely processes won't fail and
> you go
>    ahead and compute downstream results but co-ordinate the commit. This
>    trades more work rolling back when there are failures for lower latency.
>    This lets you push the end-to-end latency closer to 100ms rather than
> the
>    100ms*num_stages.
>
> Hopefully that gives a bit more color on the latency concern and desire for
> "read uncommitted".
>
> -Jay
>
> On Tue, Dec 20, 2016 at 10:33 AM, radai <ra...@gmail.com>
> wrote:
>
> > obviously anything committed would need to be replicated to all
> followers -
> > just like current msgs.
> >
> > what im trying to say is that in-flight data (written as part of an
> ongoing
> > TX and not committed yet) does not necessarily need to be replicated, or
> > even written out to disk. taken to the extreme it means i can buffer in
> > memory on the leader alone and incur no extra writes at all.
> >
> > if you dont want to just buffer in-memory on the leader (or are forced to
> > spool to disk because of size) you could still avoid a double write by
> > messing around with segment files (so the TX file becomes part of the
> > "linked-list" of segment files instead of reading it and appending it's
> > contents verbatim to the current segment file).
> >
> > the area when this does inevitably come short is latency and "read
> > uncommitted" (which are related). the added delay (after cutting all the
> > corners above) would really be the "time span" of a TX - the amount of
> time
> > from the moment the producer started the TX to the time when it was
> > committed. in my mind this time span is very short. am I failing to
> > understand the proposed "typical" use case? is the plan to use
> long-running
> > transactions and only commit at, say, 5 minute "checkpoints" ?
> >
> > On Tue, Dec 20, 2016 at 10:00 AM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Cool. It sounds like you guys will sync up and come up with a specific
> > > proposal. I think point (3) does require full replication of the
> > pre-commit
> > > transaction, but I'm not sure, and I would be very happy to learn
> > > otherwise. That was actually the blocker on that alternate proposal.
> From
> > > my point of view 2x overhead is kind of a deal breaker since it makes
> > > correctness so expensive you'd have to think very hard before turning
> it
> > > on, but if there is a way to do it with less and there aren't too many
> > > other negative side effects that would be very appealing. I think we
> can
> > > also dive a bit into why we are so perf and latency sensitive as it
> > relates
> > > to the stream processing use cases...I'm not sure how much of that is
> > > obvious from the proposal.
> > >
> > > -Jay
> > >
> > >
> > >
> > >
> > >
> > > On Tue, Dec 20, 2016 at 9:11 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> > >
> > > > Just got some time to go through most of this thread and KIP - great
> to
> > > see
> > > > this materialize and discussed!!
> > > > I will add more comments in the coming days on some of the other
> > "tracks"
> > > > in this thread; but since Radai brought up the double-journaling
> > approach
> > > > that we had discussed I thought I would move over some content from
> > > > our internal
> > > > wiki on double-journalling
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/
> > > > Double+journaling+with+local+data+copy>
> > > > It is thin on details with a few invalid statements because I don't
> > think
> > > > we dwelt long enough on it - it was cast aside as being too expensive
> > > from
> > > > a storage and latency perspective. As the immediately preceding
> emails
> > > > state, I tend to agree that those are compelling enough reasons to
> > take a
> > > > hit in complexity/increased memory usage in the consumer. Anyway,
> > couple
> > > of
> > > > us at LinkedIn can spend some time today brainstorming a little more
> on
> > > > this today.
> > > >
> > > > 1. on write amplification: i dont see x6 the writes, at worst i see
> x2
> > > the
> > > > > writes - once to the "tx log", then read and again to the
> destination
> > > > > partition. if you have some != 1 replication factor than both the
> 1st
> > > and
> > > > > the 2nd writes get replicated, but it is still a relative factor of
> > x2.
> > > > > what am I missing?
> > > > >
> > > >
> > > > I think that's right - it would be six total copies if we are doing
> RF
> > 3.
> > > >
> > > >
> > > > > 3. why do writes to a TX need the same guarantees as "plain"
> writes?
> > in
> > > > > cases where the user can live with a TX rollback on change of
> > > > > leadership/broker crash the TX log can be unreplicated, and even
> live
> > > in
> > > > > the leader's memory. that would cut down on writes. this is also an
> > > > > acceptable default in SQL - if your socket connection to a DB dies
> > > mid-TX
> > > > > your TX is toast (mysql is even worse)
> > > > >
> > > >
> > > > I may have misunderstood - while the above may be true for
> transactions
> > > > in-flight, it definitely needs the same guarantees at the point of
> > commit
> > > > and the straightforward way to achieve that is to rely on the same
> > > > guarantees while the transaction is in flight.
> > > >
> > > > 4. even if we replicate the TX log, why do we need to re-read it and
> > > > > re-write it to the underlying partition? if its already written to
> > disk
> > > > all
> > > > > I would need is to make that file the current segment of the "real"
> > > > > partition and i've avoided the double write (at the cost of
> > > complicating
> > > > > segment management). if the data is replicated fetchers could do
> the
> > > > same.
> > > > >
> > > >
> > > > I think we had considered the above as well - i.e., if you abstract
> the
> > > > partition's segments into segments that contain non-transactional
> > > messages
> > > > and those that contain transactional messages then it should be
> > possible
> > > to
> > > > jump from one to the other and back. It does add quite a bit of
> > > complexity
> > > > though and you still need to do buffering on reads so the upside
> > perhaps
> > > > isn't worth the effort. I'm not convinced about that though - i.e.,
> may
> > > > help to spend more time thinking this one through.
> > > >
> > > >
> > > > > 5. on latency - youre right, what im suggesting would result in tx
> > > > ordering
> > > > > of messages ,"read committed" semantics and therefore higher
> latency.
> > > >
> > > >
> > > > *"read committed"* only if you do the copy back to actual log. If you
> > > don't
> > > > do that (your point 4) then I think you still need to do buffering to
> > > > achieve read-committed semantics.
> > > >
> > > >
> > > >
> > > > > 6. the added delay (vs your read uncommitted) would be roughly the
> > time
> > > > > span of a TX.
> > > >
> > > >
> > > > I think it would be significantly less given that this is local
> > copying.
> > > >
> > > >
> > > >
> > > > >
> > > > >
> > > > > On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > One more thing about the double journal proposal: when discussing
> > > about
> > > > > > this method back at LinkedIn, another raised issue besides double
> > > > writing
> > > > > > was that it will void the offset ordering and enforce people to
> > > accept
> > > > > > "transaction ordering", that is, consumer will not see messages
> > from
> > > > the
> > > > > > same partition in the order where they were produced, but only in
> > the
> > > > > order
> > > > > > of when the corresponding transaction was committed. For some
> > > > scenarios,
> > > > > we
> > > > > > believe that offset ordering would still be preferred than
> > > transaction
> > > > > > ordering and that is why in KIP-98 proposal we default to the
> > former
> > > > > while
> > > > > > leave the door open if users want to switch to the latter case.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > > >
> > > > > > > Hey Radai,
> > > > > > >
> > > > > > > I'm not sure if I fully understand what you are proposing, but
> I
> > > > > > > interpreted it to be similar to a proposal we worked through
> back
> > > at
> > > > > > > LinkedIn. The proposal was to commit to a central txlog topic,
> > and
> > > > then
> > > > > > > recopy to the destination topic upon transaction commit. The
> > > > > observation
> > > > > > on
> > > > > > > that approach at the time were the following:
> > > > > > >
> > > > > > >    1. It is cleaner since the output topics have only committed
> > > data!
> > > > > > >    2. You need full replication on the txlog topic to ensure
> > > > atomicity.
> > > > > > We
> > > > > > >    weren't able to come up with a solution where you buffer in
> > > memory
> > > > > or
> > > > > > > use
> > > > > > >    renaming tricks the way you are describing. The reason is
> that
> > > > once
> > > > > > you
> > > > > > >    begin committing you must ensure that the commit eventually
> > > > succeeds
> > > > > > to
> > > > > > >    guarantee atomicity. If you use a transient store you might
> > > commit
> > > > > > some
> > > > > > >    data and then have a server failure that causes you to lose
> > the
> > > > rest
> > > > > > of
> > > > > > > the
> > > > > > >    transaction.
> > > > > > >    3. Having a single log allows the reader to choose a "read
> > > > > > uncommitted"
> > > > > > >    mode that hands out messages immediately. This is important
> > for
> > > > > cases
> > > > > > > where
> > > > > > >    latency is important, especially for stream processing
> > > topologies
> > > > > > where
> > > > > > >    these latencies stack up across multiple stages.
> > > > > > >
> > > > > > > For the stream processing use case, item (2) is a bit of a deal
> > > > killer.
> > > > > > > This takes the cost of a transient message write (say the
> > > > intermediate
> > > > > > > result of a stream processing topology) from 3x writes
> (assuming
> > 3x
> > > > > > > replication) to 6x writes. This means you basically can't
> default
> > > it
> > > > > on.
> > > > > > If
> > > > > > > we can in fact get the cost down to a single buffered write
> (i.e.
> > > 1x
> > > > > the
> > > > > > > data is written to memory and buffered to disk if the
> transaction
> > > is
> > > > > > large)
> > > > > > > as in the KIP-98 proposal without too many other negative side
> > > > effects
> > > > > I
> > > > > > > think that could be compelling.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 19, 2016 at 9:36 AM, radai <
> > radai.rosenblatt@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > regarding efficiency:
> > > > > > > >
> > > > > > > > I'd like to distinguish between server efficiency (resource
> > > > > utilization
> > > > > > > of
> > > > > > > > the broker machine alone) and overall network efficiency
> > > (resource
> > > > > > > > utilization on brokers, producers and consumers, including
> > > network
> > > > > > > > traffic).
> > > > > > > > my proposal is not as resource-efficient on the broker
> > (although
> > > it
> > > > > can
> > > > > > > be,
> > > > > > > > depends on a few trade offs and implementation details).
> > HOWEVER,
> > > > if
> > > > > i
> > > > > > > look
> > > > > > > > at the overall efficiency:
> > > > > > > >
> > > > > > > >    1.clients would need to either buffer or double-read
> > > uncommitted
> > > > > > msgs.
> > > > > > > > for N clients reading the stream M times (after re-starts and
> > > > > > reconsumes)
> > > > > > > > this would mean a M*N factor in either network BW or
> > disk/memory
> > > > > space
> > > > > > > > (depends on if buffer vs re-read). potentially N*M more
> > > broker-side
> > > > > > reads
> > > > > > > > too.
> > > > > > > >    2 to reduce the broker side cost several things can be
> done
> > > > (this
> > > > > is
> > > > > > > not
> > > > > > > > an either-or list, these are commulative):
> > > > > > > >       2.1 - keep TX logs in mem (+overflow to disk) - trades
> > disk
> > > > > > writes
> > > > > > > > for TX resiliency
> > > > > > > >       2.2 - when "appending" TX logs to real partitions -
> > instead
> > > > of
> > > > > > > > reading from (disk-based) TX log and writing to partition log
> > (x2
> > > > > disk
> > > > > > > > writes) the TX log can be made a segment file (so file
> rename,
> > > with
> > > > > > > > associated protocol changes). this would avoid double writing
> > by
> > > > > simply
> > > > > > > > making the TX file part of the partition (for large enough
> TXs.
> > > > > smaller
> > > > > > > > ones can be rewritten).
> > > > > > > >       2.3 - the approach above could be combined with a
> > > background
> > > > > > > "defrag"
> > > > > > > > - similar in concept to compaction - to further reduce the
> > total
> > > of
> > > > > > > > resulting number of files.
> > > > > > > >
> > > > > > > > I think my main issue with the current proposal, more
> important
> > > > than
> > > > > > > > performance, is lack of proper "encapsulation" of
> transactions
> > -
> > > I
> > > > > dont
> > > > > > > > think downstream consumers should see uncommitted msgs. ever.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <
> > > becket.qin@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > @Jason
> > > > > > > > >
> > > > > > > > > Yes, second thought on the number of messages included, the
> > > > offset
> > > > > > > delta
> > > > > > > > > will probably be sufficient. The use case I encounter
> before
> > > for
> > > > > > number
> > > > > > > > of
> > > > > > > > > messages in a message set is an embedded mirror maker on
> the
> > > > > > > destination
> > > > > > > > > broker side which fetches message directly from the source
> > > > cluster.
> > > > > > > > Ideally
> > > > > > > > > the destination cluster only needs to check CRC and assign
> > the
> > > > > > offsets
> > > > > > > > > because all the message verification has been done by the
> > > source
> > > > > > > cluster,
> > > > > > > > > but due to the lack of the number of messages in the
> message
> > > set,
> > > > > we
> > > > > > > have
> > > > > > > > > to decompress the message set to increment offsets
> correctly.
> > > By
> > > > > > > knowing
> > > > > > > > > the number of the messages in the message set, we can avoid
> > > doing
> > > > > > that.
> > > > > > > > The
> > > > > > > > > offset delta will also help. It's just then the offsets may
> > > have
> > > > > > holes
> > > > > > > > for
> > > > > > > > > log compacted topics, but that may be fine.
> > > > > > > > >
> > > > > > > > > @Apurva
> > > > > > > > >
> > > > > > > > > I am not sure if it is true that the consumer will either
> > > deliver
> > > > > all
> > > > > > > the
> > > > > > > > > message for the entire transaction or none of them from one
> > > > poll()
> > > > > > > call.
> > > > > > > > If
> > > > > > > > > we allow the transactions to be across partitions, unless
> the
> > > > > > consumer
> > > > > > > > > consumes from all the partitions involved in a
> transactions,
> > it
> > > > > seems
> > > > > > > > > impossible for it to deliver *all* the messages in a
> > > transaction,
> > > > > > > right?
> > > > > > > > A
> > > > > > > > > weaker guarantee is we will deliver all or none of the
> > messages
> > > > > that
> > > > > > > > belong
> > > > > > > > > to the same transaction in ONE partition, but this would be
> > > > > different
> > > > > > > > from
> > > > > > > > > the guarantee from the producer side.
> > > > > > > > >
> > > > > > > > > My two cents on Radai's sideways partition design:
> > > > > > > > > 1. If we consider the producer side behavior as doing a two
> > > phase
> > > > > > > commit
> > > > > > > > > which including the committing the consumer offsets, it is
> a
> > > > little
> > > > > > > > awkward
> > > > > > > > > that we allow uncommitted message goes into the main log
> and
> > > rely
> > > > > on
> > > > > > > the
> > > > > > > > > consumer to filter out. So semantic wise I think it would
> be
> > > > better
> > > > > > if
> > > > > > > we
> > > > > > > > > can avoid this. Radai's suggestion is actually intuitive
> > > because
> > > > if
> > > > > > the
> > > > > > > > > brokers do not want to expose uncommitted transactions to
> the
> > > > > > consumer,
> > > > > > > > the
> > > > > > > > > brokers have to buffer it.
> > > > > > > > >
> > > > > > > > > 2. Regarding the efficiency. I think may be it worth
> looking
> > at
> > > > the
> > > > > > > > > efficiency cost v.s benefit. The efficiency includes both
> > > server
> > > > > side
> > > > > > > > > efficiency and consumer side efficiency.
> > > > > > > > >
> > > > > > > > > Regarding the server side efficiency, the current proposal
> > > would
> > > > > > > probably
> > > > > > > > > have better efficiency regardless of whether something goes
> > > > wrong.
> > > > > > > > Radai's
> > > > > > > > > suggestion would put more burden on the server side. If
> > nothing
> > > > > goes
> > > > > > > > wrong
> > > > > > > > > we always pay the cost of having double copy of the
> > > transactional
> > > > > > > > messages
> > > > > > > > > and do not get the semantic benefit. But if something goes
> > > wrong,
> > > > > the
> > > > > > > > > efficiency cost we pay we get us a better semantic.
> > > > > > > > >
> > > > > > > > > For the consumer side efficiency, because there is no need
> to
> > > > > buffer
> > > > > > > the
> > > > > > > > > uncommitted messages. The current proposal may have to
> > > > potentially
> > > > > > > buffer
> > > > > > > > > uncommitted messages so it would be less efficient than
> > Radai's
> > > > > > > > suggestion
> > > > > > > > > when a transaction aborts. When everything goes well, both
> > > design
> > > > > > seems
> > > > > > > > > having the similar performance. However, it depends on
> > whether
> > > we
> > > > > are
> > > > > > > > > willing to loosen the consumer side transaction guarantee
> > that
> > > I
> > > > > > > > mentioned
> > > > > > > > > earlier to Apurva.
> > > > > > > > >
> > > > > > > > > Currently the biggest pressure on the consumer side is that
> > it
> > > > has
> > > > > to
> > > > > > > > > buffer incomplete transactions. There are two reasons for
> it,
> > > > > > > > > A. A transaction may be aborted so we cannot expose the
> > > messages
> > > > to
> > > > > > the
> > > > > > > > > users.
> > > > > > > > > B. We want to return all or none of the messages in a
> > > transaction
> > > > > in
> > > > > > > ONE
> > > > > > > > > partition.
> > > > > > > > >
> > > > > > > > > While reason A is mandatory, I think reason B may be
> > > discussable.
> > > > > > > Radai's
> > > > > > > > > design actually removes reason A because there is no
> > > uncommitted
> > > > > > > messages
> > > > > > > > > exposed to the consumers. This may potentially give us a
> > chance
> > > > to
> > > > > > > > > significantly improve consumer side efficiency in normal
> > cases.
> > > > It
> > > > > > > again
> > > > > > > > > depends on the use case, i.e. whether user can process a
> > > > > transaction
> > > > > > > > > progressively (message by message) or it has to be buffered
> > and
> > > > > > > returned
> > > > > > > > > all together. If in most cases, users can process the
> > > > transactions
> > > > > > > > message
> > > > > > > > > by message (most stream processing tasks probably can do
> so),
> > > > then
> > > > > > with
> > > > > > > > > Radai's proposal we don't need to buffer the transactions
> for
> > > the
> > > > > > users
> > > > > > > > > anymore, which is a big difference. For the latter case,
> the
> > > > > consumer
> > > > > > > may
> > > > > > > > > have to buffer the incomplete transactions otherwise we are
> > > just
> > > > > > > throwing
> > > > > > > > > the burden onto the users.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jiangjie (Becket) Qin
> > > > > > > > >
> > > > > > > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <
> jay@confluent.io
> > >
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Yeah good point. I relent!
> > > > > > > > > >
> > > > > > > > > > -jay
> > > > > > > > > >
> > > > > > > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> > > > > > jason@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Jay/Ismael,
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > I agree that lazy initialization of metadata seems
> > > > unavoidable.
> > > > > > > > > Ideally,
> > > > > > > > > > we
> > > > > > > > > > >
> > > > > > > > > > > could follow the same pattern for transactions, but
> > > remember
> > > > > that
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > consumer+producer use case, the initialization needs to
> > be
> > > > > > > completed
> > > > > > > > > > prior
> > > > > > > > > > >
> > > > > > > > > > > to setting the consumer's position. Otherwise we risk
> > > reading
> > > > > > stale
> > > > > > > > > > >
> > > > > > > > > > > offsets. But it would be pretty awkward if you have to
> > > begin
> > > > a
> > > > > > > > > > transaction
> > > > > > > > > > >
> > > > > > > > > > > first to ensure that your consumer can read the right
> > > offset
> > > > > from
> > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > consumer, right? It's a bit easier to explain that you
> > > should
> > > > > > > always
> > > > > > > > > call
> > > > > > > > > > >
> > > > > > > > > > > `producer.init()` prior to initializing the consumer.
> > Users
> > > > > would
> > > > > > > > > > probably
> > > > > > > > > > >
> > > > > > > > > > > get this right without any special effort.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > -Jason
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > > > > > > rsivaram@pivotal.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > Hi Apurva,
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > Thank you for the answers. Just one follow-on.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > 15. Let me rephrase my original question. If all
> > control
> > > > > > messages
> > > > > > > > > > > (messages
> > > > > > > > > > >
> > > > > > > > > > > > to transaction logs and markers on user logs) were
> > > > > acknowledged
> > > > > > > > only
> > > > > > > > > > > after
> > > > > > > > > > >
> > > > > > > > > > > > flushing the log segment, will transactions become
> > > durable
> > > > in
> > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > traditional sense (i.e. not restricted to
> > > > min.insync.replicas
> > > > > > > > > > failures) ?
> > > > > > > > > > >
> > > > > > > > > > > > This is not a suggestion to update the KIP. It seems
> to
> > > me
> > > > > that
> > > > > > > the
> > > > > > > > > > > design
> > > > > > > > > > >
> > > > > > > > > > > > enables full durability if required in the future
> with
> > a
> > > > > rather
> > > > > > > > > > >
> > > > > > > > > > > > non-intrusive change. I just wanted to make sure I
> > > haven't
> > > > > > missed
> > > > > > > > > > > anything
> > > > > > > > > > >
> > > > > > > > > > > > fundamental that prevents Kafka from doing this.
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <
> > ben@kirw.in
> > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > Hi Apurva,
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the detailed answers... and sorry for
> the
> > > late
> > > > > > > reply!
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > It does sound like, if the
> input-partitions-to-app-id
> > > > > mapping
> > > > > > > > never
> > > > > > > > > > >
> > > > > > > > > > > > > changes, the existing fencing mechanisms should
> > prevent
> > > > > > > > duplicates.
> > > > > > > > > > >
> > > > > > > > > > > > Great!
> > > > > > > > > > >
> > > > > > > > > > > > > I'm a bit concerned the proposed API will be
> delicate
> > > to
> > > > > > > program
> > > > > > > > > > > against
> > > > > > > > > > >
> > > > > > > > > > > > > successfully -- even in the simple case, we need to
> > > > create
> > > > > a
> > > > > > > new
> > > > > > > > > > > producer
> > > > > > > > > > >
> > > > > > > > > > > > > instance per input partition, and anything fancier
> is
> > > > going
> > > > > > to
> > > > > > > > need
> > > > > > > > > > its
> > > > > > > > > > >
> > > > > > > > > > > > own
> > > > > > > > > > >
> > > > > > > > > > > > > implementation of the Streams/Samza-style 'task'
> idea
> > > --
> > > > > but
> > > > > > > that
> > > > > > > > > may
> > > > > > > > > > > be
> > > > > > > > > > >
> > > > > > > > > > > > > fine for this sort of advanced feature.
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > For the second question, I notice that Jason also
> > > > > elaborated
> > > > > > on
> > > > > > > > > this
> > > > > > > > > > >
> > > > > > > > > > > > > downthread:
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > We also looked at removing the producer ID.
> > > > > > > > > > >
> > > > > > > > > > > > > > This was discussed somewhere above, but basically
> > the
> > > > > idea
> > > > > > is
> > > > > > > > to
> > > > > > > > > > > store
> > > > > > > > > > >
> > > > > > > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > > > AppID in the message set header directly and
> avoid
> > > the
> > > > > > > mapping
> > > > > > > > to
> > > > > > > > > > >
> > > > > > > > > > > > > producer
> > > > > > > > > > >
> > > > > > > > > > > > > > ID altogether. As long as batching isn't too bad,
> > the
> > > > > > impact
> > > > > > > on
> > > > > > > > > > total
> > > > > > > > > > >
> > > > > > > > > > > > > size
> > > > > > > > > > >
> > > > > > > > > > > > > > may not be too bad, but we were ultimately more
> > > > > comfortable
> > > > > > > > with
> > > > > > > > > a
> > > > > > > > > > >
> > > > > > > > > > > > fixed
> > > > > > > > > > >
> > > > > > > > > > > > > > size ID.
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > ...which suggests that the distinction is useful
> for
> > > > > > > performance,
> > > > > > > > > but
> > > > > > > > > > > not
> > > > > > > > > > >
> > > > > > > > > > > > > necessary for correctness, which makes good sense
> to
> > > me.
> > > > > > > (Would a
> > > > > > > > > > > 128-bid
> > > > > > > > > > >
> > > > > > > > > > > > > ID be a reasonable compromise? That's enough room
> > for a
> > > > > UUID,
> > > > > > > or
> > > > > > > > a
> > > > > > > > > > >
> > > > > > > > > > > > > reasonable hash of an arbitrary string, and has
> only
> > a
> > > > > > marginal
> > > > > > > > > > > increase
> > > > > > > > > > >
> > > > > > > > > > > > on
> > > > > > > > > > >
> > > > > > > > > > > > > the message size.)
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > > > > > > apurva@confluent.io
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > Hi Ben,
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > Now, on to your first question of how deal with
> > > > consumer
> > > > > > > > > > rebalances.
> > > > > > > > > > >
> > > > > > > > > > > > The
> > > > > > > > > > >
> > > > > > > > > > > > > > short answer is that the application needs to
> > ensure
> > > > that
> > > > > > the
> > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > > > assignment of input partitions to appId is
> > consistent
> > > > > > across
> > > > > > > > > > >
> > > > > > > > > > > > rebalances.
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > For Kafka streams, they already ensure that the
> > > mapping
> > > > > of
> > > > > > > > input
> > > > > > > > > > >
> > > > > > > > > > > > > partitions
> > > > > > > > > > >
> > > > > > > > > > > > > > to task Id is invariant across rebalances by
> > > > > implementing a
> > > > > > > > > custom
> > > > > > > > > > >
> > > > > > > > > > > > sticky
> > > > > > > > > > >
> > > > > > > > > > > > > > assignor. Other non-streams apps can trivially
> have
> > > one
> > > > > > > > producer
> > > > > > > > > > per
> > > > > > > > > > >
> > > > > > > > > > > > > input
> > > > > > > > > > >
> > > > > > > > > > > > > > partition and have the appId be the same as the
> > > > partition
> > > > > > > > number
> > > > > > > > > to
> > > > > > > > > > >
> > > > > > > > > > > > > achieve
> > > > > > > > > > >
> > > > > > > > > > > > > > the same effect.
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > With this precondition in place, we can maintain
> > > > > > transactions
> > > > > > > > > > across
> > > > > > > > > > >
> > > > > > > > > > > > > > rebalances.
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > Hope this answers your question.
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > > > > Apurva
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <
> > > > ben@kirw.in>
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for this! I'm looking forward to going
> > > through
> > > > > the
> > > > > > > > full
> > > > > > > > > > >
> > > > > > > > > > > > proposal
> > > > > > > > > > >
> > > > > > > > > > > > > > in
> > > > > > > > > > >
> > > > > > > > > > > > > > > detail soon; a few early questions:
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > First: what happens when a consumer rebalances
> in
> > > the
> > > > > > > middle
> > > > > > > > > of a
> > > > > > > > > > >
> > > > > > > > > > > > > > > transaction? The full documentation suggests
> that
> > > > such
> > > > > a
> > > > > > > > > > > transaction
> > > > > > > > > > >
> > > > > > > > > > > > > > ought
> > > > > > > > > > >
> > > > > > > > > > > > > > > to be rejected:
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > [...] if a rebalance has happened and this
> > > consumer
> > > > > > > > > > >
> > > > > > > > > > > > > > > > instance becomes a zombie, even if this
> offset
> > > > > message
> > > > > > is
> > > > > > > > > > > appended
> > > > > > > > > > >
> > > > > > > > > > > > in
> > > > > > > > > > >
> > > > > > > > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > > > > > offset topic, the transaction will be
> rejected
> > > > later
> > > > > on
> > > > > > > > when
> > > > > > > > > it
> > > > > > > > > > >
> > > > > > > > > > > > tries
> > > > > > > > > > >
> > > > > > > > > > > > > > to
> > > > > > > > > > >
> > > > > > > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > ...but it's unclear to me how we ensure that a
> > > > > > transaction
> > > > > > > > > can't
> > > > > > > > > > >
> > > > > > > > > > > > > complete
> > > > > > > > > > >
> > > > > > > > > > > > > > > if a rebalance has happened. (It's quite
> possible
> > > I'm
> > > > > > > missing
> > > > > > > > > > >
> > > > > > > > > > > > something
> > > > > > > > > > >
> > > > > > > > > > > > > > > obvious!)
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > As a concrete example: suppose a process with
> > PID 1
> > > > > adds
> > > > > > > > > offsets
> > > > > > > > > > > for
> > > > > > > > > > >
> > > > > > > > > > > > > some
> > > > > > > > > > >
> > > > > > > > > > > > > > > partition to a transaction; a consumer
> rebalance
> > > > > happens
> > > > > > > that
> > > > > > > > > > > assigns
> > > > > > > > > > >
> > > > > > > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > > > > partition to a process with PID 2, which adds
> > some
> > > > > > offsets
> > > > > > > to
> > > > > > > > > its
> > > > > > > > > > >
> > > > > > > > > > > > > current
> > > > > > > > > > >
> > > > > > > > > > > > > > > transaction; both processes try and commit.
> > > Allowing
> > > > > both
> > > > > > > > > commits
> > > > > > > > > > >
> > > > > > > > > > > > would
> > > > > > > > > > >
> > > > > > > > > > > > > > > cause the messages to be processed twice -- how
> > is
> > > > that
> > > > > > > > > avoided?
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > Second: App IDs normally map to a single PID.
> It
> > > > seems
> > > > > > like
> > > > > > > > one
> > > > > > > > > > > could
> > > > > > > > > > >
> > > > > > > > > > > > > do
> > > > > > > > > > >
> > > > > > > > > > > > > > > away with the PID concept entirely, and just
> use
> > > App
> > > > > IDs
> > > > > > in
> > > > > > > > > most
> > > > > > > > > > >
> > > > > > > > > > > > places
> > > > > > > > > > >
> > > > > > > > > > > > > > > that require a PID. This feels like it would be
> > > > > > > significantly
> > > > > > > > > > >
> > > > > > > > > > > > simpler,
> > > > > > > > > > >
> > > > > > > > > > > > > > > though it does increase the message size. Are
> > there
> > > > > other
> > > > > > > > > reasons
> > > > > > > > > > > why
> > > > > > > > > > >
> > > > > > > > > > > > > the
> > > > > > > > > > >
> > > > > > > > > > > > > > > App ID / PID split is necessary?
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang
> <
> > > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > I have just created KIP-98 to enhance Kafka
> > with
> > > > > > exactly
> > > > > > > > once
> > > > > > > > > > >
> > > > > > > > > > > > > delivery
> > > > > > > > > > >
> > > > > > > > > > > > > > > > semantics:
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > *https://cwiki.apache.org/
> > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > >
> > > > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > > > and+Transactional+Messaging
> > > > > > > > > > >
> > > > > > > > > > > > > > > > <https://cwiki.apache.org/
> > > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > > >
> > > > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > > > > and+Transactional+Messaging>*
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > This KIP adds a transactional messaging
> > mechanism
> > > > > along
> > > > > > > > with
> > > > > > > > > an
> > > > > > > > > > >
> > > > > > > > > > > > > > > idempotent
> > > > > > > > > > >
> > > > > > > > > > > > > > > > producer implementation to make sure that 1)
> > > > > duplicated
> > > > > > > > > > messages
> > > > > > > > > > >
> > > > > > > > > > > > sent
> > > > > > > > > > >
> > > > > > > > > > > > > > > from
> > > > > > > > > > >
> > > > > > > > > > > > > > > > the same identified producer can be detected
> on
> > > the
> > > > > > > broker
> > > > > > > > > > side,
> > > > > > > > > > >
> > > > > > > > > > > > and
> > > > > > > > > > >
> > > > > > > > > > > > > > 2) a
> > > > > > > > > > >
> > > > > > > > > > > > > > > > group of messages sent within a transaction
> > will
> > > > > > > atomically
> > > > > > > > > be
> > > > > > > > > > >
> > > > > > > > > > > > either
> > > > > > > > > > >
> > > > > > > > > > > > > > > > reflected and fetchable to consumers or not
> as
> > a
> > > > > whole.
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > The above wiki page provides a high-level
> view
> > of
> > > > the
> > > > > > > > > proposed
> > > > > > > > > > >
> > > > > > > > > > > > > changes
> > > > > > > > > > >
> > > > > > > > > > > > > > as
> > > > > > > > > > >
> > > > > > > > > > > > > > > > well as summarized guarantees. Initial draft
> of
> > > the
> > > > > > > > detailed
> > > > > > > > > > >
> > > > > > > > > > > > > > > implementation
> > > > > > > > > > >
> > > > > > > > > > > > > > > > design is described in this Google doc:
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > > > > > > >
> > > > > > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > > >
> > > > > > > > > > > > > > > > 0wSw9ra8
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > We would love to hear your comments and
> > > > suggestions.
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > > > -- Guozhang
> > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
I don't think the simple approach of writing to a local store (in memory or
on disk) and then copying out to the destination topics would work but
there could well be more sophisticated things that would. As you say, it is
fine for the data to be un-replicated while you are accumulating the
transaction, because you can always just abort the transaction if that node
fails, but once you decided to commit and begin the process of copying out
data you must guarantee you eventually will copy out the full transaction.
If you have a non-durable store on one broker, and that broker crashes in
the middle of copying out the transaction to the destination brokers, if it
is possible that some of the writes have already succeeded, and the others
are now lost, then you would violate atomicity.

This is similar in classic two-phase commit protocols: a post-condition of
a successful prepare commit is a promise that the transaction will
eventually be successfully committed if requested so full durability is
required in the pre-commit phase.

But the flaw in the simple approach doesn't mean there isn't some less
obvious solution that hasn't been thought of yet.

For latency, yeah you're exactly right. We're assuming the latency of
transactions can be pushed down to almost the duration of the transaction
and obviously it can't be less than that. Let me try to flesh out the
motivation for caring about latency (I think Sriram touched on this):

   - We're primarily motivated by uses that fit a generalized notion of
   correct, stateful stream processing. That is you consume/process/produce
   potentially with associated local state in the processing. This fits KS and
   Samza, but potentially a whole world of things that do transformation of
   data. I think this is a really general notion of stream processing as a
   kind of "protocol" and the proposed semantics give a kind of "closure" to
   Kafka's producer and consumer protocols so they can be correctly chained.
   - These use cases end up being a kind of DAG of transformations, often
   even a fairly simple flow will have a depth of 5 stages and more realistic
   flows can be more like 10.
   - The transaction size is proportional to the efficiency since the
   overhead of the transaction is fixed irrespective of the number of
   messages. A transaction with two messages will be extremely inefficient,
   but one with a few thousand should be much better. So you can't comfortably
   make the transactions too small but yes you probably wouldn't need them to
   be multisecond.
   - The latency of the transactions stack up with the stages in the DAG in
   a naive usage. Say you commit every 100ms, if you have 10 stages your
   latency is going to be 1 second.
   - This latency is definitely a concern in many domains. This is why we
   are interested in having the option of supporting speculative execution.
   For speculative execution you assume likely processes won't fail and you go
   ahead and compute downstream results but co-ordinate the commit. This
   trades more work rolling back when there are failures for lower latency.
   This lets you push the end-to-end latency closer to 100ms rather than the
   100ms*num_stages.

Hopefully that gives a bit more color on the latency concern and desire for
"read uncommitted".

-Jay

On Tue, Dec 20, 2016 at 10:33 AM, radai <ra...@gmail.com> wrote:

> obviously anything committed would need to be replicated to all followers -
> just like current msgs.
>
> what im trying to say is that in-flight data (written as part of an ongoing
> TX and not committed yet) does not necessarily need to be replicated, or
> even written out to disk. taken to the extreme it means i can buffer in
> memory on the leader alone and incur no extra writes at all.
>
> if you dont want to just buffer in-memory on the leader (or are forced to
> spool to disk because of size) you could still avoid a double write by
> messing around with segment files (so the TX file becomes part of the
> "linked-list" of segment files instead of reading it and appending it's
> contents verbatim to the current segment file).
>
> the area when this does inevitably come short is latency and "read
> uncommitted" (which are related). the added delay (after cutting all the
> corners above) would really be the "time span" of a TX - the amount of time
> from the moment the producer started the TX to the time when it was
> committed. in my mind this time span is very short. am I failing to
> understand the proposed "typical" use case? is the plan to use long-running
> transactions and only commit at, say, 5 minute "checkpoints" ?
>
> On Tue, Dec 20, 2016 at 10:00 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Cool. It sounds like you guys will sync up and come up with a specific
> > proposal. I think point (3) does require full replication of the
> pre-commit
> > transaction, but I'm not sure, and I would be very happy to learn
> > otherwise. That was actually the blocker on that alternate proposal. From
> > my point of view 2x overhead is kind of a deal breaker since it makes
> > correctness so expensive you'd have to think very hard before turning it
> > on, but if there is a way to do it with less and there aren't too many
> > other negative side effects that would be very appealing. I think we can
> > also dive a bit into why we are so perf and latency sensitive as it
> relates
> > to the stream processing use cases...I'm not sure how much of that is
> > obvious from the proposal.
> >
> > -Jay
> >
> >
> >
> >
> >
> > On Tue, Dec 20, 2016 at 9:11 AM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > > Just got some time to go through most of this thread and KIP - great to
> > see
> > > this materialize and discussed!!
> > > I will add more comments in the coming days on some of the other
> "tracks"
> > > in this thread; but since Radai brought up the double-journaling
> approach
> > > that we had discussed I thought I would move over some content from
> > > our internal
> > > wiki on double-journalling
> > > <https://cwiki.apache.org/confluence/display/KAFKA/
> > > Double+journaling+with+local+data+copy>
> > > It is thin on details with a few invalid statements because I don't
> think
> > > we dwelt long enough on it - it was cast aside as being too expensive
> > from
> > > a storage and latency perspective. As the immediately preceding emails
> > > state, I tend to agree that those are compelling enough reasons to
> take a
> > > hit in complexity/increased memory usage in the consumer. Anyway,
> couple
> > of
> > > us at LinkedIn can spend some time today brainstorming a little more on
> > > this today.
> > >
> > > 1. on write amplification: i dont see x6 the writes, at worst i see x2
> > the
> > > > writes - once to the "tx log", then read and again to the destination
> > > > partition. if you have some != 1 replication factor than both the 1st
> > and
> > > > the 2nd writes get replicated, but it is still a relative factor of
> x2.
> > > > what am I missing?
> > > >
> > >
> > > I think that's right - it would be six total copies if we are doing RF
> 3.
> > >
> > >
> > > > 3. why do writes to a TX need the same guarantees as "plain" writes?
> in
> > > > cases where the user can live with a TX rollback on change of
> > > > leadership/broker crash the TX log can be unreplicated, and even live
> > in
> > > > the leader's memory. that would cut down on writes. this is also an
> > > > acceptable default in SQL - if your socket connection to a DB dies
> > mid-TX
> > > > your TX is toast (mysql is even worse)
> > > >
> > >
> > > I may have misunderstood - while the above may be true for transactions
> > > in-flight, it definitely needs the same guarantees at the point of
> commit
> > > and the straightforward way to achieve that is to rely on the same
> > > guarantees while the transaction is in flight.
> > >
> > > 4. even if we replicate the TX log, why do we need to re-read it and
> > > > re-write it to the underlying partition? if its already written to
> disk
> > > all
> > > > I would need is to make that file the current segment of the "real"
> > > > partition and i've avoided the double write (at the cost of
> > complicating
> > > > segment management). if the data is replicated fetchers could do the
> > > same.
> > > >
> > >
> > > I think we had considered the above as well - i.e., if you abstract the
> > > partition's segments into segments that contain non-transactional
> > messages
> > > and those that contain transactional messages then it should be
> possible
> > to
> > > jump from one to the other and back. It does add quite a bit of
> > complexity
> > > though and you still need to do buffering on reads so the upside
> perhaps
> > > isn't worth the effort. I'm not convinced about that though - i.e., may
> > > help to spend more time thinking this one through.
> > >
> > >
> > > > 5. on latency - youre right, what im suggesting would result in tx
> > > ordering
> > > > of messages ,"read committed" semantics and therefore higher latency.
> > >
> > >
> > > *"read committed"* only if you do the copy back to actual log. If you
> > don't
> > > do that (your point 4) then I think you still need to do buffering to
> > > achieve read-committed semantics.
> > >
> > >
> > >
> > > > 6. the added delay (vs your read uncommitted) would be roughly the
> time
> > > > span of a TX.
> > >
> > >
> > > I think it would be significantly less given that this is local
> copying.
> > >
> > >
> > >
> > > >
> > > >
> > > > On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > One more thing about the double journal proposal: when discussing
> > about
> > > > > this method back at LinkedIn, another raised issue besides double
> > > writing
> > > > > was that it will void the offset ordering and enforce people to
> > accept
> > > > > "transaction ordering", that is, consumer will not see messages
> from
> > > the
> > > > > same partition in the order where they were produced, but only in
> the
> > > > order
> > > > > of when the corresponding transaction was committed. For some
> > > scenarios,
> > > > we
> > > > > believe that offset ordering would still be preferred than
> > transaction
> > > > > ordering and that is why in KIP-98 proposal we default to the
> former
> > > > while
> > > > > leave the door open if users want to switch to the latter case.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > > >
> > > > > > Hey Radai,
> > > > > >
> > > > > > I'm not sure if I fully understand what you are proposing, but I
> > > > > > interpreted it to be similar to a proposal we worked through back
> > at
> > > > > > LinkedIn. The proposal was to commit to a central txlog topic,
> and
> > > then
> > > > > > recopy to the destination topic upon transaction commit. The
> > > > observation
> > > > > on
> > > > > > that approach at the time were the following:
> > > > > >
> > > > > >    1. It is cleaner since the output topics have only committed
> > data!
> > > > > >    2. You need full replication on the txlog topic to ensure
> > > atomicity.
> > > > > We
> > > > > >    weren't able to come up with a solution where you buffer in
> > memory
> > > > or
> > > > > > use
> > > > > >    renaming tricks the way you are describing. The reason is that
> > > once
> > > > > you
> > > > > >    begin committing you must ensure that the commit eventually
> > > succeeds
> > > > > to
> > > > > >    guarantee atomicity. If you use a transient store you might
> > commit
> > > > > some
> > > > > >    data and then have a server failure that causes you to lose
> the
> > > rest
> > > > > of
> > > > > > the
> > > > > >    transaction.
> > > > > >    3. Having a single log allows the reader to choose a "read
> > > > > uncommitted"
> > > > > >    mode that hands out messages immediately. This is important
> for
> > > > cases
> > > > > > where
> > > > > >    latency is important, especially for stream processing
> > topologies
> > > > > where
> > > > > >    these latencies stack up across multiple stages.
> > > > > >
> > > > > > For the stream processing use case, item (2) is a bit of a deal
> > > killer.
> > > > > > This takes the cost of a transient message write (say the
> > > intermediate
> > > > > > result of a stream processing topology) from 3x writes (assuming
> 3x
> > > > > > replication) to 6x writes. This means you basically can't default
> > it
> > > > on.
> > > > > If
> > > > > > we can in fact get the cost down to a single buffered write (i.e.
> > 1x
> > > > the
> > > > > > data is written to memory and buffered to disk if the transaction
> > is
> > > > > large)
> > > > > > as in the KIP-98 proposal without too many other negative side
> > > effects
> > > > I
> > > > > > think that could be compelling.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 19, 2016 at 9:36 AM, radai <
> radai.rosenblatt@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > regarding efficiency:
> > > > > > >
> > > > > > > I'd like to distinguish between server efficiency (resource
> > > > utilization
> > > > > > of
> > > > > > > the broker machine alone) and overall network efficiency
> > (resource
> > > > > > > utilization on brokers, producers and consumers, including
> > network
> > > > > > > traffic).
> > > > > > > my proposal is not as resource-efficient on the broker
> (although
> > it
> > > > can
> > > > > > be,
> > > > > > > depends on a few trade offs and implementation details).
> HOWEVER,
> > > if
> > > > i
> > > > > > look
> > > > > > > at the overall efficiency:
> > > > > > >
> > > > > > >    1.clients would need to either buffer or double-read
> > uncommitted
> > > > > msgs.
> > > > > > > for N clients reading the stream M times (after re-starts and
> > > > > reconsumes)
> > > > > > > this would mean a M*N factor in either network BW or
> disk/memory
> > > > space
> > > > > > > (depends on if buffer vs re-read). potentially N*M more
> > broker-side
> > > > > reads
> > > > > > > too.
> > > > > > >    2 to reduce the broker side cost several things can be done
> > > (this
> > > > is
> > > > > > not
> > > > > > > an either-or list, these are commulative):
> > > > > > >       2.1 - keep TX logs in mem (+overflow to disk) - trades
> disk
> > > > > writes
> > > > > > > for TX resiliency
> > > > > > >       2.2 - when "appending" TX logs to real partitions -
> instead
> > > of
> > > > > > > reading from (disk-based) TX log and writing to partition log
> (x2
> > > > disk
> > > > > > > writes) the TX log can be made a segment file (so file rename,
> > with
> > > > > > > associated protocol changes). this would avoid double writing
> by
> > > > simply
> > > > > > > making the TX file part of the partition (for large enough TXs.
> > > > smaller
> > > > > > > ones can be rewritten).
> > > > > > >       2.3 - the approach above could be combined with a
> > background
> > > > > > "defrag"
> > > > > > > - similar in concept to compaction - to further reduce the
> total
> > of
> > > > > > > resulting number of files.
> > > > > > >
> > > > > > > I think my main issue with the current proposal, more important
> > > than
> > > > > > > performance, is lack of proper "encapsulation" of transactions
> -
> > I
> > > > dont
> > > > > > > think downstream consumers should see uncommitted msgs. ever.
> > > > > > >
> > > > > > >
> > > > > > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <
> > becket.qin@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > @Jason
> > > > > > > >
> > > > > > > > Yes, second thought on the number of messages included, the
> > > offset
> > > > > > delta
> > > > > > > > will probably be sufficient. The use case I encounter before
> > for
> > > > > number
> > > > > > > of
> > > > > > > > messages in a message set is an embedded mirror maker on the
> > > > > > destination
> > > > > > > > broker side which fetches message directly from the source
> > > cluster.
> > > > > > > Ideally
> > > > > > > > the destination cluster only needs to check CRC and assign
> the
> > > > > offsets
> > > > > > > > because all the message verification has been done by the
> > source
> > > > > > cluster,
> > > > > > > > but due to the lack of the number of messages in the message
> > set,
> > > > we
> > > > > > have
> > > > > > > > to decompress the message set to increment offsets correctly.
> > By
> > > > > > knowing
> > > > > > > > the number of the messages in the message set, we can avoid
> > doing
> > > > > that.
> > > > > > > The
> > > > > > > > offset delta will also help. It's just then the offsets may
> > have
> > > > > holes
> > > > > > > for
> > > > > > > > log compacted topics, but that may be fine.
> > > > > > > >
> > > > > > > > @Apurva
> > > > > > > >
> > > > > > > > I am not sure if it is true that the consumer will either
> > deliver
> > > > all
> > > > > > the
> > > > > > > > message for the entire transaction or none of them from one
> > > poll()
> > > > > > call.
> > > > > > > If
> > > > > > > > we allow the transactions to be across partitions, unless the
> > > > > consumer
> > > > > > > > consumes from all the partitions involved in a transactions,
> it
> > > > seems
> > > > > > > > impossible for it to deliver *all* the messages in a
> > transaction,
> > > > > > right?
> > > > > > > A
> > > > > > > > weaker guarantee is we will deliver all or none of the
> messages
> > > > that
> > > > > > > belong
> > > > > > > > to the same transaction in ONE partition, but this would be
> > > > different
> > > > > > > from
> > > > > > > > the guarantee from the producer side.
> > > > > > > >
> > > > > > > > My two cents on Radai's sideways partition design:
> > > > > > > > 1. If we consider the producer side behavior as doing a two
> > phase
> > > > > > commit
> > > > > > > > which including the committing the consumer offsets, it is a
> > > little
> > > > > > > awkward
> > > > > > > > that we allow uncommitted message goes into the main log and
> > rely
> > > > on
> > > > > > the
> > > > > > > > consumer to filter out. So semantic wise I think it would be
> > > better
> > > > > if
> > > > > > we
> > > > > > > > can avoid this. Radai's suggestion is actually intuitive
> > because
> > > if
> > > > > the
> > > > > > > > brokers do not want to expose uncommitted transactions to the
> > > > > consumer,
> > > > > > > the
> > > > > > > > brokers have to buffer it.
> > > > > > > >
> > > > > > > > 2. Regarding the efficiency. I think may be it worth looking
> at
> > > the
> > > > > > > > efficiency cost v.s benefit. The efficiency includes both
> > server
> > > > side
> > > > > > > > efficiency and consumer side efficiency.
> > > > > > > >
> > > > > > > > Regarding the server side efficiency, the current proposal
> > would
> > > > > > probably
> > > > > > > > have better efficiency regardless of whether something goes
> > > wrong.
> > > > > > > Radai's
> > > > > > > > suggestion would put more burden on the server side. If
> nothing
> > > > goes
> > > > > > > wrong
> > > > > > > > we always pay the cost of having double copy of the
> > transactional
> > > > > > > messages
> > > > > > > > and do not get the semantic benefit. But if something goes
> > wrong,
> > > > the
> > > > > > > > efficiency cost we pay we get us a better semantic.
> > > > > > > >
> > > > > > > > For the consumer side efficiency, because there is no need to
> > > > buffer
> > > > > > the
> > > > > > > > uncommitted messages. The current proposal may have to
> > > potentially
> > > > > > buffer
> > > > > > > > uncommitted messages so it would be less efficient than
> Radai's
> > > > > > > suggestion
> > > > > > > > when a transaction aborts. When everything goes well, both
> > design
> > > > > seems
> > > > > > > > having the similar performance. However, it depends on
> whether
> > we
> > > > are
> > > > > > > > willing to loosen the consumer side transaction guarantee
> that
> > I
> > > > > > > mentioned
> > > > > > > > earlier to Apurva.
> > > > > > > >
> > > > > > > > Currently the biggest pressure on the consumer side is that
> it
> > > has
> > > > to
> > > > > > > > buffer incomplete transactions. There are two reasons for it,
> > > > > > > > A. A transaction may be aborted so we cannot expose the
> > messages
> > > to
> > > > > the
> > > > > > > > users.
> > > > > > > > B. We want to return all or none of the messages in a
> > transaction
> > > > in
> > > > > > ONE
> > > > > > > > partition.
> > > > > > > >
> > > > > > > > While reason A is mandatory, I think reason B may be
> > discussable.
> > > > > > Radai's
> > > > > > > > design actually removes reason A because there is no
> > uncommitted
> > > > > > messages
> > > > > > > > exposed to the consumers. This may potentially give us a
> chance
> > > to
> > > > > > > > significantly improve consumer side efficiency in normal
> cases.
> > > It
> > > > > > again
> > > > > > > > depends on the use case, i.e. whether user can process a
> > > > transaction
> > > > > > > > progressively (message by message) or it has to be buffered
> and
> > > > > > returned
> > > > > > > > all together. If in most cases, users can process the
> > > transactions
> > > > > > > message
> > > > > > > > by message (most stream processing tasks probably can do so),
> > > then
> > > > > with
> > > > > > > > Radai's proposal we don't need to buffer the transactions for
> > the
> > > > > users
> > > > > > > > anymore, which is a big difference. For the latter case, the
> > > > consumer
> > > > > > may
> > > > > > > > have to buffer the incomplete transactions otherwise we are
> > just
> > > > > > throwing
> > > > > > > > the burden onto the users.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jiangjie (Becket) Qin
> > > > > > > >
> > > > > > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <jay@confluent.io
> >
> > > > wrote:
> > > > > > > >
> > > > > > > > > Yeah good point. I relent!
> > > > > > > > >
> > > > > > > > > -jay
> > > > > > > > >
> > > > > > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Jay/Ismael,
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > I agree that lazy initialization of metadata seems
> > > unavoidable.
> > > > > > > > Ideally,
> > > > > > > > > we
> > > > > > > > > >
> > > > > > > > > > could follow the same pattern for transactions, but
> > remember
> > > > that
> > > > > > in
> > > > > > > > the
> > > > > > > > > >
> > > > > > > > > > consumer+producer use case, the initialization needs to
> be
> > > > > > completed
> > > > > > > > > prior
> > > > > > > > > >
> > > > > > > > > > to setting the consumer's position. Otherwise we risk
> > reading
> > > > > stale
> > > > > > > > > >
> > > > > > > > > > offsets. But it would be pretty awkward if you have to
> > begin
> > > a
> > > > > > > > > transaction
> > > > > > > > > >
> > > > > > > > > > first to ensure that your consumer can read the right
> > offset
> > > > from
> > > > > > the
> > > > > > > > > >
> > > > > > > > > > consumer, right? It's a bit easier to explain that you
> > should
> > > > > > always
> > > > > > > > call
> > > > > > > > > >
> > > > > > > > > > `producer.init()` prior to initializing the consumer.
> Users
> > > > would
> > > > > > > > > probably
> > > > > > > > > >
> > > > > > > > > > get this right without any special effort.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > -Jason
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > > > > > rsivaram@pivotal.io
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > Hi Apurva,
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > Thank you for the answers. Just one follow-on.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > 15. Let me rephrase my original question. If all
> control
> > > > > messages
> > > > > > > > > > (messages
> > > > > > > > > >
> > > > > > > > > > > to transaction logs and markers on user logs) were
> > > > acknowledged
> > > > > > > only
> > > > > > > > > > after
> > > > > > > > > >
> > > > > > > > > > > flushing the log segment, will transactions become
> > durable
> > > in
> > > > > the
> > > > > > > > > >
> > > > > > > > > > > traditional sense (i.e. not restricted to
> > > min.insync.replicas
> > > > > > > > > failures) ?
> > > > > > > > > >
> > > > > > > > > > > This is not a suggestion to update the KIP. It seems to
> > me
> > > > that
> > > > > > the
> > > > > > > > > > design
> > > > > > > > > >
> > > > > > > > > > > enables full durability if required in the future with
> a
> > > > rather
> > > > > > > > > >
> > > > > > > > > > > non-intrusive change. I just wanted to make sure I
> > haven't
> > > > > missed
> > > > > > > > > > anything
> > > > > > > > > >
> > > > > > > > > > > fundamental that prevents Kafka from doing this.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <
> ben@kirw.in
> > >
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > Hi Apurva,
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > Thanks for the detailed answers... and sorry for the
> > late
> > > > > > reply!
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > It does sound like, if the input-partitions-to-app-id
> > > > mapping
> > > > > > > never
> > > > > > > > > >
> > > > > > > > > > > > changes, the existing fencing mechanisms should
> prevent
> > > > > > > duplicates.
> > > > > > > > > >
> > > > > > > > > > > Great!
> > > > > > > > > >
> > > > > > > > > > > > I'm a bit concerned the proposed API will be delicate
> > to
> > > > > > program
> > > > > > > > > > against
> > > > > > > > > >
> > > > > > > > > > > > successfully -- even in the simple case, we need to
> > > create
> > > > a
> > > > > > new
> > > > > > > > > > producer
> > > > > > > > > >
> > > > > > > > > > > > instance per input partition, and anything fancier is
> > > going
> > > > > to
> > > > > > > need
> > > > > > > > > its
> > > > > > > > > >
> > > > > > > > > > > own
> > > > > > > > > >
> > > > > > > > > > > > implementation of the Streams/Samza-style 'task' idea
> > --
> > > > but
> > > > > > that
> > > > > > > > may
> > > > > > > > > > be
> > > > > > > > > >
> > > > > > > > > > > > fine for this sort of advanced feature.
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > For the second question, I notice that Jason also
> > > > elaborated
> > > > > on
> > > > > > > > this
> > > > > > > > > >
> > > > > > > > > > > > downthread:
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > We also looked at removing the producer ID.
> > > > > > > > > >
> > > > > > > > > > > > > This was discussed somewhere above, but basically
> the
> > > > idea
> > > > > is
> > > > > > > to
> > > > > > > > > > store
> > > > > > > > > >
> > > > > > > > > > > > the
> > > > > > > > > >
> > > > > > > > > > > > > AppID in the message set header directly and avoid
> > the
> > > > > > mapping
> > > > > > > to
> > > > > > > > > >
> > > > > > > > > > > > producer
> > > > > > > > > >
> > > > > > > > > > > > > ID altogether. As long as batching isn't too bad,
> the
> > > > > impact
> > > > > > on
> > > > > > > > > total
> > > > > > > > > >
> > > > > > > > > > > > size
> > > > > > > > > >
> > > > > > > > > > > > > may not be too bad, but we were ultimately more
> > > > comfortable
> > > > > > > with
> > > > > > > > a
> > > > > > > > > >
> > > > > > > > > > > fixed
> > > > > > > > > >
> > > > > > > > > > > > > size ID.
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > ...which suggests that the distinction is useful for
> > > > > > performance,
> > > > > > > > but
> > > > > > > > > > not
> > > > > > > > > >
> > > > > > > > > > > > necessary for correctness, which makes good sense to
> > me.
> > > > > > (Would a
> > > > > > > > > > 128-bid
> > > > > > > > > >
> > > > > > > > > > > > ID be a reasonable compromise? That's enough room
> for a
> > > > UUID,
> > > > > > or
> > > > > > > a
> > > > > > > > > >
> > > > > > > > > > > > reasonable hash of an arbitrary string, and has only
> a
> > > > > marginal
> > > > > > > > > > increase
> > > > > > > > > >
> > > > > > > > > > > on
> > > > > > > > > >
> > > > > > > > > > > > the message size.)
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > > > > > apurva@confluent.io
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > Hi Ben,
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > Now, on to your first question of how deal with
> > > consumer
> > > > > > > > > rebalances.
> > > > > > > > > >
> > > > > > > > > > > The
> > > > > > > > > >
> > > > > > > > > > > > > short answer is that the application needs to
> ensure
> > > that
> > > > > the
> > > > > > > the
> > > > > > > > > >
> > > > > > > > > > > > > assignment of input partitions to appId is
> consistent
> > > > > across
> > > > > > > > > >
> > > > > > > > > > > rebalances.
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > For Kafka streams, they already ensure that the
> > mapping
> > > > of
> > > > > > > input
> > > > > > > > > >
> > > > > > > > > > > > partitions
> > > > > > > > > >
> > > > > > > > > > > > > to task Id is invariant across rebalances by
> > > > implementing a
> > > > > > > > custom
> > > > > > > > > >
> > > > > > > > > > > sticky
> > > > > > > > > >
> > > > > > > > > > > > > assignor. Other non-streams apps can trivially have
> > one
> > > > > > > producer
> > > > > > > > > per
> > > > > > > > > >
> > > > > > > > > > > > input
> > > > > > > > > >
> > > > > > > > > > > > > partition and have the appId be the same as the
> > > partition
> > > > > > > number
> > > > > > > > to
> > > > > > > > > >
> > > > > > > > > > > > achieve
> > > > > > > > > >
> > > > > > > > > > > > > the same effect.
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > With this precondition in place, we can maintain
> > > > > transactions
> > > > > > > > > across
> > > > > > > > > >
> > > > > > > > > > > > > rebalances.
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > Hope this answers your question.
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > > > > Apurva
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <
> > > ben@kirw.in>
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > Thanks for this! I'm looking forward to going
> > through
> > > > the
> > > > > > > full
> > > > > > > > > >
> > > > > > > > > > > proposal
> > > > > > > > > >
> > > > > > > > > > > > > in
> > > > > > > > > >
> > > > > > > > > > > > > > detail soon; a few early questions:
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > First: what happens when a consumer rebalances in
> > the
> > > > > > middle
> > > > > > > > of a
> > > > > > > > > >
> > > > > > > > > > > > > > transaction? The full documentation suggests that
> > > such
> > > > a
> > > > > > > > > > transaction
> > > > > > > > > >
> > > > > > > > > > > > > ought
> > > > > > > > > >
> > > > > > > > > > > > > > to be rejected:
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > [...] if a rebalance has happened and this
> > consumer
> > > > > > > > > >
> > > > > > > > > > > > > > > instance becomes a zombie, even if this offset
> > > > message
> > > > > is
> > > > > > > > > > appended
> > > > > > > > > >
> > > > > > > > > > > in
> > > > > > > > > >
> > > > > > > > > > > > > the
> > > > > > > > > >
> > > > > > > > > > > > > > > offset topic, the transaction will be rejected
> > > later
> > > > on
> > > > > > > when
> > > > > > > > it
> > > > > > > > > >
> > > > > > > > > > > tries
> > > > > > > > > >
> > > > > > > > > > > > > to
> > > > > > > > > >
> > > > > > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > ...but it's unclear to me how we ensure that a
> > > > > transaction
> > > > > > > > can't
> > > > > > > > > >
> > > > > > > > > > > > complete
> > > > > > > > > >
> > > > > > > > > > > > > > if a rebalance has happened. (It's quite possible
> > I'm
> > > > > > missing
> > > > > > > > > >
> > > > > > > > > > > something
> > > > > > > > > >
> > > > > > > > > > > > > > obvious!)
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > As a concrete example: suppose a process with
> PID 1
> > > > adds
> > > > > > > > offsets
> > > > > > > > > > for
> > > > > > > > > >
> > > > > > > > > > > > some
> > > > > > > > > >
> > > > > > > > > > > > > > partition to a transaction; a consumer rebalance
> > > > happens
> > > > > > that
> > > > > > > > > > assigns
> > > > > > > > > >
> > > > > > > > > > > > the
> > > > > > > > > >
> > > > > > > > > > > > > > partition to a process with PID 2, which adds
> some
> > > > > offsets
> > > > > > to
> > > > > > > > its
> > > > > > > > > >
> > > > > > > > > > > > current
> > > > > > > > > >
> > > > > > > > > > > > > > transaction; both processes try and commit.
> > Allowing
> > > > both
> > > > > > > > commits
> > > > > > > > > >
> > > > > > > > > > > would
> > > > > > > > > >
> > > > > > > > > > > > > > cause the messages to be processed twice -- how
> is
> > > that
> > > > > > > > avoided?
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > Second: App IDs normally map to a single PID. It
> > > seems
> > > > > like
> > > > > > > one
> > > > > > > > > > could
> > > > > > > > > >
> > > > > > > > > > > > do
> > > > > > > > > >
> > > > > > > > > > > > > > away with the PID concept entirely, and just use
> > App
> > > > IDs
> > > > > in
> > > > > > > > most
> > > > > > > > > >
> > > > > > > > > > > places
> > > > > > > > > >
> > > > > > > > > > > > > > that require a PID. This feels like it would be
> > > > > > significantly
> > > > > > > > > >
> > > > > > > > > > > simpler,
> > > > > > > > > >
> > > > > > > > > > > > > > though it does increase the message size. Are
> there
> > > > other
> > > > > > > > reasons
> > > > > > > > > > why
> > > > > > > > > >
> > > > > > > > > > > > the
> > > > > > > > > >
> > > > > > > > > > > > > > App ID / PID split is necessary?
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > I have just created KIP-98 to enhance Kafka
> with
> > > > > exactly
> > > > > > > once
> > > > > > > > > >
> > > > > > > > > > > > delivery
> > > > > > > > > >
> > > > > > > > > > > > > > > semantics:
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > *https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > >
> > > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > > and+Transactional+Messaging
> > > > > > > > > >
> > > > > > > > > > > > > > > <https://cwiki.apache.org/
> > > > > confluence/display/KAFKA/KIP-
> > > > > > > > > >
> > > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > > > and+Transactional+Messaging>*
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > This KIP adds a transactional messaging
> mechanism
> > > > along
> > > > > > > with
> > > > > > > > an
> > > > > > > > > >
> > > > > > > > > > > > > > idempotent
> > > > > > > > > >
> > > > > > > > > > > > > > > producer implementation to make sure that 1)
> > > > duplicated
> > > > > > > > > messages
> > > > > > > > > >
> > > > > > > > > > > sent
> > > > > > > > > >
> > > > > > > > > > > > > > from
> > > > > > > > > >
> > > > > > > > > > > > > > > the same identified producer can be detected on
> > the
> > > > > > broker
> > > > > > > > > side,
> > > > > > > > > >
> > > > > > > > > > > and
> > > > > > > > > >
> > > > > > > > > > > > > 2) a
> > > > > > > > > >
> > > > > > > > > > > > > > > group of messages sent within a transaction
> will
> > > > > > atomically
> > > > > > > > be
> > > > > > > > > >
> > > > > > > > > > > either
> > > > > > > > > >
> > > > > > > > > > > > > > > reflected and fetchable to consumers or not as
> a
> > > > whole.
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > The above wiki page provides a high-level view
> of
> > > the
> > > > > > > > proposed
> > > > > > > > > >
> > > > > > > > > > > > changes
> > > > > > > > > >
> > > > > > > > > > > > > as
> > > > > > > > > >
> > > > > > > > > > > > > > > well as summarized guarantees. Initial draft of
> > the
> > > > > > > detailed
> > > > > > > > > >
> > > > > > > > > > > > > > implementation
> > > > > > > > > >
> > > > > > > > > > > > > > > design is described in this Google doc:
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > > > > > >
> > > > > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > > >
> > > > > > > > > > > > > > > 0wSw9ra8
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > We would love to hear your comments and
> > > suggestions.
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > > > -- Guozhang
> > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
obviously anything committed would need to be replicated to all followers -
just like current msgs.

what im trying to say is that in-flight data (written as part of an ongoing
TX and not committed yet) does not necessarily need to be replicated, or
even written out to disk. taken to the extreme it means i can buffer in
memory on the leader alone and incur no extra writes at all.

if you dont want to just buffer in-memory on the leader (or are forced to
spool to disk because of size) you could still avoid a double write by
messing around with segment files (so the TX file becomes part of the
"linked-list" of segment files instead of reading it and appending it's
contents verbatim to the current segment file).

the area when this does inevitably come short is latency and "read
uncommitted" (which are related). the added delay (after cutting all the
corners above) would really be the "time span" of a TX - the amount of time
from the moment the producer started the TX to the time when it was
committed. in my mind this time span is very short. am I failing to
understand the proposed "typical" use case? is the plan to use long-running
transactions and only commit at, say, 5 minute "checkpoints" ?

On Tue, Dec 20, 2016 at 10:00 AM, Jay Kreps <ja...@confluent.io> wrote:

> Cool. It sounds like you guys will sync up and come up with a specific
> proposal. I think point (3) does require full replication of the pre-commit
> transaction, but I'm not sure, and I would be very happy to learn
> otherwise. That was actually the blocker on that alternate proposal. From
> my point of view 2x overhead is kind of a deal breaker since it makes
> correctness so expensive you'd have to think very hard before turning it
> on, but if there is a way to do it with less and there aren't too many
> other negative side effects that would be very appealing. I think we can
> also dive a bit into why we are so perf and latency sensitive as it relates
> to the stream processing use cases...I'm not sure how much of that is
> obvious from the proposal.
>
> -Jay
>
>
>
>
>
> On Tue, Dec 20, 2016 at 9:11 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > Just got some time to go through most of this thread and KIP - great to
> see
> > this materialize and discussed!!
> > I will add more comments in the coming days on some of the other "tracks"
> > in this thread; but since Radai brought up the double-journaling approach
> > that we had discussed I thought I would move over some content from
> > our internal
> > wiki on double-journalling
> > <https://cwiki.apache.org/confluence/display/KAFKA/
> > Double+journaling+with+local+data+copy>
> > It is thin on details with a few invalid statements because I don't think
> > we dwelt long enough on it - it was cast aside as being too expensive
> from
> > a storage and latency perspective. As the immediately preceding emails
> > state, I tend to agree that those are compelling enough reasons to take a
> > hit in complexity/increased memory usage in the consumer. Anyway, couple
> of
> > us at LinkedIn can spend some time today brainstorming a little more on
> > this today.
> >
> > 1. on write amplification: i dont see x6 the writes, at worst i see x2
> the
> > > writes - once to the "tx log", then read and again to the destination
> > > partition. if you have some != 1 replication factor than both the 1st
> and
> > > the 2nd writes get replicated, but it is still a relative factor of x2.
> > > what am I missing?
> > >
> >
> > I think that's right - it would be six total copies if we are doing RF 3.
> >
> >
> > > 3. why do writes to a TX need the same guarantees as "plain" writes? in
> > > cases where the user can live with a TX rollback on change of
> > > leadership/broker crash the TX log can be unreplicated, and even live
> in
> > > the leader's memory. that would cut down on writes. this is also an
> > > acceptable default in SQL - if your socket connection to a DB dies
> mid-TX
> > > your TX is toast (mysql is even worse)
> > >
> >
> > I may have misunderstood - while the above may be true for transactions
> > in-flight, it definitely needs the same guarantees at the point of commit
> > and the straightforward way to achieve that is to rely on the same
> > guarantees while the transaction is in flight.
> >
> > 4. even if we replicate the TX log, why do we need to re-read it and
> > > re-write it to the underlying partition? if its already written to disk
> > all
> > > I would need is to make that file the current segment of the "real"
> > > partition and i've avoided the double write (at the cost of
> complicating
> > > segment management). if the data is replicated fetchers could do the
> > same.
> > >
> >
> > I think we had considered the above as well - i.e., if you abstract the
> > partition's segments into segments that contain non-transactional
> messages
> > and those that contain transactional messages then it should be possible
> to
> > jump from one to the other and back. It does add quite a bit of
> complexity
> > though and you still need to do buffering on reads so the upside perhaps
> > isn't worth the effort. I'm not convinced about that though - i.e., may
> > help to spend more time thinking this one through.
> >
> >
> > > 5. on latency - youre right, what im suggesting would result in tx
> > ordering
> > > of messages ,"read committed" semantics and therefore higher latency.
> >
> >
> > *"read committed"* only if you do the copy back to actual log. If you
> don't
> > do that (your point 4) then I think you still need to do buffering to
> > achieve read-committed semantics.
> >
> >
> >
> > > 6. the added delay (vs your read uncommitted) would be roughly the time
> > > span of a TX.
> >
> >
> > I think it would be significantly less given that this is local copying.
> >
> >
> >
> > >
> > >
> > > On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > One more thing about the double journal proposal: when discussing
> about
> > > > this method back at LinkedIn, another raised issue besides double
> > writing
> > > > was that it will void the offset ordering and enforce people to
> accept
> > > > "transaction ordering", that is, consumer will not see messages from
> > the
> > > > same partition in the order where they were produced, but only in the
> > > order
> > > > of when the corresponding transaction was committed. For some
> > scenarios,
> > > we
> > > > believe that offset ordering would still be preferred than
> transaction
> > > > ordering and that is why in KIP-98 proposal we default to the former
> > > while
> > > > leave the door open if users want to switch to the latter case.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > >
> > > > > Hey Radai,
> > > > >
> > > > > I'm not sure if I fully understand what you are proposing, but I
> > > > > interpreted it to be similar to a proposal we worked through back
> at
> > > > > LinkedIn. The proposal was to commit to a central txlog topic, and
> > then
> > > > > recopy to the destination topic upon transaction commit. The
> > > observation
> > > > on
> > > > > that approach at the time were the following:
> > > > >
> > > > >    1. It is cleaner since the output topics have only committed
> data!
> > > > >    2. You need full replication on the txlog topic to ensure
> > atomicity.
> > > > We
> > > > >    weren't able to come up with a solution where you buffer in
> memory
> > > or
> > > > > use
> > > > >    renaming tricks the way you are describing. The reason is that
> > once
> > > > you
> > > > >    begin committing you must ensure that the commit eventually
> > succeeds
> > > > to
> > > > >    guarantee atomicity. If you use a transient store you might
> commit
> > > > some
> > > > >    data and then have a server failure that causes you to lose the
> > rest
> > > > of
> > > > > the
> > > > >    transaction.
> > > > >    3. Having a single log allows the reader to choose a "read
> > > > uncommitted"
> > > > >    mode that hands out messages immediately. This is important for
> > > cases
> > > > > where
> > > > >    latency is important, especially for stream processing
> topologies
> > > > where
> > > > >    these latencies stack up across multiple stages.
> > > > >
> > > > > For the stream processing use case, item (2) is a bit of a deal
> > killer.
> > > > > This takes the cost of a transient message write (say the
> > intermediate
> > > > > result of a stream processing topology) from 3x writes (assuming 3x
> > > > > replication) to 6x writes. This means you basically can't default
> it
> > > on.
> > > > If
> > > > > we can in fact get the cost down to a single buffered write (i.e.
> 1x
> > > the
> > > > > data is written to memory and buffered to disk if the transaction
> is
> > > > large)
> > > > > as in the KIP-98 proposal without too many other negative side
> > effects
> > > I
> > > > > think that could be compelling.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Dec 19, 2016 at 9:36 AM, radai <radai.rosenblatt@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > regarding efficiency:
> > > > > >
> > > > > > I'd like to distinguish between server efficiency (resource
> > > utilization
> > > > > of
> > > > > > the broker machine alone) and overall network efficiency
> (resource
> > > > > > utilization on brokers, producers and consumers, including
> network
> > > > > > traffic).
> > > > > > my proposal is not as resource-efficient on the broker (although
> it
> > > can
> > > > > be,
> > > > > > depends on a few trade offs and implementation details). HOWEVER,
> > if
> > > i
> > > > > look
> > > > > > at the overall efficiency:
> > > > > >
> > > > > >    1.clients would need to either buffer or double-read
> uncommitted
> > > > msgs.
> > > > > > for N clients reading the stream M times (after re-starts and
> > > > reconsumes)
> > > > > > this would mean a M*N factor in either network BW or disk/memory
> > > space
> > > > > > (depends on if buffer vs re-read). potentially N*M more
> broker-side
> > > > reads
> > > > > > too.
> > > > > >    2 to reduce the broker side cost several things can be done
> > (this
> > > is
> > > > > not
> > > > > > an either-or list, these are commulative):
> > > > > >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk
> > > > writes
> > > > > > for TX resiliency
> > > > > >       2.2 - when "appending" TX logs to real partitions - instead
> > of
> > > > > > reading from (disk-based) TX log and writing to partition log (x2
> > > disk
> > > > > > writes) the TX log can be made a segment file (so file rename,
> with
> > > > > > associated protocol changes). this would avoid double writing by
> > > simply
> > > > > > making the TX file part of the partition (for large enough TXs.
> > > smaller
> > > > > > ones can be rewritten).
> > > > > >       2.3 - the approach above could be combined with a
> background
> > > > > "defrag"
> > > > > > - similar in concept to compaction - to further reduce the total
> of
> > > > > > resulting number of files.
> > > > > >
> > > > > > I think my main issue with the current proposal, more important
> > than
> > > > > > performance, is lack of proper "encapsulation" of transactions -
> I
> > > dont
> > > > > > think downstream consumers should see uncommitted msgs. ever.
> > > > > >
> > > > > >
> > > > > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <
> becket.qin@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > @Jason
> > > > > > >
> > > > > > > Yes, second thought on the number of messages included, the
> > offset
> > > > > delta
> > > > > > > will probably be sufficient. The use case I encounter before
> for
> > > > number
> > > > > > of
> > > > > > > messages in a message set is an embedded mirror maker on the
> > > > > destination
> > > > > > > broker side which fetches message directly from the source
> > cluster.
> > > > > > Ideally
> > > > > > > the destination cluster only needs to check CRC and assign the
> > > > offsets
> > > > > > > because all the message verification has been done by the
> source
> > > > > cluster,
> > > > > > > but due to the lack of the number of messages in the message
> set,
> > > we
> > > > > have
> > > > > > > to decompress the message set to increment offsets correctly.
> By
> > > > > knowing
> > > > > > > the number of the messages in the message set, we can avoid
> doing
> > > > that.
> > > > > > The
> > > > > > > offset delta will also help. It's just then the offsets may
> have
> > > > holes
> > > > > > for
> > > > > > > log compacted topics, but that may be fine.
> > > > > > >
> > > > > > > @Apurva
> > > > > > >
> > > > > > > I am not sure if it is true that the consumer will either
> deliver
> > > all
> > > > > the
> > > > > > > message for the entire transaction or none of them from one
> > poll()
> > > > > call.
> > > > > > If
> > > > > > > we allow the transactions to be across partitions, unless the
> > > > consumer
> > > > > > > consumes from all the partitions involved in a transactions, it
> > > seems
> > > > > > > impossible for it to deliver *all* the messages in a
> transaction,
> > > > > right?
> > > > > > A
> > > > > > > weaker guarantee is we will deliver all or none of the messages
> > > that
> > > > > > belong
> > > > > > > to the same transaction in ONE partition, but this would be
> > > different
> > > > > > from
> > > > > > > the guarantee from the producer side.
> > > > > > >
> > > > > > > My two cents on Radai's sideways partition design:
> > > > > > > 1. If we consider the producer side behavior as doing a two
> phase
> > > > > commit
> > > > > > > which including the committing the consumer offsets, it is a
> > little
> > > > > > awkward
> > > > > > > that we allow uncommitted message goes into the main log and
> rely
> > > on
> > > > > the
> > > > > > > consumer to filter out. So semantic wise I think it would be
> > better
> > > > if
> > > > > we
> > > > > > > can avoid this. Radai's suggestion is actually intuitive
> because
> > if
> > > > the
> > > > > > > brokers do not want to expose uncommitted transactions to the
> > > > consumer,
> > > > > > the
> > > > > > > brokers have to buffer it.
> > > > > > >
> > > > > > > 2. Regarding the efficiency. I think may be it worth looking at
> > the
> > > > > > > efficiency cost v.s benefit. The efficiency includes both
> server
> > > side
> > > > > > > efficiency and consumer side efficiency.
> > > > > > >
> > > > > > > Regarding the server side efficiency, the current proposal
> would
> > > > > probably
> > > > > > > have better efficiency regardless of whether something goes
> > wrong.
> > > > > > Radai's
> > > > > > > suggestion would put more burden on the server side. If nothing
> > > goes
> > > > > > wrong
> > > > > > > we always pay the cost of having double copy of the
> transactional
> > > > > > messages
> > > > > > > and do not get the semantic benefit. But if something goes
> wrong,
> > > the
> > > > > > > efficiency cost we pay we get us a better semantic.
> > > > > > >
> > > > > > > For the consumer side efficiency, because there is no need to
> > > buffer
> > > > > the
> > > > > > > uncommitted messages. The current proposal may have to
> > potentially
> > > > > buffer
> > > > > > > uncommitted messages so it would be less efficient than Radai's
> > > > > > suggestion
> > > > > > > when a transaction aborts. When everything goes well, both
> design
> > > > seems
> > > > > > > having the similar performance. However, it depends on whether
> we
> > > are
> > > > > > > willing to loosen the consumer side transaction guarantee that
> I
> > > > > > mentioned
> > > > > > > earlier to Apurva.
> > > > > > >
> > > > > > > Currently the biggest pressure on the consumer side is that it
> > has
> > > to
> > > > > > > buffer incomplete transactions. There are two reasons for it,
> > > > > > > A. A transaction may be aborted so we cannot expose the
> messages
> > to
> > > > the
> > > > > > > users.
> > > > > > > B. We want to return all or none of the messages in a
> transaction
> > > in
> > > > > ONE
> > > > > > > partition.
> > > > > > >
> > > > > > > While reason A is mandatory, I think reason B may be
> discussable.
> > > > > Radai's
> > > > > > > design actually removes reason A because there is no
> uncommitted
> > > > > messages
> > > > > > > exposed to the consumers. This may potentially give us a chance
> > to
> > > > > > > significantly improve consumer side efficiency in normal cases.
> > It
> > > > > again
> > > > > > > depends on the use case, i.e. whether user can process a
> > > transaction
> > > > > > > progressively (message by message) or it has to be buffered and
> > > > > returned
> > > > > > > all together. If in most cases, users can process the
> > transactions
> > > > > > message
> > > > > > > by message (most stream processing tasks probably can do so),
> > then
> > > > with
> > > > > > > Radai's proposal we don't need to buffer the transactions for
> the
> > > > users
> > > > > > > anymore, which is a big difference. For the latter case, the
> > > consumer
> > > > > may
> > > > > > > have to buffer the incomplete transactions otherwise we are
> just
> > > > > throwing
> > > > > > > the burden onto the users.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jiangjie (Becket) Qin
> > > > > > >
> > > > > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > > > Yeah good point. I relent!
> > > > > > > >
> > > > > > > > -jay
> > > > > > > >
> > > > > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> > > > jason@confluent.io>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Jay/Ismael,
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > I agree that lazy initialization of metadata seems
> > unavoidable.
> > > > > > > Ideally,
> > > > > > > > we
> > > > > > > > >
> > > > > > > > > could follow the same pattern for transactions, but
> remember
> > > that
> > > > > in
> > > > > > > the
> > > > > > > > >
> > > > > > > > > consumer+producer use case, the initialization needs to be
> > > > > completed
> > > > > > > > prior
> > > > > > > > >
> > > > > > > > > to setting the consumer's position. Otherwise we risk
> reading
> > > > stale
> > > > > > > > >
> > > > > > > > > offsets. But it would be pretty awkward if you have to
> begin
> > a
> > > > > > > > transaction
> > > > > > > > >
> > > > > > > > > first to ensure that your consumer can read the right
> offset
> > > from
> > > > > the
> > > > > > > > >
> > > > > > > > > consumer, right? It's a bit easier to explain that you
> should
> > > > > always
> > > > > > > call
> > > > > > > > >
> > > > > > > > > `producer.init()` prior to initializing the consumer. Users
> > > would
> > > > > > > > probably
> > > > > > > > >
> > > > > > > > > get this right without any special effort.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > -Jason
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > > > > rsivaram@pivotal.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > > Hi Apurva,
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > Thank you for the answers. Just one follow-on.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > 15. Let me rephrase my original question. If all control
> > > > messages
> > > > > > > > > (messages
> > > > > > > > >
> > > > > > > > > > to transaction logs and markers on user logs) were
> > > acknowledged
> > > > > > only
> > > > > > > > > after
> > > > > > > > >
> > > > > > > > > > flushing the log segment, will transactions become
> durable
> > in
> > > > the
> > > > > > > > >
> > > > > > > > > > traditional sense (i.e. not restricted to
> > min.insync.replicas
> > > > > > > > failures) ?
> > > > > > > > >
> > > > > > > > > > This is not a suggestion to update the KIP. It seems to
> me
> > > that
> > > > > the
> > > > > > > > > design
> > > > > > > > >
> > > > > > > > > > enables full durability if required in the future with a
> > > rather
> > > > > > > > >
> > > > > > > > > > non-intrusive change. I just wanted to make sure I
> haven't
> > > > missed
> > > > > > > > > anything
> > > > > > > > >
> > > > > > > > > > fundamental that prevents Kafka from doing this.
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <ben@kirw.in
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > Hi Apurva,
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > Thanks for the detailed answers... and sorry for the
> late
> > > > > reply!
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > It does sound like, if the input-partitions-to-app-id
> > > mapping
> > > > > > never
> > > > > > > > >
> > > > > > > > > > > changes, the existing fencing mechanisms should prevent
> > > > > > duplicates.
> > > > > > > > >
> > > > > > > > > > Great!
> > > > > > > > >
> > > > > > > > > > > I'm a bit concerned the proposed API will be delicate
> to
> > > > > program
> > > > > > > > > against
> > > > > > > > >
> > > > > > > > > > > successfully -- even in the simple case, we need to
> > create
> > > a
> > > > > new
> > > > > > > > > producer
> > > > > > > > >
> > > > > > > > > > > instance per input partition, and anything fancier is
> > going
> > > > to
> > > > > > need
> > > > > > > > its
> > > > > > > > >
> > > > > > > > > > own
> > > > > > > > >
> > > > > > > > > > > implementation of the Streams/Samza-style 'task' idea
> --
> > > but
> > > > > that
> > > > > > > may
> > > > > > > > > be
> > > > > > > > >
> > > > > > > > > > > fine for this sort of advanced feature.
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > For the second question, I notice that Jason also
> > > elaborated
> > > > on
> > > > > > > this
> > > > > > > > >
> > > > > > > > > > > downthread:
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > We also looked at removing the producer ID.
> > > > > > > > >
> > > > > > > > > > > > This was discussed somewhere above, but basically the
> > > idea
> > > > is
> > > > > > to
> > > > > > > > > store
> > > > > > > > >
> > > > > > > > > > > the
> > > > > > > > >
> > > > > > > > > > > > AppID in the message set header directly and avoid
> the
> > > > > mapping
> > > > > > to
> > > > > > > > >
> > > > > > > > > > > producer
> > > > > > > > >
> > > > > > > > > > > > ID altogether. As long as batching isn't too bad, the
> > > > impact
> > > > > on
> > > > > > > > total
> > > > > > > > >
> > > > > > > > > > > size
> > > > > > > > >
> > > > > > > > > > > > may not be too bad, but we were ultimately more
> > > comfortable
> > > > > > with
> > > > > > > a
> > > > > > > > >
> > > > > > > > > > fixed
> > > > > > > > >
> > > > > > > > > > > > size ID.
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > ...which suggests that the distinction is useful for
> > > > > performance,
> > > > > > > but
> > > > > > > > > not
> > > > > > > > >
> > > > > > > > > > > necessary for correctness, which makes good sense to
> me.
> > > > > (Would a
> > > > > > > > > 128-bid
> > > > > > > > >
> > > > > > > > > > > ID be a reasonable compromise? That's enough room for a
> > > UUID,
> > > > > or
> > > > > > a
> > > > > > > > >
> > > > > > > > > > > reasonable hash of an arbitrary string, and has only a
> > > > marginal
> > > > > > > > > increase
> > > > > > > > >
> > > > > > > > > > on
> > > > > > > > >
> > > > > > > > > > > the message size.)
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > > > > apurva@confluent.io
> > > > > > > >
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > Hi Ben,
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > Now, on to your first question of how deal with
> > consumer
> > > > > > > > rebalances.
> > > > > > > > >
> > > > > > > > > > The
> > > > > > > > >
> > > > > > > > > > > > short answer is that the application needs to ensure
> > that
> > > > the
> > > > > > the
> > > > > > > > >
> > > > > > > > > > > > assignment of input partitions to appId is consistent
> > > > across
> > > > > > > > >
> > > > > > > > > > rebalances.
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > For Kafka streams, they already ensure that the
> mapping
> > > of
> > > > > > input
> > > > > > > > >
> > > > > > > > > > > partitions
> > > > > > > > >
> > > > > > > > > > > > to task Id is invariant across rebalances by
> > > implementing a
> > > > > > > custom
> > > > > > > > >
> > > > > > > > > > sticky
> > > > > > > > >
> > > > > > > > > > > > assignor. Other non-streams apps can trivially have
> one
> > > > > > producer
> > > > > > > > per
> > > > > > > > >
> > > > > > > > > > > input
> > > > > > > > >
> > > > > > > > > > > > partition and have the appId be the same as the
> > partition
> > > > > > number
> > > > > > > to
> > > > > > > > >
> > > > > > > > > > > achieve
> > > > > > > > >
> > > > > > > > > > > > the same effect.
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > With this precondition in place, we can maintain
> > > > transactions
> > > > > > > > across
> > > > > > > > >
> > > > > > > > > > > > rebalances.
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > Hope this answers your question.
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > > > > Apurva
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <
> > ben@kirw.in>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > Thanks for this! I'm looking forward to going
> through
> > > the
> > > > > > full
> > > > > > > > >
> > > > > > > > > > proposal
> > > > > > > > >
> > > > > > > > > > > > in
> > > > > > > > >
> > > > > > > > > > > > > detail soon; a few early questions:
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > First: what happens when a consumer rebalances in
> the
> > > > > middle
> > > > > > > of a
> > > > > > > > >
> > > > > > > > > > > > > transaction? The full documentation suggests that
> > such
> > > a
> > > > > > > > > transaction
> > > > > > > > >
> > > > > > > > > > > > ought
> > > > > > > > >
> > > > > > > > > > > > > to be rejected:
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > [...] if a rebalance has happened and this
> consumer
> > > > > > > > >
> > > > > > > > > > > > > > instance becomes a zombie, even if this offset
> > > message
> > > > is
> > > > > > > > > appended
> > > > > > > > >
> > > > > > > > > > in
> > > > > > > > >
> > > > > > > > > > > > the
> > > > > > > > >
> > > > > > > > > > > > > > offset topic, the transaction will be rejected
> > later
> > > on
> > > > > > when
> > > > > > > it
> > > > > > > > >
> > > > > > > > > > tries
> > > > > > > > >
> > > > > > > > > > > > to
> > > > > > > > >
> > > > > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > ...but it's unclear to me how we ensure that a
> > > > transaction
> > > > > > > can't
> > > > > > > > >
> > > > > > > > > > > complete
> > > > > > > > >
> > > > > > > > > > > > > if a rebalance has happened. (It's quite possible
> I'm
> > > > > missing
> > > > > > > > >
> > > > > > > > > > something
> > > > > > > > >
> > > > > > > > > > > > > obvious!)
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > As a concrete example: suppose a process with PID 1
> > > adds
> > > > > > > offsets
> > > > > > > > > for
> > > > > > > > >
> > > > > > > > > > > some
> > > > > > > > >
> > > > > > > > > > > > > partition to a transaction; a consumer rebalance
> > > happens
> > > > > that
> > > > > > > > > assigns
> > > > > > > > >
> > > > > > > > > > > the
> > > > > > > > >
> > > > > > > > > > > > > partition to a process with PID 2, which adds some
> > > > offsets
> > > > > to
> > > > > > > its
> > > > > > > > >
> > > > > > > > > > > current
> > > > > > > > >
> > > > > > > > > > > > > transaction; both processes try and commit.
> Allowing
> > > both
> > > > > > > commits
> > > > > > > > >
> > > > > > > > > > would
> > > > > > > > >
> > > > > > > > > > > > > cause the messages to be processed twice -- how is
> > that
> > > > > > > avoided?
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > Second: App IDs normally map to a single PID. It
> > seems
> > > > like
> > > > > > one
> > > > > > > > > could
> > > > > > > > >
> > > > > > > > > > > do
> > > > > > > > >
> > > > > > > > > > > > > away with the PID concept entirely, and just use
> App
> > > IDs
> > > > in
> > > > > > > most
> > > > > > > > >
> > > > > > > > > > places
> > > > > > > > >
> > > > > > > > > > > > > that require a PID. This feels like it would be
> > > > > significantly
> > > > > > > > >
> > > > > > > > > > simpler,
> > > > > > > > >
> > > > > > > > > > > > > though it does increase the message size. Are there
> > > other
> > > > > > > reasons
> > > > > > > > > why
> > > > > > > > >
> > > > > > > > > > > the
> > > > > > > > >
> > > > > > > > > > > > > App ID / PID split is necessary?
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > > > > > wangguoz@gmail.com
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> > > > exactly
> > > > > > once
> > > > > > > > >
> > > > > > > > > > > delivery
> > > > > > > > >
> > > > > > > > > > > > > > semantics:
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > *https://cwiki.apache.org/
> > > > confluence/display/KAFKA/KIP-
> > > > > > > > >
> > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > and+Transactional+Messaging
> > > > > > > > >
> > > > > > > > > > > > > > <https://cwiki.apache.org/
> > > > confluence/display/KAFKA/KIP-
> > > > > > > > >
> > > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > > and+Transactional+Messaging>*
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > This KIP adds a transactional messaging mechanism
> > > along
> > > > > > with
> > > > > > > an
> > > > > > > > >
> > > > > > > > > > > > > idempotent
> > > > > > > > >
> > > > > > > > > > > > > > producer implementation to make sure that 1)
> > > duplicated
> > > > > > > > messages
> > > > > > > > >
> > > > > > > > > > sent
> > > > > > > > >
> > > > > > > > > > > > > from
> > > > > > > > >
> > > > > > > > > > > > > > the same identified producer can be detected on
> the
> > > > > broker
> > > > > > > > side,
> > > > > > > > >
> > > > > > > > > > and
> > > > > > > > >
> > > > > > > > > > > > 2) a
> > > > > > > > >
> > > > > > > > > > > > > > group of messages sent within a transaction will
> > > > > atomically
> > > > > > > be
> > > > > > > > >
> > > > > > > > > > either
> > > > > > > > >
> > > > > > > > > > > > > > reflected and fetchable to consumers or not as a
> > > whole.
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > The above wiki page provides a high-level view of
> > the
> > > > > > > proposed
> > > > > > > > >
> > > > > > > > > > > changes
> > > > > > > > >
> > > > > > > > > > > > as
> > > > > > > > >
> > > > > > > > > > > > > > well as summarized guarantees. Initial draft of
> the
> > > > > > detailed
> > > > > > > > >
> > > > > > > > > > > > > implementation
> > > > > > > > >
> > > > > > > > > > > > > > design is described in this Google doc:
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > > > > >
> > > > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > > >
> > > > > > > > > > > > > > 0wSw9ra8
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > We would love to hear your comments and
> > suggestions.
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > > > -- Guozhang
> > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Cool. It sounds like you guys will sync up and come up with a specific
proposal. I think point (3) does require full replication of the pre-commit
transaction, but I'm not sure, and I would be very happy to learn
otherwise. That was actually the blocker on that alternate proposal. From
my point of view 2x overhead is kind of a deal breaker since it makes
correctness so expensive you'd have to think very hard before turning it
on, but if there is a way to do it with less and there aren't too many
other negative side effects that would be very appealing. I think we can
also dive a bit into why we are so perf and latency sensitive as it relates
to the stream processing use cases...I'm not sure how much of that is
obvious from the proposal.

-Jay





On Tue, Dec 20, 2016 at 9:11 AM, Joel Koshy <jj...@gmail.com> wrote:

> Just got some time to go through most of this thread and KIP - great to see
> this materialize and discussed!!
> I will add more comments in the coming days on some of the other "tracks"
> in this thread; but since Radai brought up the double-journaling approach
> that we had discussed I thought I would move over some content from
> our internal
> wiki on double-journalling
> <https://cwiki.apache.org/confluence/display/KAFKA/
> Double+journaling+with+local+data+copy>
> It is thin on details with a few invalid statements because I don't think
> we dwelt long enough on it - it was cast aside as being too expensive from
> a storage and latency perspective. As the immediately preceding emails
> state, I tend to agree that those are compelling enough reasons to take a
> hit in complexity/increased memory usage in the consumer. Anyway, couple of
> us at LinkedIn can spend some time today brainstorming a little more on
> this today.
>
> 1. on write amplification: i dont see x6 the writes, at worst i see x2 the
> > writes - once to the "tx log", then read and again to the destination
> > partition. if you have some != 1 replication factor than both the 1st and
> > the 2nd writes get replicated, but it is still a relative factor of x2.
> > what am I missing?
> >
>
> I think that's right - it would be six total copies if we are doing RF 3.
>
>
> > 3. why do writes to a TX need the same guarantees as "plain" writes? in
> > cases where the user can live with a TX rollback on change of
> > leadership/broker crash the TX log can be unreplicated, and even live in
> > the leader's memory. that would cut down on writes. this is also an
> > acceptable default in SQL - if your socket connection to a DB dies mid-TX
> > your TX is toast (mysql is even worse)
> >
>
> I may have misunderstood - while the above may be true for transactions
> in-flight, it definitely needs the same guarantees at the point of commit
> and the straightforward way to achieve that is to rely on the same
> guarantees while the transaction is in flight.
>
> 4. even if we replicate the TX log, why do we need to re-read it and
> > re-write it to the underlying partition? if its already written to disk
> all
> > I would need is to make that file the current segment of the "real"
> > partition and i've avoided the double write (at the cost of complicating
> > segment management). if the data is replicated fetchers could do the
> same.
> >
>
> I think we had considered the above as well - i.e., if you abstract the
> partition's segments into segments that contain non-transactional messages
> and those that contain transactional messages then it should be possible to
> jump from one to the other and back. It does add quite a bit of complexity
> though and you still need to do buffering on reads so the upside perhaps
> isn't worth the effort. I'm not convinced about that though - i.e., may
> help to spend more time thinking this one through.
>
>
> > 5. on latency - youre right, what im suggesting would result in tx
> ordering
> > of messages ,"read committed" semantics and therefore higher latency.
>
>
> *"read committed"* only if you do the copy back to actual log. If you don't
> do that (your point 4) then I think you still need to do buffering to
> achieve read-committed semantics.
>
>
>
> > 6. the added delay (vs your read uncommitted) would be roughly the time
> > span of a TX.
>
>
> I think it would be significantly less given that this is local copying.
>
>
>
> >
> >
> > On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > One more thing about the double journal proposal: when discussing about
> > > this method back at LinkedIn, another raised issue besides double
> writing
> > > was that it will void the offset ordering and enforce people to accept
> > > "transaction ordering", that is, consumer will not see messages from
> the
> > > same partition in the order where they were produced, but only in the
> > order
> > > of when the corresponding transaction was committed. For some
> scenarios,
> > we
> > > believe that offset ordering would still be preferred than transaction
> > > ordering and that is why in KIP-98 proposal we default to the former
> > while
> > > leave the door open if users want to switch to the latter case.
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey Radai,
> > > >
> > > > I'm not sure if I fully understand what you are proposing, but I
> > > > interpreted it to be similar to a proposal we worked through back at
> > > > LinkedIn. The proposal was to commit to a central txlog topic, and
> then
> > > > recopy to the destination topic upon transaction commit. The
> > observation
> > > on
> > > > that approach at the time were the following:
> > > >
> > > >    1. It is cleaner since the output topics have only committed data!
> > > >    2. You need full replication on the txlog topic to ensure
> atomicity.
> > > We
> > > >    weren't able to come up with a solution where you buffer in memory
> > or
> > > > use
> > > >    renaming tricks the way you are describing. The reason is that
> once
> > > you
> > > >    begin committing you must ensure that the commit eventually
> succeeds
> > > to
> > > >    guarantee atomicity. If you use a transient store you might commit
> > > some
> > > >    data and then have a server failure that causes you to lose the
> rest
> > > of
> > > > the
> > > >    transaction.
> > > >    3. Having a single log allows the reader to choose a "read
> > > uncommitted"
> > > >    mode that hands out messages immediately. This is important for
> > cases
> > > > where
> > > >    latency is important, especially for stream processing topologies
> > > where
> > > >    these latencies stack up across multiple stages.
> > > >
> > > > For the stream processing use case, item (2) is a bit of a deal
> killer.
> > > > This takes the cost of a transient message write (say the
> intermediate
> > > > result of a stream processing topology) from 3x writes (assuming 3x
> > > > replication) to 6x writes. This means you basically can't default it
> > on.
> > > If
> > > > we can in fact get the cost down to a single buffered write (i.e. 1x
> > the
> > > > data is written to memory and buffered to disk if the transaction is
> > > large)
> > > > as in the KIP-98 proposal without too many other negative side
> effects
> > I
> > > > think that could be compelling.
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > > On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com>
> > > wrote:
> > > >
> > > > > regarding efficiency:
> > > > >
> > > > > I'd like to distinguish between server efficiency (resource
> > utilization
> > > > of
> > > > > the broker machine alone) and overall network efficiency (resource
> > > > > utilization on brokers, producers and consumers, including network
> > > > > traffic).
> > > > > my proposal is not as resource-efficient on the broker (although it
> > can
> > > > be,
> > > > > depends on a few trade offs and implementation details). HOWEVER,
> if
> > i
> > > > look
> > > > > at the overall efficiency:
> > > > >
> > > > >    1.clients would need to either buffer or double-read uncommitted
> > > msgs.
> > > > > for N clients reading the stream M times (after re-starts and
> > > reconsumes)
> > > > > this would mean a M*N factor in either network BW or disk/memory
> > space
> > > > > (depends on if buffer vs re-read). potentially N*M more broker-side
> > > reads
> > > > > too.
> > > > >    2 to reduce the broker side cost several things can be done
> (this
> > is
> > > > not
> > > > > an either-or list, these are commulative):
> > > > >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk
> > > writes
> > > > > for TX resiliency
> > > > >       2.2 - when "appending" TX logs to real partitions - instead
> of
> > > > > reading from (disk-based) TX log and writing to partition log (x2
> > disk
> > > > > writes) the TX log can be made a segment file (so file rename, with
> > > > > associated protocol changes). this would avoid double writing by
> > simply
> > > > > making the TX file part of the partition (for large enough TXs.
> > smaller
> > > > > ones can be rewritten).
> > > > >       2.3 - the approach above could be combined with a background
> > > > "defrag"
> > > > > - similar in concept to compaction - to further reduce the total of
> > > > > resulting number of files.
> > > > >
> > > > > I think my main issue with the current proposal, more important
> than
> > > > > performance, is lack of proper "encapsulation" of transactions - I
> > dont
> > > > > think downstream consumers should see uncommitted msgs. ever.
> > > > >
> > > > >
> > > > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <becket.qin@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > @Jason
> > > > > >
> > > > > > Yes, second thought on the number of messages included, the
> offset
> > > > delta
> > > > > > will probably be sufficient. The use case I encounter before for
> > > number
> > > > > of
> > > > > > messages in a message set is an embedded mirror maker on the
> > > > destination
> > > > > > broker side which fetches message directly from the source
> cluster.
> > > > > Ideally
> > > > > > the destination cluster only needs to check CRC and assign the
> > > offsets
> > > > > > because all the message verification has been done by the source
> > > > cluster,
> > > > > > but due to the lack of the number of messages in the message set,
> > we
> > > > have
> > > > > > to decompress the message set to increment offsets correctly. By
> > > > knowing
> > > > > > the number of the messages in the message set, we can avoid doing
> > > that.
> > > > > The
> > > > > > offset delta will also help. It's just then the offsets may have
> > > holes
> > > > > for
> > > > > > log compacted topics, but that may be fine.
> > > > > >
> > > > > > @Apurva
> > > > > >
> > > > > > I am not sure if it is true that the consumer will either deliver
> > all
> > > > the
> > > > > > message for the entire transaction or none of them from one
> poll()
> > > > call.
> > > > > If
> > > > > > we allow the transactions to be across partitions, unless the
> > > consumer
> > > > > > consumes from all the partitions involved in a transactions, it
> > seems
> > > > > > impossible for it to deliver *all* the messages in a transaction,
> > > > right?
> > > > > A
> > > > > > weaker guarantee is we will deliver all or none of the messages
> > that
> > > > > belong
> > > > > > to the same transaction in ONE partition, but this would be
> > different
> > > > > from
> > > > > > the guarantee from the producer side.
> > > > > >
> > > > > > My two cents on Radai's sideways partition design:
> > > > > > 1. If we consider the producer side behavior as doing a two phase
> > > > commit
> > > > > > which including the committing the consumer offsets, it is a
> little
> > > > > awkward
> > > > > > that we allow uncommitted message goes into the main log and rely
> > on
> > > > the
> > > > > > consumer to filter out. So semantic wise I think it would be
> better
> > > if
> > > > we
> > > > > > can avoid this. Radai's suggestion is actually intuitive because
> if
> > > the
> > > > > > brokers do not want to expose uncommitted transactions to the
> > > consumer,
> > > > > the
> > > > > > brokers have to buffer it.
> > > > > >
> > > > > > 2. Regarding the efficiency. I think may be it worth looking at
> the
> > > > > > efficiency cost v.s benefit. The efficiency includes both server
> > side
> > > > > > efficiency and consumer side efficiency.
> > > > > >
> > > > > > Regarding the server side efficiency, the current proposal would
> > > > probably
> > > > > > have better efficiency regardless of whether something goes
> wrong.
> > > > > Radai's
> > > > > > suggestion would put more burden on the server side. If nothing
> > goes
> > > > > wrong
> > > > > > we always pay the cost of having double copy of the transactional
> > > > > messages
> > > > > > and do not get the semantic benefit. But if something goes wrong,
> > the
> > > > > > efficiency cost we pay we get us a better semantic.
> > > > > >
> > > > > > For the consumer side efficiency, because there is no need to
> > buffer
> > > > the
> > > > > > uncommitted messages. The current proposal may have to
> potentially
> > > > buffer
> > > > > > uncommitted messages so it would be less efficient than Radai's
> > > > > suggestion
> > > > > > when a transaction aborts. When everything goes well, both design
> > > seems
> > > > > > having the similar performance. However, it depends on whether we
> > are
> > > > > > willing to loosen the consumer side transaction guarantee that I
> > > > > mentioned
> > > > > > earlier to Apurva.
> > > > > >
> > > > > > Currently the biggest pressure on the consumer side is that it
> has
> > to
> > > > > > buffer incomplete transactions. There are two reasons for it,
> > > > > > A. A transaction may be aborted so we cannot expose the messages
> to
> > > the
> > > > > > users.
> > > > > > B. We want to return all or none of the messages in a transaction
> > in
> > > > ONE
> > > > > > partition.
> > > > > >
> > > > > > While reason A is mandatory, I think reason B may be discussable.
> > > > Radai's
> > > > > > design actually removes reason A because there is no uncommitted
> > > > messages
> > > > > > exposed to the consumers. This may potentially give us a chance
> to
> > > > > > significantly improve consumer side efficiency in normal cases.
> It
> > > > again
> > > > > > depends on the use case, i.e. whether user can process a
> > transaction
> > > > > > progressively (message by message) or it has to be buffered and
> > > > returned
> > > > > > all together. If in most cases, users can process the
> transactions
> > > > > message
> > > > > > by message (most stream processing tasks probably can do so),
> then
> > > with
> > > > > > Radai's proposal we don't need to buffer the transactions for the
> > > users
> > > > > > anymore, which is a big difference. For the latter case, the
> > consumer
> > > > may
> > > > > > have to buffer the incomplete transactions otherwise we are just
> > > > throwing
> > > > > > the burden onto the users.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jiangjie (Becket) Qin
> > > > > >
> > > > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > > > >
> > > > > > > Yeah good point. I relent!
> > > > > > >
> > > > > > > -jay
> > > > > > >
> > > > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> > > jason@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Jay/Ismael,
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > I agree that lazy initialization of metadata seems
> unavoidable.
> > > > > > Ideally,
> > > > > > > we
> > > > > > > >
> > > > > > > > could follow the same pattern for transactions, but remember
> > that
> > > > in
> > > > > > the
> > > > > > > >
> > > > > > > > consumer+producer use case, the initialization needs to be
> > > > completed
> > > > > > > prior
> > > > > > > >
> > > > > > > > to setting the consumer's position. Otherwise we risk reading
> > > stale
> > > > > > > >
> > > > > > > > offsets. But it would be pretty awkward if you have to begin
> a
> > > > > > > transaction
> > > > > > > >
> > > > > > > > first to ensure that your consumer can read the right offset
> > from
> > > > the
> > > > > > > >
> > > > > > > > consumer, right? It's a bit easier to explain that you should
> > > > always
> > > > > > call
> > > > > > > >
> > > > > > > > `producer.init()` prior to initializing the consumer. Users
> > would
> > > > > > > probably
> > > > > > > >
> > > > > > > > get this right without any special effort.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > -Jason
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > > > rsivaram@pivotal.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > > Hi Apurva,
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > > Thank you for the answers. Just one follow-on.
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > > 15. Let me rephrase my original question. If all control
> > > messages
> > > > > > > > (messages
> > > > > > > >
> > > > > > > > > to transaction logs and markers on user logs) were
> > acknowledged
> > > > > only
> > > > > > > > after
> > > > > > > >
> > > > > > > > > flushing the log segment, will transactions become durable
> in
> > > the
> > > > > > > >
> > > > > > > > > traditional sense (i.e. not restricted to
> min.insync.replicas
> > > > > > > failures) ?
> > > > > > > >
> > > > > > > > > This is not a suggestion to update the KIP. It seems to me
> > that
> > > > the
> > > > > > > > design
> > > > > > > >
> > > > > > > > > enables full durability if required in the future with a
> > rather
> > > > > > > >
> > > > > > > > > non-intrusive change. I just wanted to make sure I haven't
> > > missed
> > > > > > > > anything
> > > > > > > >
> > > > > > > > > fundamental that prevents Kafka from doing this.
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in>
> > > wrote:
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > > > > Hi Apurva,
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > Thanks for the detailed answers... and sorry for the late
> > > > reply!
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > It does sound like, if the input-partitions-to-app-id
> > mapping
> > > > > never
> > > > > > > >
> > > > > > > > > > changes, the existing fencing mechanisms should prevent
> > > > > duplicates.
> > > > > > > >
> > > > > > > > > Great!
> > > > > > > >
> > > > > > > > > > I'm a bit concerned the proposed API will be delicate to
> > > > program
> > > > > > > > against
> > > > > > > >
> > > > > > > > > > successfully -- even in the simple case, we need to
> create
> > a
> > > > new
> > > > > > > > producer
> > > > > > > >
> > > > > > > > > > instance per input partition, and anything fancier is
> going
> > > to
> > > > > need
> > > > > > > its
> > > > > > > >
> > > > > > > > > own
> > > > > > > >
> > > > > > > > > > implementation of the Streams/Samza-style 'task' idea --
> > but
> > > > that
> > > > > > may
> > > > > > > > be
> > > > > > > >
> > > > > > > > > > fine for this sort of advanced feature.
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > For the second question, I notice that Jason also
> > elaborated
> > > on
> > > > > > this
> > > > > > > >
> > > > > > > > > > downthread:
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > We also looked at removing the producer ID.
> > > > > > > >
> > > > > > > > > > > This was discussed somewhere above, but basically the
> > idea
> > > is
> > > > > to
> > > > > > > > store
> > > > > > > >
> > > > > > > > > > the
> > > > > > > >
> > > > > > > > > > > AppID in the message set header directly and avoid the
> > > > mapping
> > > > > to
> > > > > > > >
> > > > > > > > > > producer
> > > > > > > >
> > > > > > > > > > > ID altogether. As long as batching isn't too bad, the
> > > impact
> > > > on
> > > > > > > total
> > > > > > > >
> > > > > > > > > > size
> > > > > > > >
> > > > > > > > > > > may not be too bad, but we were ultimately more
> > comfortable
> > > > > with
> > > > > > a
> > > > > > > >
> > > > > > > > > fixed
> > > > > > > >
> > > > > > > > > > > size ID.
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > ...which suggests that the distinction is useful for
> > > > performance,
> > > > > > but
> > > > > > > > not
> > > > > > > >
> > > > > > > > > > necessary for correctness, which makes good sense to me.
> > > > (Would a
> > > > > > > > 128-bid
> > > > > > > >
> > > > > > > > > > ID be a reasonable compromise? That's enough room for a
> > UUID,
> > > > or
> > > > > a
> > > > > > > >
> > > > > > > > > > reasonable hash of an arbitrary string, and has only a
> > > marginal
> > > > > > > > increase
> > > > > > > >
> > > > > > > > > on
> > > > > > > >
> > > > > > > > > > the message size.)
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > > > apurva@confluent.io
> > > > > > >
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > Hi Ben,
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > Now, on to your first question of how deal with
> consumer
> > > > > > > rebalances.
> > > > > > > >
> > > > > > > > > The
> > > > > > > >
> > > > > > > > > > > short answer is that the application needs to ensure
> that
> > > the
> > > > > the
> > > > > > > >
> > > > > > > > > > > assignment of input partitions to appId is consistent
> > > across
> > > > > > > >
> > > > > > > > > rebalances.
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > For Kafka streams, they already ensure that the mapping
> > of
> > > > > input
> > > > > > > >
> > > > > > > > > > partitions
> > > > > > > >
> > > > > > > > > > > to task Id is invariant across rebalances by
> > implementing a
> > > > > > custom
> > > > > > > >
> > > > > > > > > sticky
> > > > > > > >
> > > > > > > > > > > assignor. Other non-streams apps can trivially have one
> > > > > producer
> > > > > > > per
> > > > > > > >
> > > > > > > > > > input
> > > > > > > >
> > > > > > > > > > > partition and have the appId be the same as the
> partition
> > > > > number
> > > > > > to
> > > > > > > >
> > > > > > > > > > achieve
> > > > > > > >
> > > > > > > > > > > the same effect.
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > With this precondition in place, we can maintain
> > > transactions
> > > > > > > across
> > > > > > > >
> > > > > > > > > > > rebalances.
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > Hope this answers your question.
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > > > > Apurva
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <
> ben@kirw.in>
> > > > > wrote:
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > Thanks for this! I'm looking forward to going through
> > the
> > > > > full
> > > > > > > >
> > > > > > > > > proposal
> > > > > > > >
> > > > > > > > > > > in
> > > > > > > >
> > > > > > > > > > > > detail soon; a few early questions:
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > First: what happens when a consumer rebalances in the
> > > > middle
> > > > > > of a
> > > > > > > >
> > > > > > > > > > > > transaction? The full documentation suggests that
> such
> > a
> > > > > > > > transaction
> > > > > > > >
> > > > > > > > > > > ought
> > > > > > > >
> > > > > > > > > > > > to be rejected:
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > [...] if a rebalance has happened and this consumer
> > > > > > > >
> > > > > > > > > > > > > instance becomes a zombie, even if this offset
> > message
> > > is
> > > > > > > > appended
> > > > > > > >
> > > > > > > > > in
> > > > > > > >
> > > > > > > > > > > the
> > > > > > > >
> > > > > > > > > > > > > offset topic, the transaction will be rejected
> later
> > on
> > > > > when
> > > > > > it
> > > > > > > >
> > > > > > > > > tries
> > > > > > > >
> > > > > > > > > > > to
> > > > > > > >
> > > > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > ...but it's unclear to me how we ensure that a
> > > transaction
> > > > > > can't
> > > > > > > >
> > > > > > > > > > complete
> > > > > > > >
> > > > > > > > > > > > if a rebalance has happened. (It's quite possible I'm
> > > > missing
> > > > > > > >
> > > > > > > > > something
> > > > > > > >
> > > > > > > > > > > > obvious!)
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > As a concrete example: suppose a process with PID 1
> > adds
> > > > > > offsets
> > > > > > > > for
> > > > > > > >
> > > > > > > > > > some
> > > > > > > >
> > > > > > > > > > > > partition to a transaction; a consumer rebalance
> > happens
> > > > that
> > > > > > > > assigns
> > > > > > > >
> > > > > > > > > > the
> > > > > > > >
> > > > > > > > > > > > partition to a process with PID 2, which adds some
> > > offsets
> > > > to
> > > > > > its
> > > > > > > >
> > > > > > > > > > current
> > > > > > > >
> > > > > > > > > > > > transaction; both processes try and commit. Allowing
> > both
> > > > > > commits
> > > > > > > >
> > > > > > > > > would
> > > > > > > >
> > > > > > > > > > > > cause the messages to be processed twice -- how is
> that
> > > > > > avoided?
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > Second: App IDs normally map to a single PID. It
> seems
> > > like
> > > > > one
> > > > > > > > could
> > > > > > > >
> > > > > > > > > > do
> > > > > > > >
> > > > > > > > > > > > away with the PID concept entirely, and just use App
> > IDs
> > > in
> > > > > > most
> > > > > > > >
> > > > > > > > > places
> > > > > > > >
> > > > > > > > > > > > that require a PID. This feels like it would be
> > > > significantly
> > > > > > > >
> > > > > > > > > simpler,
> > > > > > > >
> > > > > > > > > > > > though it does increase the message size. Are there
> > other
> > > > > > reasons
> > > > > > > > why
> > > > > > > >
> > > > > > > > > > the
> > > > > > > >
> > > > > > > > > > > > App ID / PID split is necessary?
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > > > > wangguoz@gmail.com
> > > > > > > > >
> > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> > > exactly
> > > > > once
> > > > > > > >
> > > > > > > > > > delivery
> > > > > > > >
> > > > > > > > > > > > > semantics:
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > *https://cwiki.apache.org/
> > > confluence/display/KAFKA/KIP-
> > > > > > > >
> > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > and+Transactional+Messaging
> > > > > > > >
> > > > > > > > > > > > > <https://cwiki.apache.org/
> > > confluence/display/KAFKA/KIP-
> > > > > > > >
> > > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > > and+Transactional+Messaging>*
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > This KIP adds a transactional messaging mechanism
> > along
> > > > > with
> > > > > > an
> > > > > > > >
> > > > > > > > > > > > idempotent
> > > > > > > >
> > > > > > > > > > > > > producer implementation to make sure that 1)
> > duplicated
> > > > > > > messages
> > > > > > > >
> > > > > > > > > sent
> > > > > > > >
> > > > > > > > > > > > from
> > > > > > > >
> > > > > > > > > > > > > the same identified producer can be detected on the
> > > > broker
> > > > > > > side,
> > > > > > > >
> > > > > > > > > and
> > > > > > > >
> > > > > > > > > > > 2) a
> > > > > > > >
> > > > > > > > > > > > > group of messages sent within a transaction will
> > > > atomically
> > > > > > be
> > > > > > > >
> > > > > > > > > either
> > > > > > > >
> > > > > > > > > > > > > reflected and fetchable to consumers or not as a
> > whole.
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > The above wiki page provides a high-level view of
> the
> > > > > > proposed
> > > > > > > >
> > > > > > > > > > changes
> > > > > > > >
> > > > > > > > > > > as
> > > > > > > >
> > > > > > > > > > > > > well as summarized guarantees. Initial draft of the
> > > > > detailed
> > > > > > > >
> > > > > > > > > > > > implementation
> > > > > > > >
> > > > > > > > > > > > > design is described in this Google doc:
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > > > >
> > > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > > >
> > > > > > > > > > > > > 0wSw9ra8
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > We would love to hear your comments and
> suggestions.
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > > > -- Guozhang
> > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Joel Koshy <jj...@gmail.com>.
Just got some time to go through most of this thread and KIP - great to see
this materialize and discussed!!
I will add more comments in the coming days on some of the other "tracks"
in this thread; but since Radai brought up the double-journaling approach
that we had discussed I thought I would move over some content from
our internal
wiki on double-journalling
<https://cwiki.apache.org/confluence/display/KAFKA/Double+journaling+with+local+data+copy>
It is thin on details with a few invalid statements because I don't think
we dwelt long enough on it - it was cast aside as being too expensive from
a storage and latency perspective. As the immediately preceding emails
state, I tend to agree that those are compelling enough reasons to take a
hit in complexity/increased memory usage in the consumer. Anyway, couple of
us at LinkedIn can spend some time today brainstorming a little more on
this today.

1. on write amplification: i dont see x6 the writes, at worst i see x2 the
> writes - once to the "tx log", then read and again to the destination
> partition. if you have some != 1 replication factor than both the 1st and
> the 2nd writes get replicated, but it is still a relative factor of x2.
> what am I missing?
>

I think that's right - it would be six total copies if we are doing RF 3.


> 3. why do writes to a TX need the same guarantees as "plain" writes? in
> cases where the user can live with a TX rollback on change of
> leadership/broker crash the TX log can be unreplicated, and even live in
> the leader's memory. that would cut down on writes. this is also an
> acceptable default in SQL - if your socket connection to a DB dies mid-TX
> your TX is toast (mysql is even worse)
>

I may have misunderstood - while the above may be true for transactions
in-flight, it definitely needs the same guarantees at the point of commit
and the straightforward way to achieve that is to rely on the same
guarantees while the transaction is in flight.

4. even if we replicate the TX log, why do we need to re-read it and
> re-write it to the underlying partition? if its already written to disk all
> I would need is to make that file the current segment of the "real"
> partition and i've avoided the double write (at the cost of complicating
> segment management). if the data is replicated fetchers could do the same.
>

I think we had considered the above as well - i.e., if you abstract the
partition's segments into segments that contain non-transactional messages
and those that contain transactional messages then it should be possible to
jump from one to the other and back. It does add quite a bit of complexity
though and you still need to do buffering on reads so the upside perhaps
isn't worth the effort. I'm not convinced about that though - i.e., may
help to spend more time thinking this one through.


> 5. on latency - youre right, what im suggesting would result in tx ordering
> of messages ,"read committed" semantics and therefore higher latency.


*"read committed"* only if you do the copy back to actual log. If you don't
do that (your point 4) then I think you still need to do buffering to
achieve read-committed semantics.



> 6. the added delay (vs your read uncommitted) would be roughly the time
> span of a TX.


I think it would be significantly less given that this is local copying.



>
>
> On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > One more thing about the double journal proposal: when discussing about
> > this method back at LinkedIn, another raised issue besides double writing
> > was that it will void the offset ordering and enforce people to accept
> > "transaction ordering", that is, consumer will not see messages from the
> > same partition in the order where they were produced, but only in the
> order
> > of when the corresponding transaction was committed. For some scenarios,
> we
> > believe that offset ordering would still be preferred than transaction
> > ordering and that is why in KIP-98 proposal we default to the former
> while
> > leave the door open if users want to switch to the latter case.
> >
> >
> > Guozhang
> >
> > On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Radai,
> > >
> > > I'm not sure if I fully understand what you are proposing, but I
> > > interpreted it to be similar to a proposal we worked through back at
> > > LinkedIn. The proposal was to commit to a central txlog topic, and then
> > > recopy to the destination topic upon transaction commit. The
> observation
> > on
> > > that approach at the time were the following:
> > >
> > >    1. It is cleaner since the output topics have only committed data!
> > >    2. You need full replication on the txlog topic to ensure atomicity.
> > We
> > >    weren't able to come up with a solution where you buffer in memory
> or
> > > use
> > >    renaming tricks the way you are describing. The reason is that once
> > you
> > >    begin committing you must ensure that the commit eventually succeeds
> > to
> > >    guarantee atomicity. If you use a transient store you might commit
> > some
> > >    data and then have a server failure that causes you to lose the rest
> > of
> > > the
> > >    transaction.
> > >    3. Having a single log allows the reader to choose a "read
> > uncommitted"
> > >    mode that hands out messages immediately. This is important for
> cases
> > > where
> > >    latency is important, especially for stream processing topologies
> > where
> > >    these latencies stack up across multiple stages.
> > >
> > > For the stream processing use case, item (2) is a bit of a deal killer.
> > > This takes the cost of a transient message write (say the intermediate
> > > result of a stream processing topology) from 3x writes (assuming 3x
> > > replication) to 6x writes. This means you basically can't default it
> on.
> > If
> > > we can in fact get the cost down to a single buffered write (i.e. 1x
> the
> > > data is written to memory and buffered to disk if the transaction is
> > large)
> > > as in the KIP-98 proposal without too many other negative side effects
> I
> > > think that could be compelling.
> > >
> > > -Jay
> > >
> > >
> > >
> > > On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com>
> > wrote:
> > >
> > > > regarding efficiency:
> > > >
> > > > I'd like to distinguish between server efficiency (resource
> utilization
> > > of
> > > > the broker machine alone) and overall network efficiency (resource
> > > > utilization on brokers, producers and consumers, including network
> > > > traffic).
> > > > my proposal is not as resource-efficient on the broker (although it
> can
> > > be,
> > > > depends on a few trade offs and implementation details). HOWEVER, if
> i
> > > look
> > > > at the overall efficiency:
> > > >
> > > >    1.clients would need to either buffer or double-read uncommitted
> > msgs.
> > > > for N clients reading the stream M times (after re-starts and
> > reconsumes)
> > > > this would mean a M*N factor in either network BW or disk/memory
> space
> > > > (depends on if buffer vs re-read). potentially N*M more broker-side
> > reads
> > > > too.
> > > >    2 to reduce the broker side cost several things can be done (this
> is
> > > not
> > > > an either-or list, these are commulative):
> > > >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk
> > writes
> > > > for TX resiliency
> > > >       2.2 - when "appending" TX logs to real partitions - instead of
> > > > reading from (disk-based) TX log and writing to partition log (x2
> disk
> > > > writes) the TX log can be made a segment file (so file rename, with
> > > > associated protocol changes). this would avoid double writing by
> simply
> > > > making the TX file part of the partition (for large enough TXs.
> smaller
> > > > ones can be rewritten).
> > > >       2.3 - the approach above could be combined with a background
> > > "defrag"
> > > > - similar in concept to compaction - to further reduce the total of
> > > > resulting number of files.
> > > >
> > > > I think my main issue with the current proposal, more important than
> > > > performance, is lack of proper "encapsulation" of transactions - I
> dont
> > > > think downstream consumers should see uncommitted msgs. ever.
> > > >
> > > >
> > > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com>
> > > wrote:
> > > >
> > > > > @Jason
> > > > >
> > > > > Yes, second thought on the number of messages included, the offset
> > > delta
> > > > > will probably be sufficient. The use case I encounter before for
> > number
> > > > of
> > > > > messages in a message set is an embedded mirror maker on the
> > > destination
> > > > > broker side which fetches message directly from the source cluster.
> > > > Ideally
> > > > > the destination cluster only needs to check CRC and assign the
> > offsets
> > > > > because all the message verification has been done by the source
> > > cluster,
> > > > > but due to the lack of the number of messages in the message set,
> we
> > > have
> > > > > to decompress the message set to increment offsets correctly. By
> > > knowing
> > > > > the number of the messages in the message set, we can avoid doing
> > that.
> > > > The
> > > > > offset delta will also help. It's just then the offsets may have
> > holes
> > > > for
> > > > > log compacted topics, but that may be fine.
> > > > >
> > > > > @Apurva
> > > > >
> > > > > I am not sure if it is true that the consumer will either deliver
> all
> > > the
> > > > > message for the entire transaction or none of them from one poll()
> > > call.
> > > > If
> > > > > we allow the transactions to be across partitions, unless the
> > consumer
> > > > > consumes from all the partitions involved in a transactions, it
> seems
> > > > > impossible for it to deliver *all* the messages in a transaction,
> > > right?
> > > > A
> > > > > weaker guarantee is we will deliver all or none of the messages
> that
> > > > belong
> > > > > to the same transaction in ONE partition, but this would be
> different
> > > > from
> > > > > the guarantee from the producer side.
> > > > >
> > > > > My two cents on Radai's sideways partition design:
> > > > > 1. If we consider the producer side behavior as doing a two phase
> > > commit
> > > > > which including the committing the consumer offsets, it is a little
> > > > awkward
> > > > > that we allow uncommitted message goes into the main log and rely
> on
> > > the
> > > > > consumer to filter out. So semantic wise I think it would be better
> > if
> > > we
> > > > > can avoid this. Radai's suggestion is actually intuitive because if
> > the
> > > > > brokers do not want to expose uncommitted transactions to the
> > consumer,
> > > > the
> > > > > brokers have to buffer it.
> > > > >
> > > > > 2. Regarding the efficiency. I think may be it worth looking at the
> > > > > efficiency cost v.s benefit. The efficiency includes both server
> side
> > > > > efficiency and consumer side efficiency.
> > > > >
> > > > > Regarding the server side efficiency, the current proposal would
> > > probably
> > > > > have better efficiency regardless of whether something goes wrong.
> > > > Radai's
> > > > > suggestion would put more burden on the server side. If nothing
> goes
> > > > wrong
> > > > > we always pay the cost of having double copy of the transactional
> > > > messages
> > > > > and do not get the semantic benefit. But if something goes wrong,
> the
> > > > > efficiency cost we pay we get us a better semantic.
> > > > >
> > > > > For the consumer side efficiency, because there is no need to
> buffer
> > > the
> > > > > uncommitted messages. The current proposal may have to potentially
> > > buffer
> > > > > uncommitted messages so it would be less efficient than Radai's
> > > > suggestion
> > > > > when a transaction aborts. When everything goes well, both design
> > seems
> > > > > having the similar performance. However, it depends on whether we
> are
> > > > > willing to loosen the consumer side transaction guarantee that I
> > > > mentioned
> > > > > earlier to Apurva.
> > > > >
> > > > > Currently the biggest pressure on the consumer side is that it has
> to
> > > > > buffer incomplete transactions. There are two reasons for it,
> > > > > A. A transaction may be aborted so we cannot expose the messages to
> > the
> > > > > users.
> > > > > B. We want to return all or none of the messages in a transaction
> in
> > > ONE
> > > > > partition.
> > > > >
> > > > > While reason A is mandatory, I think reason B may be discussable.
> > > Radai's
> > > > > design actually removes reason A because there is no uncommitted
> > > messages
> > > > > exposed to the consumers. This may potentially give us a chance to
> > > > > significantly improve consumer side efficiency in normal cases. It
> > > again
> > > > > depends on the use case, i.e. whether user can process a
> transaction
> > > > > progressively (message by message) or it has to be buffered and
> > > returned
> > > > > all together. If in most cases, users can process the transactions
> > > > message
> > > > > by message (most stream processing tasks probably can do so), then
> > with
> > > > > Radai's proposal we don't need to buffer the transactions for the
> > users
> > > > > anymore, which is a big difference. For the latter case, the
> consumer
> > > may
> > > > > have to buffer the incomplete transactions otherwise we are just
> > > throwing
> > > > > the burden onto the users.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jiangjie (Becket) Qin
> > > > >
> > > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > > >
> > > > > > Yeah good point. I relent!
> > > > > >
> > > > > > -jay
> > > > > >
> > > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> > jason@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > Jay/Ismael,
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > I agree that lazy initialization of metadata seems unavoidable.
> > > > > Ideally,
> > > > > > we
> > > > > > >
> > > > > > > could follow the same pattern for transactions, but remember
> that
> > > in
> > > > > the
> > > > > > >
> > > > > > > consumer+producer use case, the initialization needs to be
> > > completed
> > > > > > prior
> > > > > > >
> > > > > > > to setting the consumer's position. Otherwise we risk reading
> > stale
> > > > > > >
> > > > > > > offsets. But it would be pretty awkward if you have to begin a
> > > > > > transaction
> > > > > > >
> > > > > > > first to ensure that your consumer can read the right offset
> from
> > > the
> > > > > > >
> > > > > > > consumer, right? It's a bit easier to explain that you should
> > > always
> > > > > call
> > > > > > >
> > > > > > > `producer.init()` prior to initializing the consumer. Users
> would
> > > > > > probably
> > > > > > >
> > > > > > > get this right without any special effort.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > -Jason
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > > rsivaram@pivotal.io
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > Hi Apurva,
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > > Thank you for the answers. Just one follow-on.
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > > 15. Let me rephrase my original question. If all control
> > messages
> > > > > > > (messages
> > > > > > >
> > > > > > > > to transaction logs and markers on user logs) were
> acknowledged
> > > > only
> > > > > > > after
> > > > > > >
> > > > > > > > flushing the log segment, will transactions become durable in
> > the
> > > > > > >
> > > > > > > > traditional sense (i.e. not restricted to min.insync.replicas
> > > > > > failures) ?
> > > > > > >
> > > > > > > > This is not a suggestion to update the KIP. It seems to me
> that
> > > the
> > > > > > > design
> > > > > > >
> > > > > > > > enables full durability if required in the future with a
> rather
> > > > > > >
> > > > > > > > non-intrusive change. I just wanted to make sure I haven't
> > missed
> > > > > > > anything
> > > > > > >
> > > > > > > > fundamental that prevents Kafka from doing this.
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in>
> > wrote:
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > > > Hi Apurva,
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > Thanks for the detailed answers... and sorry for the late
> > > reply!
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > It does sound like, if the input-partitions-to-app-id
> mapping
> > > > never
> > > > > > >
> > > > > > > > > changes, the existing fencing mechanisms should prevent
> > > > duplicates.
> > > > > > >
> > > > > > > > Great!
> > > > > > >
> > > > > > > > > I'm a bit concerned the proposed API will be delicate to
> > > program
> > > > > > > against
> > > > > > >
> > > > > > > > > successfully -- even in the simple case, we need to create
> a
> > > new
> > > > > > > producer
> > > > > > >
> > > > > > > > > instance per input partition, and anything fancier is going
> > to
> > > > need
> > > > > > its
> > > > > > >
> > > > > > > > own
> > > > > > >
> > > > > > > > > implementation of the Streams/Samza-style 'task' idea --
> but
> > > that
> > > > > may
> > > > > > > be
> > > > > > >
> > > > > > > > > fine for this sort of advanced feature.
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > For the second question, I notice that Jason also
> elaborated
> > on
> > > > > this
> > > > > > >
> > > > > > > > > downthread:
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > > We also looked at removing the producer ID.
> > > > > > >
> > > > > > > > > > This was discussed somewhere above, but basically the
> idea
> > is
> > > > to
> > > > > > > store
> > > > > > >
> > > > > > > > > the
> > > > > > >
> > > > > > > > > > AppID in the message set header directly and avoid the
> > > mapping
> > > > to
> > > > > > >
> > > > > > > > > producer
> > > > > > >
> > > > > > > > > > ID altogether. As long as batching isn't too bad, the
> > impact
> > > on
> > > > > > total
> > > > > > >
> > > > > > > > > size
> > > > > > >
> > > > > > > > > > may not be too bad, but we were ultimately more
> comfortable
> > > > with
> > > > > a
> > > > > > >
> > > > > > > > fixed
> > > > > > >
> > > > > > > > > > size ID.
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > ...which suggests that the distinction is useful for
> > > performance,
> > > > > but
> > > > > > > not
> > > > > > >
> > > > > > > > > necessary for correctness, which makes good sense to me.
> > > (Would a
> > > > > > > 128-bid
> > > > > > >
> > > > > > > > > ID be a reasonable compromise? That's enough room for a
> UUID,
> > > or
> > > > a
> > > > > > >
> > > > > > > > > reasonable hash of an arbitrary string, and has only a
> > marginal
> > > > > > > increase
> > > > > > >
> > > > > > > > on
> > > > > > >
> > > > > > > > > the message size.)
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > > apurva@confluent.io
> > > > > >
> > > > > > >
> > > > > > > > wrote:
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > > > > Hi Ben,
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > Now, on to your first question of how deal with consumer
> > > > > > rebalances.
> > > > > > >
> > > > > > > > The
> > > > > > >
> > > > > > > > > > short answer is that the application needs to ensure that
> > the
> > > > the
> > > > > > >
> > > > > > > > > > assignment of input partitions to appId is consistent
> > across
> > > > > > >
> > > > > > > > rebalances.
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > For Kafka streams, they already ensure that the mapping
> of
> > > > input
> > > > > > >
> > > > > > > > > partitions
> > > > > > >
> > > > > > > > > > to task Id is invariant across rebalances by
> implementing a
> > > > > custom
> > > > > > >
> > > > > > > > sticky
> > > > > > >
> > > > > > > > > > assignor. Other non-streams apps can trivially have one
> > > > producer
> > > > > > per
> > > > > > >
> > > > > > > > > input
> > > > > > >
> > > > > > > > > > partition and have the appId be the same as the partition
> > > > number
> > > > > to
> > > > > > >
> > > > > > > > > achieve
> > > > > > >
> > > > > > > > > > the same effect.
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > With this precondition in place, we can maintain
> > transactions
> > > > > > across
> > > > > > >
> > > > > > > > > > rebalances.
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > Hope this answers your question.
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > Thanks,
> > > > > > >
> > > > > > > > > > Apurva
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> > > > wrote:
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > > > > Thanks for this! I'm looking forward to going through
> the
> > > > full
> > > > > > >
> > > > > > > > proposal
> > > > > > >
> > > > > > > > > > in
> > > > > > >
> > > > > > > > > > > detail soon; a few early questions:
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > First: what happens when a consumer rebalances in the
> > > middle
> > > > > of a
> > > > > > >
> > > > > > > > > > > transaction? The full documentation suggests that such
> a
> > > > > > > transaction
> > > > > > >
> > > > > > > > > > ought
> > > > > > >
> > > > > > > > > > > to be rejected:
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > [...] if a rebalance has happened and this consumer
> > > > > > >
> > > > > > > > > > > > instance becomes a zombie, even if this offset
> message
> > is
> > > > > > > appended
> > > > > > >
> > > > > > > > in
> > > > > > >
> > > > > > > > > > the
> > > > > > >
> > > > > > > > > > > > offset topic, the transaction will be rejected later
> on
> > > > when
> > > > > it
> > > > > > >
> > > > > > > > tries
> > > > > > >
> > > > > > > > > > to
> > > > > > >
> > > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > ...but it's unclear to me how we ensure that a
> > transaction
> > > > > can't
> > > > > > >
> > > > > > > > > complete
> > > > > > >
> > > > > > > > > > > if a rebalance has happened. (It's quite possible I'm
> > > missing
> > > > > > >
> > > > > > > > something
> > > > > > >
> > > > > > > > > > > obvious!)
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > As a concrete example: suppose a process with PID 1
> adds
> > > > > offsets
> > > > > > > for
> > > > > > >
> > > > > > > > > some
> > > > > > >
> > > > > > > > > > > partition to a transaction; a consumer rebalance
> happens
> > > that
> > > > > > > assigns
> > > > > > >
> > > > > > > > > the
> > > > > > >
> > > > > > > > > > > partition to a process with PID 2, which adds some
> > offsets
> > > to
> > > > > its
> > > > > > >
> > > > > > > > > current
> > > > > > >
> > > > > > > > > > > transaction; both processes try and commit. Allowing
> both
> > > > > commits
> > > > > > >
> > > > > > > > would
> > > > > > >
> > > > > > > > > > > cause the messages to be processed twice -- how is that
> > > > > avoided?
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > Second: App IDs normally map to a single PID. It seems
> > like
> > > > one
> > > > > > > could
> > > > > > >
> > > > > > > > > do
> > > > > > >
> > > > > > > > > > > away with the PID concept entirely, and just use App
> IDs
> > in
> > > > > most
> > > > > > >
> > > > > > > > places
> > > > > > >
> > > > > > > > > > > that require a PID. This feels like it would be
> > > significantly
> > > > > > >
> > > > > > > > simpler,
> > > > > > >
> > > > > > > > > > > though it does increase the message size. Are there
> other
> > > > > reasons
> > > > > > > why
> > > > > > >
> > > > > > > > > the
> > > > > > >
> > > > > > > > > > > App ID / PID split is necessary?
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > > >
> > > > > > >
> > > > > > > > > > wrote:
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > Hi all,
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> > exactly
> > > > once
> > > > > > >
> > > > > > > > > delivery
> > > > > > >
> > > > > > > > > > > > semantics:
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > *https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > > > > > >
> > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> and+Transactional+Messaging
> > > > > > >
> > > > > > > > > > > > <https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > > > > > >
> > > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> > and+Transactional+Messaging>*
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > This KIP adds a transactional messaging mechanism
> along
> > > > with
> > > > > an
> > > > > > >
> > > > > > > > > > > idempotent
> > > > > > >
> > > > > > > > > > > > producer implementation to make sure that 1)
> duplicated
> > > > > > messages
> > > > > > >
> > > > > > > > sent
> > > > > > >
> > > > > > > > > > > from
> > > > > > >
> > > > > > > > > > > > the same identified producer can be detected on the
> > > broker
> > > > > > side,
> > > > > > >
> > > > > > > > and
> > > > > > >
> > > > > > > > > > 2) a
> > > > > > >
> > > > > > > > > > > > group of messages sent within a transaction will
> > > atomically
> > > > > be
> > > > > > >
> > > > > > > > either
> > > > > > >
> > > > > > > > > > > > reflected and fetchable to consumers or not as a
> whole.
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > The above wiki page provides a high-level view of the
> > > > > proposed
> > > > > > >
> > > > > > > > > changes
> > > > > > >
> > > > > > > > > > as
> > > > > > >
> > > > > > > > > > > > well as summarized guarantees. Initial draft of the
> > > > detailed
> > > > > > >
> > > > > > > > > > > implementation
> > > > > > >
> > > > > > > > > > > > design is described in this Google doc:
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > > >
> > > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > >
> > > > > > > > > > > > 0wSw9ra8
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > > > -- Guozhang
> > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
1. on write amplification: i dont see x6 the writes, at worst i see x2 the
writes - once to the "tx log", then read and again to the destination
partition. if you have some != 1 replication factor than both the 1st and
the 2nd writes get replicated, but it is still a relative factor of x2.
what am I missing?

2. the effect of write amplification on broker throughput really depends on
the hardware youre running. just as an example - here (
http://www.samsung.com/semiconductor/minisite/ssd/product/consumer/ssd960.html)
is a laptop ssd that can max out a 10gig ethernet NIC on writes. I would
expect that on "high performance" hardware kafka would be CPU bound.

3. why do writes to a TX need the same guarantees as "plain" writes? in
cases where the user can live with a TX rollback on change of
leadership/broker crash the TX log can be unreplicated, and even live in
the leader's memory. that would cut down on writes. this is also an
acceptable default in SQL - if your socket connection to a DB dies mid-TX
your TX is toast (mysql is even worse)

4. even if we replicate the TX log, why do we need to re-read it and
re-write it to the underlying partition? if its already written to disk all
I would need is to make that file the current segment of the "real"
partition and i've avoided the double write (at the cost of complicating
segment management). if the data is replicated fetchers could do the same.

5. on latency - youre right, what im suggesting would result in tx ordering
of messages ,"read committed" semantics and therefore higher latency. its
theoretically possible to implement "read uncommitted" on top of it but it
would inevitably result in opt-in vs vanilla clients seeing a different
order of msgs.

6. the added delay (vs your read uncommitted) would be roughly the time
span of a TX. are you designing for long-running transactions? seems to me
that if the common use case is start a TX, deliver some batch of msgs that
I already have on hand, then commit the delay isnt very long.

7. the need to buffer (or re-read) messages on consumers who do not opt-in
(which are expected to be the majority in your use case?), and do so again
and again if clients reset to earlier offsets/reconsume might make the
system as a whole less efficient.

8. what is the ratio of read-uncommitted vs read-committed clients you
expect to see?

9. what is the ratio of TX writes vs vanilla writes you expect to see?

On Mon, Dec 19, 2016 at 3:15 PM, Guozhang Wang <wa...@gmail.com> wrote:

> One more thing about the double journal proposal: when discussing about
> this method back at LinkedIn, another raised issue besides double writing
> was that it will void the offset ordering and enforce people to accept
> "transaction ordering", that is, consumer will not see messages from the
> same partition in the order where they were produced, but only in the order
> of when the corresponding transaction was committed. For some scenarios, we
> believe that offset ordering would still be preferred than transaction
> ordering and that is why in KIP-98 proposal we default to the former while
> leave the door open if users want to switch to the latter case.
>
>
> Guozhang
>
> On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Radai,
> >
> > I'm not sure if I fully understand what you are proposing, but I
> > interpreted it to be similar to a proposal we worked through back at
> > LinkedIn. The proposal was to commit to a central txlog topic, and then
> > recopy to the destination topic upon transaction commit. The observation
> on
> > that approach at the time were the following:
> >
> >    1. It is cleaner since the output topics have only committed data!
> >    2. You need full replication on the txlog topic to ensure atomicity.
> We
> >    weren't able to come up with a solution where you buffer in memory or
> > use
> >    renaming tricks the way you are describing. The reason is that once
> you
> >    begin committing you must ensure that the commit eventually succeeds
> to
> >    guarantee atomicity. If you use a transient store you might commit
> some
> >    data and then have a server failure that causes you to lose the rest
> of
> > the
> >    transaction.
> >    3. Having a single log allows the reader to choose a "read
> uncommitted"
> >    mode that hands out messages immediately. This is important for cases
> > where
> >    latency is important, especially for stream processing topologies
> where
> >    these latencies stack up across multiple stages.
> >
> > For the stream processing use case, item (2) is a bit of a deal killer.
> > This takes the cost of a transient message write (say the intermediate
> > result of a stream processing topology) from 3x writes (assuming 3x
> > replication) to 6x writes. This means you basically can't default it on.
> If
> > we can in fact get the cost down to a single buffered write (i.e. 1x the
> > data is written to memory and buffered to disk if the transaction is
> large)
> > as in the KIP-98 proposal without too many other negative side effects I
> > think that could be compelling.
> >
> > -Jay
> >
> >
> >
> > On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com>
> wrote:
> >
> > > regarding efficiency:
> > >
> > > I'd like to distinguish between server efficiency (resource utilization
> > of
> > > the broker machine alone) and overall network efficiency (resource
> > > utilization on brokers, producers and consumers, including network
> > > traffic).
> > > my proposal is not as resource-efficient on the broker (although it can
> > be,
> > > depends on a few trade offs and implementation details). HOWEVER, if i
> > look
> > > at the overall efficiency:
> > >
> > >    1.clients would need to either buffer or double-read uncommitted
> msgs.
> > > for N clients reading the stream M times (after re-starts and
> reconsumes)
> > > this would mean a M*N factor in either network BW or disk/memory space
> > > (depends on if buffer vs re-read). potentially N*M more broker-side
> reads
> > > too.
> > >    2 to reduce the broker side cost several things can be done (this is
> > not
> > > an either-or list, these are commulative):
> > >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk
> writes
> > > for TX resiliency
> > >       2.2 - when "appending" TX logs to real partitions - instead of
> > > reading from (disk-based) TX log and writing to partition log (x2 disk
> > > writes) the TX log can be made a segment file (so file rename, with
> > > associated protocol changes). this would avoid double writing by simply
> > > making the TX file part of the partition (for large enough TXs. smaller
> > > ones can be rewritten).
> > >       2.3 - the approach above could be combined with a background
> > "defrag"
> > > - similar in concept to compaction - to further reduce the total of
> > > resulting number of files.
> > >
> > > I think my main issue with the current proposal, more important than
> > > performance, is lack of proper "encapsulation" of transactions - I dont
> > > think downstream consumers should see uncommitted msgs. ever.
> > >
> > >
> > > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com>
> > wrote:
> > >
> > > > @Jason
> > > >
> > > > Yes, second thought on the number of messages included, the offset
> > delta
> > > > will probably be sufficient. The use case I encounter before for
> number
> > > of
> > > > messages in a message set is an embedded mirror maker on the
> > destination
> > > > broker side which fetches message directly from the source cluster.
> > > Ideally
> > > > the destination cluster only needs to check CRC and assign the
> offsets
> > > > because all the message verification has been done by the source
> > cluster,
> > > > but due to the lack of the number of messages in the message set, we
> > have
> > > > to decompress the message set to increment offsets correctly. By
> > knowing
> > > > the number of the messages in the message set, we can avoid doing
> that.
> > > The
> > > > offset delta will also help. It's just then the offsets may have
> holes
> > > for
> > > > log compacted topics, but that may be fine.
> > > >
> > > > @Apurva
> > > >
> > > > I am not sure if it is true that the consumer will either deliver all
> > the
> > > > message for the entire transaction or none of them from one poll()
> > call.
> > > If
> > > > we allow the transactions to be across partitions, unless the
> consumer
> > > > consumes from all the partitions involved in a transactions, it seems
> > > > impossible for it to deliver *all* the messages in a transaction,
> > right?
> > > A
> > > > weaker guarantee is we will deliver all or none of the messages that
> > > belong
> > > > to the same transaction in ONE partition, but this would be different
> > > from
> > > > the guarantee from the producer side.
> > > >
> > > > My two cents on Radai's sideways partition design:
> > > > 1. If we consider the producer side behavior as doing a two phase
> > commit
> > > > which including the committing the consumer offsets, it is a little
> > > awkward
> > > > that we allow uncommitted message goes into the main log and rely on
> > the
> > > > consumer to filter out. So semantic wise I think it would be better
> if
> > we
> > > > can avoid this. Radai's suggestion is actually intuitive because if
> the
> > > > brokers do not want to expose uncommitted transactions to the
> consumer,
> > > the
> > > > brokers have to buffer it.
> > > >
> > > > 2. Regarding the efficiency. I think may be it worth looking at the
> > > > efficiency cost v.s benefit. The efficiency includes both server side
> > > > efficiency and consumer side efficiency.
> > > >
> > > > Regarding the server side efficiency, the current proposal would
> > probably
> > > > have better efficiency regardless of whether something goes wrong.
> > > Radai's
> > > > suggestion would put more burden on the server side. If nothing goes
> > > wrong
> > > > we always pay the cost of having double copy of the transactional
> > > messages
> > > > and do not get the semantic benefit. But if something goes wrong, the
> > > > efficiency cost we pay we get us a better semantic.
> > > >
> > > > For the consumer side efficiency, because there is no need to buffer
> > the
> > > > uncommitted messages. The current proposal may have to potentially
> > buffer
> > > > uncommitted messages so it would be less efficient than Radai's
> > > suggestion
> > > > when a transaction aborts. When everything goes well, both design
> seems
> > > > having the similar performance. However, it depends on whether we are
> > > > willing to loosen the consumer side transaction guarantee that I
> > > mentioned
> > > > earlier to Apurva.
> > > >
> > > > Currently the biggest pressure on the consumer side is that it has to
> > > > buffer incomplete transactions. There are two reasons for it,
> > > > A. A transaction may be aborted so we cannot expose the messages to
> the
> > > > users.
> > > > B. We want to return all or none of the messages in a transaction in
> > ONE
> > > > partition.
> > > >
> > > > While reason A is mandatory, I think reason B may be discussable.
> > Radai's
> > > > design actually removes reason A because there is no uncommitted
> > messages
> > > > exposed to the consumers. This may potentially give us a chance to
> > > > significantly improve consumer side efficiency in normal cases. It
> > again
> > > > depends on the use case, i.e. whether user can process a transaction
> > > > progressively (message by message) or it has to be buffered and
> > returned
> > > > all together. If in most cases, users can process the transactions
> > > message
> > > > by message (most stream processing tasks probably can do so), then
> with
> > > > Radai's proposal we don't need to buffer the transactions for the
> users
> > > > anymore, which is a big difference. For the latter case, the consumer
> > may
> > > > have to buffer the incomplete transactions otherwise we are just
> > throwing
> > > > the burden onto the users.
> > > >
> > > > Thanks,
> > > >
> > > > Jiangjie (Becket) Qin
> > > >
> > > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > > > Yeah good point. I relent!
> > > > >
> > > > > -jay
> > > > >
> > > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <
> jason@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Jay/Ismael,
> > > > > >
> > > > > >
> > > > > >
> > > > > > I agree that lazy initialization of metadata seems unavoidable.
> > > > Ideally,
> > > > > we
> > > > > >
> > > > > > could follow the same pattern for transactions, but remember that
> > in
> > > > the
> > > > > >
> > > > > > consumer+producer use case, the initialization needs to be
> > completed
> > > > > prior
> > > > > >
> > > > > > to setting the consumer's position. Otherwise we risk reading
> stale
> > > > > >
> > > > > > offsets. But it would be pretty awkward if you have to begin a
> > > > > transaction
> > > > > >
> > > > > > first to ensure that your consumer can read the right offset from
> > the
> > > > > >
> > > > > > consumer, right? It's a bit easier to explain that you should
> > always
> > > > call
> > > > > >
> > > > > > `producer.init()` prior to initializing the consumer. Users would
> > > > > probably
> > > > > >
> > > > > > get this right without any special effort.
> > > > > >
> > > > > >
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> > rsivaram@pivotal.io
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >
> > > > > >
> > > > > > > Hi Apurva,
> > > > > >
> > > > > > >
> > > > > >
> > > > > > > Thank you for the answers. Just one follow-on.
> > > > > >
> > > > > > >
> > > > > >
> > > > > > > 15. Let me rephrase my original question. If all control
> messages
> > > > > > (messages
> > > > > >
> > > > > > > to transaction logs and markers on user logs) were acknowledged
> > > only
> > > > > > after
> > > > > >
> > > > > > > flushing the log segment, will transactions become durable in
> the
> > > > > >
> > > > > > > traditional sense (i.e. not restricted to min.insync.replicas
> > > > > failures) ?
> > > > > >
> > > > > > > This is not a suggestion to update the KIP. It seems to me that
> > the
> > > > > > design
> > > > > >
> > > > > > > enables full durability if required in the future with a rather
> > > > > >
> > > > > > > non-intrusive change. I just wanted to make sure I haven't
> missed
> > > > > > anything
> > > > > >
> > > > > > > fundamental that prevents Kafka from doing this.
> > > > > >
> > > > > > >
> > > > > >
> > > > > > >
> > > > > >
> > > > > > >
> > > > > >
> > > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in>
> wrote:
> > > > > >
> > > > > > >
> > > > > >
> > > > > > > > Hi Apurva,
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > Thanks for the detailed answers... and sorry for the late
> > reply!
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > It does sound like, if the input-partitions-to-app-id mapping
> > > never
> > > > > >
> > > > > > > > changes, the existing fencing mechanisms should prevent
> > > duplicates.
> > > > > >
> > > > > > > Great!
> > > > > >
> > > > > > > > I'm a bit concerned the proposed API will be delicate to
> > program
> > > > > > against
> > > > > >
> > > > > > > > successfully -- even in the simple case, we need to create a
> > new
> > > > > > producer
> > > > > >
> > > > > > > > instance per input partition, and anything fancier is going
> to
> > > need
> > > > > its
> > > > > >
> > > > > > > own
> > > > > >
> > > > > > > > implementation of the Streams/Samza-style 'task' idea -- but
> > that
> > > > may
> > > > > > be
> > > > > >
> > > > > > > > fine for this sort of advanced feature.
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > For the second question, I notice that Jason also elaborated
> on
> > > > this
> > > > > >
> > > > > > > > downthread:
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > > We also looked at removing the producer ID.
> > > > > >
> > > > > > > > > This was discussed somewhere above, but basically the idea
> is
> > > to
> > > > > > store
> > > > > >
> > > > > > > > the
> > > > > >
> > > > > > > > > AppID in the message set header directly and avoid the
> > mapping
> > > to
> > > > > >
> > > > > > > > producer
> > > > > >
> > > > > > > > > ID altogether. As long as batching isn't too bad, the
> impact
> > on
> > > > > total
> > > > > >
> > > > > > > > size
> > > > > >
> > > > > > > > > may not be too bad, but we were ultimately more comfortable
> > > with
> > > > a
> > > > > >
> > > > > > > fixed
> > > > > >
> > > > > > > > > size ID.
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > ...which suggests that the distinction is useful for
> > performance,
> > > > but
> > > > > > not
> > > > > >
> > > > > > > > necessary for correctness, which makes good sense to me.
> > (Would a
> > > > > > 128-bid
> > > > > >
> > > > > > > > ID be a reasonable compromise? That's enough room for a UUID,
> > or
> > > a
> > > > > >
> > > > > > > > reasonable hash of an arbitrary string, and has only a
> marginal
> > > > > > increase
> > > > > >
> > > > > > > on
> > > > > >
> > > > > > > > the message size.)
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > > apurva@confluent.io
> > > > >
> > > > > >
> > > > > > > wrote:
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > > > > Hi Ben,
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > Now, on to your first question of how deal with consumer
> > > > > rebalances.
> > > > > >
> > > > > > > The
> > > > > >
> > > > > > > > > short answer is that the application needs to ensure that
> the
> > > the
> > > > > >
> > > > > > > > > assignment of input partitions to appId is consistent
> across
> > > > > >
> > > > > > > rebalances.
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > For Kafka streams, they already ensure that the mapping of
> > > input
> > > > > >
> > > > > > > > partitions
> > > > > >
> > > > > > > > > to task Id is invariant across rebalances by implementing a
> > > > custom
> > > > > >
> > > > > > > sticky
> > > > > >
> > > > > > > > > assignor. Other non-streams apps can trivially have one
> > > producer
> > > > > per
> > > > > >
> > > > > > > > input
> > > > > >
> > > > > > > > > partition and have the appId be the same as the partition
> > > number
> > > > to
> > > > > >
> > > > > > > > achieve
> > > > > >
> > > > > > > > > the same effect.
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > With this precondition in place, we can maintain
> transactions
> > > > > across
> > > > > >
> > > > > > > > > rebalances.
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > Hope this answers your question.
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > Thanks,
> > > > > >
> > > > > > > > > Apurva
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> > > wrote:
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > > > > Thanks for this! I'm looking forward to going through the
> > > full
> > > > > >
> > > > > > > proposal
> > > > > >
> > > > > > > > > in
> > > > > >
> > > > > > > > > > detail soon; a few early questions:
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > First: what happens when a consumer rebalances in the
> > middle
> > > > of a
> > > > > >
> > > > > > > > > > transaction? The full documentation suggests that such a
> > > > > > transaction
> > > > > >
> > > > > > > > > ought
> > > > > >
> > > > > > > > > > to be rejected:
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > > [...] if a rebalance has happened and this consumer
> > > > > >
> > > > > > > > > > > instance becomes a zombie, even if this offset message
> is
> > > > > > appended
> > > > > >
> > > > > > > in
> > > > > >
> > > > > > > > > the
> > > > > >
> > > > > > > > > > > offset topic, the transaction will be rejected later on
> > > when
> > > > it
> > > > > >
> > > > > > > tries
> > > > > >
> > > > > > > > > to
> > > > > >
> > > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > ...but it's unclear to me how we ensure that a
> transaction
> > > > can't
> > > > > >
> > > > > > > > complete
> > > > > >
> > > > > > > > > > if a rebalance has happened. (It's quite possible I'm
> > missing
> > > > > >
> > > > > > > something
> > > > > >
> > > > > > > > > > obvious!)
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > As a concrete example: suppose a process with PID 1 adds
> > > > offsets
> > > > > > for
> > > > > >
> > > > > > > > some
> > > > > >
> > > > > > > > > > partition to a transaction; a consumer rebalance happens
> > that
> > > > > > assigns
> > > > > >
> > > > > > > > the
> > > > > >
> > > > > > > > > > partition to a process with PID 2, which adds some
> offsets
> > to
> > > > its
> > > > > >
> > > > > > > > current
> > > > > >
> > > > > > > > > > transaction; both processes try and commit. Allowing both
> > > > commits
> > > > > >
> > > > > > > would
> > > > > >
> > > > > > > > > > cause the messages to be processed twice -- how is that
> > > > avoided?
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > Second: App IDs normally map to a single PID. It seems
> like
> > > one
> > > > > > could
> > > > > >
> > > > > > > > do
> > > > > >
> > > > > > > > > > away with the PID concept entirely, and just use App IDs
> in
> > > > most
> > > > > >
> > > > > > > places
> > > > > >
> > > > > > > > > > that require a PID. This feels like it would be
> > significantly
> > > > > >
> > > > > > > simpler,
> > > > > >
> > > > > > > > > > though it does increase the message size. Are there other
> > > > reasons
> > > > > > why
> > > > > >
> > > > > > > > the
> > > > > >
> > > > > > > > > > App ID / PID split is necessary?
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > > >
> > > > > >
> > > > > > > > > wrote:
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > > > > Hi all,
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > I have just created KIP-98 to enhance Kafka with
> exactly
> > > once
> > > > > >
> > > > > > > > delivery
> > > > > >
> > > > > > > > > > > semantics:
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > *https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > >
> > > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > >
> > > > > > > > > > > <https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-
> > > > > >
> > > > > > > > > > > 98+-+Exactly+Once+Delivery+
> and+Transactional+Messaging>*
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > > with
> > > > an
> > > > > >
> > > > > > > > > > idempotent
> > > > > >
> > > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > > messages
> > > > > >
> > > > > > > sent
> > > > > >
> > > > > > > > > > from
> > > > > >
> > > > > > > > > > > the same identified producer can be detected on the
> > broker
> > > > > side,
> > > > > >
> > > > > > > and
> > > > > >
> > > > > > > > > 2) a
> > > > > >
> > > > > > > > > > > group of messages sent within a transaction will
> > atomically
> > > > be
> > > > > >
> > > > > > > either
> > > > > >
> > > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > The above wiki page provides a high-level view of the
> > > > proposed
> > > > > >
> > > > > > > > changes
> > > > > >
> > > > > > > > > as
> > > > > >
> > > > > > > > > > > well as summarized guarantees. Initial draft of the
> > > detailed
> > > > > >
> > > > > > > > > > implementation
> > > > > >
> > > > > > > > > > > design is described in this Google doc:
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > > >
> > > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > >
> > > > > > > > > > > 0wSw9ra8
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > We would love to hear your comments and suggestions.
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > Thanks,
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > > > -- Guozhang
> > > > > >
> > > > > > > > > > >
> > > > > >
> > > > > > > > > >
> > > > > >
> > > > > > > > >
> > > > > >
> > > > > > > >
> > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
One more thing about the double journal proposal: when discussing about
this method back at LinkedIn, another raised issue besides double writing
was that it will void the offset ordering and enforce people to accept
"transaction ordering", that is, consumer will not see messages from the
same partition in the order where they were produced, but only in the order
of when the corresponding transaction was committed. For some scenarios, we
believe that offset ordering would still be preferred than transaction
ordering and that is why in KIP-98 proposal we default to the former while
leave the door open if users want to switch to the latter case.


Guozhang

On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Radai,
>
> I'm not sure if I fully understand what you are proposing, but I
> interpreted it to be similar to a proposal we worked through back at
> LinkedIn. The proposal was to commit to a central txlog topic, and then
> recopy to the destination topic upon transaction commit. The observation on
> that approach at the time were the following:
>
>    1. It is cleaner since the output topics have only committed data!
>    2. You need full replication on the txlog topic to ensure atomicity. We
>    weren't able to come up with a solution where you buffer in memory or
> use
>    renaming tricks the way you are describing. The reason is that once you
>    begin committing you must ensure that the commit eventually succeeds to
>    guarantee atomicity. If you use a transient store you might commit some
>    data and then have a server failure that causes you to lose the rest of
> the
>    transaction.
>    3. Having a single log allows the reader to choose a "read uncommitted"
>    mode that hands out messages immediately. This is important for cases
> where
>    latency is important, especially for stream processing topologies where
>    these latencies stack up across multiple stages.
>
> For the stream processing use case, item (2) is a bit of a deal killer.
> This takes the cost of a transient message write (say the intermediate
> result of a stream processing topology) from 3x writes (assuming 3x
> replication) to 6x writes. This means you basically can't default it on. If
> we can in fact get the cost down to a single buffered write (i.e. 1x the
> data is written to memory and buffered to disk if the transaction is large)
> as in the KIP-98 proposal without too many other negative side effects I
> think that could be compelling.
>
> -Jay
>
>
>
> On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com> wrote:
>
> > regarding efficiency:
> >
> > I'd like to distinguish between server efficiency (resource utilization
> of
> > the broker machine alone) and overall network efficiency (resource
> > utilization on brokers, producers and consumers, including network
> > traffic).
> > my proposal is not as resource-efficient on the broker (although it can
> be,
> > depends on a few trade offs and implementation details). HOWEVER, if i
> look
> > at the overall efficiency:
> >
> >    1.clients would need to either buffer or double-read uncommitted msgs.
> > for N clients reading the stream M times (after re-starts and reconsumes)
> > this would mean a M*N factor in either network BW or disk/memory space
> > (depends on if buffer vs re-read). potentially N*M more broker-side reads
> > too.
> >    2 to reduce the broker side cost several things can be done (this is
> not
> > an either-or list, these are commulative):
> >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk writes
> > for TX resiliency
> >       2.2 - when "appending" TX logs to real partitions - instead of
> > reading from (disk-based) TX log and writing to partition log (x2 disk
> > writes) the TX log can be made a segment file (so file rename, with
> > associated protocol changes). this would avoid double writing by simply
> > making the TX file part of the partition (for large enough TXs. smaller
> > ones can be rewritten).
> >       2.3 - the approach above could be combined with a background
> "defrag"
> > - similar in concept to compaction - to further reduce the total of
> > resulting number of files.
> >
> > I think my main issue with the current proposal, more important than
> > performance, is lack of proper "encapsulation" of transactions - I dont
> > think downstream consumers should see uncommitted msgs. ever.
> >
> >
> > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com>
> wrote:
> >
> > > @Jason
> > >
> > > Yes, second thought on the number of messages included, the offset
> delta
> > > will probably be sufficient. The use case I encounter before for number
> > of
> > > messages in a message set is an embedded mirror maker on the
> destination
> > > broker side which fetches message directly from the source cluster.
> > Ideally
> > > the destination cluster only needs to check CRC and assign the offsets
> > > because all the message verification has been done by the source
> cluster,
> > > but due to the lack of the number of messages in the message set, we
> have
> > > to decompress the message set to increment offsets correctly. By
> knowing
> > > the number of the messages in the message set, we can avoid doing that.
> > The
> > > offset delta will also help. It's just then the offsets may have holes
> > for
> > > log compacted topics, but that may be fine.
> > >
> > > @Apurva
> > >
> > > I am not sure if it is true that the consumer will either deliver all
> the
> > > message for the entire transaction or none of them from one poll()
> call.
> > If
> > > we allow the transactions to be across partitions, unless the consumer
> > > consumes from all the partitions involved in a transactions, it seems
> > > impossible for it to deliver *all* the messages in a transaction,
> right?
> > A
> > > weaker guarantee is we will deliver all or none of the messages that
> > belong
> > > to the same transaction in ONE partition, but this would be different
> > from
> > > the guarantee from the producer side.
> > >
> > > My two cents on Radai's sideways partition design:
> > > 1. If we consider the producer side behavior as doing a two phase
> commit
> > > which including the committing the consumer offsets, it is a little
> > awkward
> > > that we allow uncommitted message goes into the main log and rely on
> the
> > > consumer to filter out. So semantic wise I think it would be better if
> we
> > > can avoid this. Radai's suggestion is actually intuitive because if the
> > > brokers do not want to expose uncommitted transactions to the consumer,
> > the
> > > brokers have to buffer it.
> > >
> > > 2. Regarding the efficiency. I think may be it worth looking at the
> > > efficiency cost v.s benefit. The efficiency includes both server side
> > > efficiency and consumer side efficiency.
> > >
> > > Regarding the server side efficiency, the current proposal would
> probably
> > > have better efficiency regardless of whether something goes wrong.
> > Radai's
> > > suggestion would put more burden on the server side. If nothing goes
> > wrong
> > > we always pay the cost of having double copy of the transactional
> > messages
> > > and do not get the semantic benefit. But if something goes wrong, the
> > > efficiency cost we pay we get us a better semantic.
> > >
> > > For the consumer side efficiency, because there is no need to buffer
> the
> > > uncommitted messages. The current proposal may have to potentially
> buffer
> > > uncommitted messages so it would be less efficient than Radai's
> > suggestion
> > > when a transaction aborts. When everything goes well, both design seems
> > > having the similar performance. However, it depends on whether we are
> > > willing to loosen the consumer side transaction guarantee that I
> > mentioned
> > > earlier to Apurva.
> > >
> > > Currently the biggest pressure on the consumer side is that it has to
> > > buffer incomplete transactions. There are two reasons for it,
> > > A. A transaction may be aborted so we cannot expose the messages to the
> > > users.
> > > B. We want to return all or none of the messages in a transaction in
> ONE
> > > partition.
> > >
> > > While reason A is mandatory, I think reason B may be discussable.
> Radai's
> > > design actually removes reason A because there is no uncommitted
> messages
> > > exposed to the consumers. This may potentially give us a chance to
> > > significantly improve consumer side efficiency in normal cases. It
> again
> > > depends on the use case, i.e. whether user can process a transaction
> > > progressively (message by message) or it has to be buffered and
> returned
> > > all together. If in most cases, users can process the transactions
> > message
> > > by message (most stream processing tasks probably can do so), then with
> > > Radai's proposal we don't need to buffer the transactions for the users
> > > anymore, which is a big difference. For the latter case, the consumer
> may
> > > have to buffer the incomplete transactions otherwise we are just
> throwing
> > > the burden onto the users.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Yeah good point. I relent!
> > > >
> > > > -jay
> > > >
> > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > Jay/Ismael,
> > > > >
> > > > >
> > > > >
> > > > > I agree that lazy initialization of metadata seems unavoidable.
> > > Ideally,
> > > > we
> > > > >
> > > > > could follow the same pattern for transactions, but remember that
> in
> > > the
> > > > >
> > > > > consumer+producer use case, the initialization needs to be
> completed
> > > > prior
> > > > >
> > > > > to setting the consumer's position. Otherwise we risk reading stale
> > > > >
> > > > > offsets. But it would be pretty awkward if you have to begin a
> > > > transaction
> > > > >
> > > > > first to ensure that your consumer can read the right offset from
> the
> > > > >
> > > > > consumer, right? It's a bit easier to explain that you should
> always
> > > call
> > > > >
> > > > > `producer.init()` prior to initializing the consumer. Users would
> > > > probably
> > > > >
> > > > > get this right without any special effort.
> > > > >
> > > > >
> > > > >
> > > > > -Jason
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> rsivaram@pivotal.io
> > >
> > > > > wrote:
> > > > >
> > > > >
> > > > >
> > > > > > Hi Apurva,
> > > > >
> > > > > >
> > > > >
> > > > > > Thank you for the answers. Just one follow-on.
> > > > >
> > > > > >
> > > > >
> > > > > > 15. Let me rephrase my original question. If all control messages
> > > > > (messages
> > > > >
> > > > > > to transaction logs and markers on user logs) were acknowledged
> > only
> > > > > after
> > > > >
> > > > > > flushing the log segment, will transactions become durable in the
> > > > >
> > > > > > traditional sense (i.e. not restricted to min.insync.replicas
> > > > failures) ?
> > > > >
> > > > > > This is not a suggestion to update the KIP. It seems to me that
> the
> > > > > design
> > > > >
> > > > > > enables full durability if required in the future with a rather
> > > > >
> > > > > > non-intrusive change. I just wanted to make sure I haven't missed
> > > > > anything
> > > > >
> > > > > > fundamental that prevents Kafka from doing this.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > Hi Apurva,
> > > > >
> > > > > > >
> > > > >
> > > > > > > Thanks for the detailed answers... and sorry for the late
> reply!
> > > > >
> > > > > > >
> > > > >
> > > > > > > It does sound like, if the input-partitions-to-app-id mapping
> > never
> > > > >
> > > > > > > changes, the existing fencing mechanisms should prevent
> > duplicates.
> > > > >
> > > > > > Great!
> > > > >
> > > > > > > I'm a bit concerned the proposed API will be delicate to
> program
> > > > > against
> > > > >
> > > > > > > successfully -- even in the simple case, we need to create a
> new
> > > > > producer
> > > > >
> > > > > > > instance per input partition, and anything fancier is going to
> > need
> > > > its
> > > > >
> > > > > > own
> > > > >
> > > > > > > implementation of the Streams/Samza-style 'task' idea -- but
> that
> > > may
> > > > > be
> > > > >
> > > > > > > fine for this sort of advanced feature.
> > > > >
> > > > > > >
> > > > >
> > > > > > > For the second question, I notice that Jason also elaborated on
> > > this
> > > > >
> > > > > > > downthread:
> > > > >
> > > > > > >
> > > > >
> > > > > > > > We also looked at removing the producer ID.
> > > > >
> > > > > > > > This was discussed somewhere above, but basically the idea is
> > to
> > > > > store
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > AppID in the message set header directly and avoid the
> mapping
> > to
> > > > >
> > > > > > > producer
> > > > >
> > > > > > > > ID altogether. As long as batching isn't too bad, the impact
> on
> > > > total
> > > > >
> > > > > > > size
> > > > >
> > > > > > > > may not be too bad, but we were ultimately more comfortable
> > with
> > > a
> > > > >
> > > > > > fixed
> > > > >
> > > > > > > > size ID.
> > > > >
> > > > > > >
> > > > >
> > > > > > > ...which suggests that the distinction is useful for
> performance,
> > > but
> > > > > not
> > > > >
> > > > > > > necessary for correctness, which makes good sense to me.
> (Would a
> > > > > 128-bid
> > > > >
> > > > > > > ID be a reasonable compromise? That's enough room for a UUID,
> or
> > a
> > > > >
> > > > > > > reasonable hash of an arbitrary string, and has only a marginal
> > > > > increase
> > > > >
> > > > > > on
> > > > >
> > > > > > > the message size.)
> > > > >
> > > > > > >
> > > > >
> > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > apurva@confluent.io
> > > >
> > > > >
> > > > > > wrote:
> > > > >
> > > > > > >
> > > > >
> > > > > > > > Hi Ben,
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Now, on to your first question of how deal with consumer
> > > > rebalances.
> > > > >
> > > > > > The
> > > > >
> > > > > > > > short answer is that the application needs to ensure that the
> > the
> > > > >
> > > > > > > > assignment of input partitions to appId is consistent across
> > > > >
> > > > > > rebalances.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > For Kafka streams, they already ensure that the mapping of
> > input
> > > > >
> > > > > > > partitions
> > > > >
> > > > > > > > to task Id is invariant across rebalances by implementing a
> > > custom
> > > > >
> > > > > > sticky
> > > > >
> > > > > > > > assignor. Other non-streams apps can trivially have one
> > producer
> > > > per
> > > > >
> > > > > > > input
> > > > >
> > > > > > > > partition and have the appId be the same as the partition
> > number
> > > to
> > > > >
> > > > > > > achieve
> > > > >
> > > > > > > > the same effect.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > With this precondition in place, we can maintain transactions
> > > > across
> > > > >
> > > > > > > > rebalances.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Hope this answers your question.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Thanks,
> > > > >
> > > > > > > > Apurva
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> > wrote:
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > > Thanks for this! I'm looking forward to going through the
> > full
> > > > >
> > > > > > proposal
> > > > >
> > > > > > > > in
> > > > >
> > > > > > > > > detail soon; a few early questions:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > First: what happens when a consumer rebalances in the
> middle
> > > of a
> > > > >
> > > > > > > > > transaction? The full documentation suggests that such a
> > > > > transaction
> > > > >
> > > > > > > > ought
> > > > >
> > > > > > > > > to be rejected:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > > [...] if a rebalance has happened and this consumer
> > > > >
> > > > > > > > > > instance becomes a zombie, even if this offset message is
> > > > > appended
> > > > >
> > > > > > in
> > > > >
> > > > > > > > the
> > > > >
> > > > > > > > > > offset topic, the transaction will be rejected later on
> > when
> > > it
> > > > >
> > > > > > tries
> > > > >
> > > > > > > > to
> > > > >
> > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > ...but it's unclear to me how we ensure that a transaction
> > > can't
> > > > >
> > > > > > > complete
> > > > >
> > > > > > > > > if a rebalance has happened. (It's quite possible I'm
> missing
> > > > >
> > > > > > something
> > > > >
> > > > > > > > > obvious!)
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > As a concrete example: suppose a process with PID 1 adds
> > > offsets
> > > > > for
> > > > >
> > > > > > > some
> > > > >
> > > > > > > > > partition to a transaction; a consumer rebalance happens
> that
> > > > > assigns
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > > partition to a process with PID 2, which adds some offsets
> to
> > > its
> > > > >
> > > > > > > current
> > > > >
> > > > > > > > > transaction; both processes try and commit. Allowing both
> > > commits
> > > > >
> > > > > > would
> > > > >
> > > > > > > > > cause the messages to be processed twice -- how is that
> > > avoided?
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > Second: App IDs normally map to a single PID. It seems like
> > one
> > > > > could
> > > > >
> > > > > > > do
> > > > >
> > > > > > > > > away with the PID concept entirely, and just use App IDs in
> > > most
> > > > >
> > > > > > places
> > > > >
> > > > > > > > > that require a PID. This feels like it would be
> significantly
> > > > >
> > > > > > simpler,
> > > > >
> > > > > > > > > though it does increase the message size. Are there other
> > > reasons
> > > > > why
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > > App ID / PID split is necessary?
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > >
> > > > > > > > wrote:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > > Hi all,
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> > once
> > > > >
> > > > > > > delivery
> > > > >
> > > > > > > > > > semantics:
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >
> > > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > with
> > > an
> > > > >
> > > > > > > > > idempotent
> > > > >
> > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > messages
> > > > >
> > > > > > sent
> > > > >
> > > > > > > > > from
> > > > >
> > > > > > > > > > the same identified producer can be detected on the
> broker
> > > > side,
> > > > >
> > > > > > and
> > > > >
> > > > > > > > 2) a
> > > > >
> > > > > > > > > > group of messages sent within a transaction will
> atomically
> > > be
> > > > >
> > > > > > either
> > > > >
> > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > The above wiki page provides a high-level view of the
> > > proposed
> > > > >
> > > > > > > changes
> > > > >
> > > > > > > > as
> > > > >
> > > > > > > > > > well as summarized guarantees. Initial draft of the
> > detailed
> > > > >
> > > > > > > > > implementation
> > > > >
> > > > > > > > > > design is described in this Google doc:
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > >
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > >
> > > > > > > > > > 0wSw9ra8
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > Thanks,
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > -- Guozhang
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Sriram Subramanian <ra...@confluent.io>.
small correction in my third point -

3. IO and memory constraints - We would want a solution that *does not take*
2x the number of writes.

On Mon, Dec 19, 2016 at 12:37 PM, Sriram Subramanian <ra...@confluent.io>
wrote:

> Radai,
>
> I think it is important to understand the key requirements that we don’t
> want to compromise. We can then understand the tradeoffs of the different
> approaches. We did in fact start with the double journal approach couple of
> years back. I will highlight the must have requirements first and then
> explain the trade offs based on my understanding.
>
> 1. End to end latency for stream processing - This is probably one of the
> biggest reasons to support transactions in Kafka. We would like to support
> very low latency for end to end processing across steam topologies. This
> means you would want your downstream processors to see the output of your
> processing immediately. The low latency is a requirement even if we only
> expose committed messages.
>
> 2. Speculative execution - We would like to go one step further for stream
> processing. 99% of the transactions will always succeed. We would like to
> take advantage of this and process the messages optimistically even if the
> transactions are still unfinished. If the transactions abort, we would do a
> cascading abort across the topology. This helps us to complete all the
> processing and keep the output ready and expose them once the transactions
> are committed. This will help us to significantly bring down the latency
> for end to end stream processing and provide the ability to keep exactly
> once as the default setting.
>
> 3. IO and memory constraints - We would want a solution that takes 2x the
> number of writes. This will bring down broker utilization by half. I don’t
> really understand the in memory solution (would be useful if you can
> explain it more if you think it solves these goals) but the same resource
> constraints apply. What has made Kafka successful is the ability to run
> very high throughput clusters with very few machines. We would like to keep
> this true when a cluster is largely dominated by stream processing
> workloads.
>
> 4. Provide both read committed and read uncommitted isolation levels -
> This is actually a desired feature. This is similar to database isolation
> levels (except that we provide only two of them for now). Downstream
> systems that need strong guarantees with some performance impact can choose
> read committed isolation level. Systems that want to optimize for
> performance and can live with approximations would choose read uncommitted
> options. This helps to nicely decouple downstream users that would like to
> share topics but have different end goals.
>
> There are other obvious goals like correctness of the protocol and
> simplicity of the design that needs to be true by default.
>
> Given these goals, the double journal approach was a non starter to enable
> low end to end latency and did not provide the ability to do speculative
> execution in the future. We also found the resource constraints
> (specifically IO/Network) to be unacceptable.
>
> We did understand the complexity of the consumers but it was the best
> tradeoff considering the other must have goals. We also thought of another
> approach to push the consumer buffering to the broker side. This would
> enable multiple consumer groups to share the same buffer pool for a
> specific topic partition. However, in the worst case, you would need to
> bring the entire log into memory to remove the aborted transaction (for a
> consumer that is catching up from time 0). This would also make us loose
> zero copy semantics.
>
> I would be excited to hear an option that can solve our must have goals
> and still keep the consumer really thin. The abstraction seems fine since
> we allow the end users to pick the guarantees they need.
>
> On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:
>
>> Hey Radai,
>>
>> I'm not sure if I fully understand what you are proposing, but I
>> interpreted it to be similar to a proposal we worked through back at
>> LinkedIn. The proposal was to commit to a central txlog topic, and then
>> recopy to the destination topic upon transaction commit. The observation
>> on
>> that approach at the time were the following:
>>
>>    1. It is cleaner since the output topics have only committed data!
>>    2. You need full replication on the txlog topic to ensure atomicity. We
>>    weren't able to come up with a solution where you buffer in memory or
>> use
>>    renaming tricks the way you are describing. The reason is that once you
>>    begin committing you must ensure that the commit eventually succeeds to
>>    guarantee atomicity. If you use a transient store you might commit some
>>    data and then have a server failure that causes you to lose the rest
>> of the
>>    transaction.
>>    3. Having a single log allows the reader to choose a "read uncommitted"
>>    mode that hands out messages immediately. This is important for cases
>> where
>>    latency is important, especially for stream processing topologies where
>>    these latencies stack up across multiple stages.
>>
>> For the stream processing use case, item (2) is a bit of a deal killer.
>> This takes the cost of a transient message write (say the intermediate
>> result of a stream processing topology) from 3x writes (assuming 3x
>> replication) to 6x writes. This means you basically can't default it on.
>> If
>> we can in fact get the cost down to a single buffered write (i.e. 1x the
>> data is written to memory and buffered to disk if the transaction is
>> large)
>> as in the KIP-98 proposal without too many other negative side effects I
>> think that could be compelling.
>>
>> -Jay
>>
>>
>>
>> On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com>
>> wrote:
>>
>> > regarding efficiency:
>> >
>> > I'd like to distinguish between server efficiency (resource utilization
>> of
>> > the broker machine alone) and overall network efficiency (resource
>> > utilization on brokers, producers and consumers, including network
>> > traffic).
>> > my proposal is not as resource-efficient on the broker (although it can
>> be,
>> > depends on a few trade offs and implementation details). HOWEVER, if i
>> look
>> > at the overall efficiency:
>> >
>> >    1.clients would need to either buffer or double-read uncommitted
>> msgs.
>> > for N clients reading the stream M times (after re-starts and
>> reconsumes)
>> > this would mean a M*N factor in either network BW or disk/memory space
>> > (depends on if buffer vs re-read). potentially N*M more broker-side
>> reads
>> > too.
>> >    2 to reduce the broker side cost several things can be done (this is
>> not
>> > an either-or list, these are commulative):
>> >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk writes
>> > for TX resiliency
>> >       2.2 - when "appending" TX logs to real partitions - instead of
>> > reading from (disk-based) TX log and writing to partition log (x2 disk
>> > writes) the TX log can be made a segment file (so file rename, with
>> > associated protocol changes). this would avoid double writing by simply
>> > making the TX file part of the partition (for large enough TXs. smaller
>> > ones can be rewritten).
>> >       2.3 - the approach above could be combined with a background
>> "defrag"
>> > - similar in concept to compaction - to further reduce the total of
>> > resulting number of files.
>> >
>> > I think my main issue with the current proposal, more important than
>> > performance, is lack of proper "encapsulation" of transactions - I dont
>> > think downstream consumers should see uncommitted msgs. ever.
>> >
>> >
>> > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com>
>> wrote:
>> >
>> > > @Jason
>> > >
>> > > Yes, second thought on the number of messages included, the offset
>> delta
>> > > will probably be sufficient. The use case I encounter before for
>> number
>> > of
>> > > messages in a message set is an embedded mirror maker on the
>> destination
>> > > broker side which fetches message directly from the source cluster.
>> > Ideally
>> > > the destination cluster only needs to check CRC and assign the offsets
>> > > because all the message verification has been done by the source
>> cluster,
>> > > but due to the lack of the number of messages in the message set, we
>> have
>> > > to decompress the message set to increment offsets correctly. By
>> knowing
>> > > the number of the messages in the message set, we can avoid doing
>> that.
>> > The
>> > > offset delta will also help. It's just then the offsets may have holes
>> > for
>> > > log compacted topics, but that may be fine.
>> > >
>> > > @Apurva
>> > >
>> > > I am not sure if it is true that the consumer will either deliver all
>> the
>> > > message for the entire transaction or none of them from one poll()
>> call.
>> > If
>> > > we allow the transactions to be across partitions, unless the consumer
>> > > consumes from all the partitions involved in a transactions, it seems
>> > > impossible for it to deliver *all* the messages in a transaction,
>> right?
>> > A
>> > > weaker guarantee is we will deliver all or none of the messages that
>> > belong
>> > > to the same transaction in ONE partition, but this would be different
>> > from
>> > > the guarantee from the producer side.
>> > >
>> > > My two cents on Radai's sideways partition design:
>> > > 1. If we consider the producer side behavior as doing a two phase
>> commit
>> > > which including the committing the consumer offsets, it is a little
>> > awkward
>> > > that we allow uncommitted message goes into the main log and rely on
>> the
>> > > consumer to filter out. So semantic wise I think it would be better
>> if we
>> > > can avoid this. Radai's suggestion is actually intuitive because if
>> the
>> > > brokers do not want to expose uncommitted transactions to the
>> consumer,
>> > the
>> > > brokers have to buffer it.
>> > >
>> > > 2. Regarding the efficiency. I think may be it worth looking at the
>> > > efficiency cost v.s benefit. The efficiency includes both server side
>> > > efficiency and consumer side efficiency.
>> > >
>> > > Regarding the server side efficiency, the current proposal would
>> probably
>> > > have better efficiency regardless of whether something goes wrong.
>> > Radai's
>> > > suggestion would put more burden on the server side. If nothing goes
>> > wrong
>> > > we always pay the cost of having double copy of the transactional
>> > messages
>> > > and do not get the semantic benefit. But if something goes wrong, the
>> > > efficiency cost we pay we get us a better semantic.
>> > >
>> > > For the consumer side efficiency, because there is no need to buffer
>> the
>> > > uncommitted messages. The current proposal may have to potentially
>> buffer
>> > > uncommitted messages so it would be less efficient than Radai's
>> > suggestion
>> > > when a transaction aborts. When everything goes well, both design
>> seems
>> > > having the similar performance. However, it depends on whether we are
>> > > willing to loosen the consumer side transaction guarantee that I
>> > mentioned
>> > > earlier to Apurva.
>> > >
>> > > Currently the biggest pressure on the consumer side is that it has to
>> > > buffer incomplete transactions. There are two reasons for it,
>> > > A. A transaction may be aborted so we cannot expose the messages to
>> the
>> > > users.
>> > > B. We want to return all or none of the messages in a transaction in
>> ONE
>> > > partition.
>> > >
>> > > While reason A is mandatory, I think reason B may be discussable.
>> Radai's
>> > > design actually removes reason A because there is no uncommitted
>> messages
>> > > exposed to the consumers. This may potentially give us a chance to
>> > > significantly improve consumer side efficiency in normal cases. It
>> again
>> > > depends on the use case, i.e. whether user can process a transaction
>> > > progressively (message by message) or it has to be buffered and
>> returned
>> > > all together. If in most cases, users can process the transactions
>> > message
>> > > by message (most stream processing tasks probably can do so), then
>> with
>> > > Radai's proposal we don't need to buffer the transactions for the
>> users
>> > > anymore, which is a big difference. For the latter case, the consumer
>> may
>> > > have to buffer the incomplete transactions otherwise we are just
>> throwing
>> > > the burden onto the users.
>> > >
>> > > Thanks,
>> > >
>> > > Jiangjie (Becket) Qin
>> > >
>> > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
>> > >
>> > > > Yeah good point. I relent!
>> > > >
>> > > > -jay
>> > > >
>> > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <jason@confluent.io
>> >
>> > > > wrote:
>> > > >
>> > > > > Jay/Ismael,
>> > > > >
>> > > > >
>> > > > >
>> > > > > I agree that lazy initialization of metadata seems unavoidable.
>> > > Ideally,
>> > > > we
>> > > > >
>> > > > > could follow the same pattern for transactions, but remember that
>> in
>> > > the
>> > > > >
>> > > > > consumer+producer use case, the initialization needs to be
>> completed
>> > > > prior
>> > > > >
>> > > > > to setting the consumer's position. Otherwise we risk reading
>> stale
>> > > > >
>> > > > > offsets. But it would be pretty awkward if you have to begin a
>> > > > transaction
>> > > > >
>> > > > > first to ensure that your consumer can read the right offset from
>> the
>> > > > >
>> > > > > consumer, right? It's a bit easier to explain that you should
>> always
>> > > call
>> > > > >
>> > > > > `producer.init()` prior to initializing the consumer. Users would
>> > > > probably
>> > > > >
>> > > > > get this right without any special effort.
>> > > > >
>> > > > >
>> > > > >
>> > > > > -Jason
>> > > > >
>> > > > >
>> > > > >
>> > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
>> rsivaram@pivotal.io
>> > >
>> > > > > wrote:
>> > > > >
>> > > > >
>> > > > >
>> > > > > > Hi Apurva,
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > Thank you for the answers. Just one follow-on.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > 15. Let me rephrase my original question. If all control
>> messages
>> > > > > (messages
>> > > > >
>> > > > > > to transaction logs and markers on user logs) were acknowledged
>> > only
>> > > > > after
>> > > > >
>> > > > > > flushing the log segment, will transactions become durable in
>> the
>> > > > >
>> > > > > > traditional sense (i.e. not restricted to min.insync.replicas
>> > > > failures) ?
>> > > > >
>> > > > > > This is not a suggestion to update the KIP. It seems to me that
>> the
>> > > > > design
>> > > > >
>> > > > > > enables full durability if required in the future with a rather
>> > > > >
>> > > > > > non-intrusive change. I just wanted to make sure I haven't
>> missed
>> > > > > anything
>> > > > >
>> > > > > > fundamental that prevents Kafka from doing this.
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in>
>> wrote:
>> > > > >
>> > > > > >
>> > > > >
>> > > > > > > Hi Apurva,
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > Thanks for the detailed answers... and sorry for the late
>> reply!
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > It does sound like, if the input-partitions-to-app-id mapping
>> > never
>> > > > >
>> > > > > > > changes, the existing fencing mechanisms should prevent
>> > duplicates.
>> > > > >
>> > > > > > Great!
>> > > > >
>> > > > > > > I'm a bit concerned the proposed API will be delicate to
>> program
>> > > > > against
>> > > > >
>> > > > > > > successfully -- even in the simple case, we need to create a
>> new
>> > > > > producer
>> > > > >
>> > > > > > > instance per input partition, and anything fancier is going to
>> > need
>> > > > its
>> > > > >
>> > > > > > own
>> > > > >
>> > > > > > > implementation of the Streams/Samza-style 'task' idea -- but
>> that
>> > > may
>> > > > > be
>> > > > >
>> > > > > > > fine for this sort of advanced feature.
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > For the second question, I notice that Jason also elaborated
>> on
>> > > this
>> > > > >
>> > > > > > > downthread:
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > > We also looked at removing the producer ID.
>> > > > >
>> > > > > > > > This was discussed somewhere above, but basically the idea
>> is
>> > to
>> > > > > store
>> > > > >
>> > > > > > > the
>> > > > >
>> > > > > > > > AppID in the message set header directly and avoid the
>> mapping
>> > to
>> > > > >
>> > > > > > > producer
>> > > > >
>> > > > > > > > ID altogether. As long as batching isn't too bad, the
>> impact on
>> > > > total
>> > > > >
>> > > > > > > size
>> > > > >
>> > > > > > > > may not be too bad, but we were ultimately more comfortable
>> > with
>> > > a
>> > > > >
>> > > > > > fixed
>> > > > >
>> > > > > > > > size ID.
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > ...which suggests that the distinction is useful for
>> performance,
>> > > but
>> > > > > not
>> > > > >
>> > > > > > > necessary for correctness, which makes good sense to me.
>> (Would a
>> > > > > 128-bid
>> > > > >
>> > > > > > > ID be a reasonable compromise? That's enough room for a UUID,
>> or
>> > a
>> > > > >
>> > > > > > > reasonable hash of an arbitrary string, and has only a
>> marginal
>> > > > > increase
>> > > > >
>> > > > > > on
>> > > > >
>> > > > > > > the message size.)
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
>> > apurva@confluent.io
>> > > >
>> > > > >
>> > > > > > wrote:
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > > > > Hi Ben,
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > Now, on to your first question of how deal with consumer
>> > > > rebalances.
>> > > > >
>> > > > > > The
>> > > > >
>> > > > > > > > short answer is that the application needs to ensure that
>> the
>> > the
>> > > > >
>> > > > > > > > assignment of input partitions to appId is consistent across
>> > > > >
>> > > > > > rebalances.
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > For Kafka streams, they already ensure that the mapping of
>> > input
>> > > > >
>> > > > > > > partitions
>> > > > >
>> > > > > > > > to task Id is invariant across rebalances by implementing a
>> > > custom
>> > > > >
>> > > > > > sticky
>> > > > >
>> > > > > > > > assignor. Other non-streams apps can trivially have one
>> > producer
>> > > > per
>> > > > >
>> > > > > > > input
>> > > > >
>> > > > > > > > partition and have the appId be the same as the partition
>> > number
>> > > to
>> > > > >
>> > > > > > > achieve
>> > > > >
>> > > > > > > > the same effect.
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > With this precondition in place, we can maintain
>> transactions
>> > > > across
>> > > > >
>> > > > > > > > rebalances.
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > Hope this answers your question.
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > Thanks,
>> > > > >
>> > > > > > > > Apurva
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
>> > wrote:
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > > > > Thanks for this! I'm looking forward to going through the
>> > full
>> > > > >
>> > > > > > proposal
>> > > > >
>> > > > > > > > in
>> > > > >
>> > > > > > > > > detail soon; a few early questions:
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > First: what happens when a consumer rebalances in the
>> middle
>> > > of a
>> > > > >
>> > > > > > > > > transaction? The full documentation suggests that such a
>> > > > > transaction
>> > > > >
>> > > > > > > > ought
>> > > > >
>> > > > > > > > > to be rejected:
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > > [...] if a rebalance has happened and this consumer
>> > > > >
>> > > > > > > > > > instance becomes a zombie, even if this offset message
>> is
>> > > > > appended
>> > > > >
>> > > > > > in
>> > > > >
>> > > > > > > > the
>> > > > >
>> > > > > > > > > > offset topic, the transaction will be rejected later on
>> > when
>> > > it
>> > > > >
>> > > > > > tries
>> > > > >
>> > > > > > > > to
>> > > > >
>> > > > > > > > > > commit the transaction via the EndTxnRequest.
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > ...but it's unclear to me how we ensure that a transaction
>> > > can't
>> > > > >
>> > > > > > > complete
>> > > > >
>> > > > > > > > > if a rebalance has happened. (It's quite possible I'm
>> missing
>> > > > >
>> > > > > > something
>> > > > >
>> > > > > > > > > obvious!)
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > As a concrete example: suppose a process with PID 1 adds
>> > > offsets
>> > > > > for
>> > > > >
>> > > > > > > some
>> > > > >
>> > > > > > > > > partition to a transaction; a consumer rebalance happens
>> that
>> > > > > assigns
>> > > > >
>> > > > > > > the
>> > > > >
>> > > > > > > > > partition to a process with PID 2, which adds some
>> offsets to
>> > > its
>> > > > >
>> > > > > > > current
>> > > > >
>> > > > > > > > > transaction; both processes try and commit. Allowing both
>> > > commits
>> > > > >
>> > > > > > would
>> > > > >
>> > > > > > > > > cause the messages to be processed twice -- how is that
>> > > avoided?
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > Second: App IDs normally map to a single PID. It seems
>> like
>> > one
>> > > > > could
>> > > > >
>> > > > > > > do
>> > > > >
>> > > > > > > > > away with the PID concept entirely, and just use App IDs
>> in
>> > > most
>> > > > >
>> > > > > > places
>> > > > >
>> > > > > > > > > that require a PID. This feels like it would be
>> significantly
>> > > > >
>> > > > > > simpler,
>> > > > >
>> > > > > > > > > though it does increase the message size. Are there other
>> > > reasons
>> > > > > why
>> > > > >
>> > > > > > > the
>> > > > >
>> > > > > > > > > App ID / PID split is necessary?
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
>> > > > wangguoz@gmail.com
>> > > > > >
>> > > > >
>> > > > > > > > wrote:
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > > > > Hi all,
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
>> > once
>> > > > >
>> > > > > > > delivery
>> > > > >
>> > > > > > > > > > semantics:
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > >
>> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>> > > > >
>> > > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > >
>> > > > > > > > > > 98+-+Exactly+Once+Delivery+and
>> +Transactional+Messaging>*
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > This KIP adds a transactional messaging mechanism along
>> > with
>> > > an
>> > > > >
>> > > > > > > > > idempotent
>> > > > >
>> > > > > > > > > > producer implementation to make sure that 1) duplicated
>> > > > messages
>> > > > >
>> > > > > > sent
>> > > > >
>> > > > > > > > > from
>> > > > >
>> > > > > > > > > > the same identified producer can be detected on the
>> broker
>> > > > side,
>> > > > >
>> > > > > > and
>> > > > >
>> > > > > > > > 2) a
>> > > > >
>> > > > > > > > > > group of messages sent within a transaction will
>> atomically
>> > > be
>> > > > >
>> > > > > > either
>> > > > >
>> > > > > > > > > > reflected and fetchable to consumers or not as a whole.
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > The above wiki page provides a high-level view of the
>> > > proposed
>> > > > >
>> > > > > > > changes
>> > > > >
>> > > > > > > > as
>> > > > >
>> > > > > > > > > > well as summarized guarantees. Initial draft of the
>> > detailed
>> > > > >
>> > > > > > > > > implementation
>> > > > >
>> > > > > > > > > > design is described in this Google doc:
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > https://docs.google.com/document/d/11Jqy_
>> > > > >
>> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>> > > > >
>> > > > > > > > > > 0wSw9ra8
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > We would love to hear your comments and suggestions.
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > Thanks,
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > > > -- Guozhang
>> > > > >
>> > > > > > > > > >
>> > > > >
>> > > > > > > > >
>> > > > >
>> > > > > > > >
>> > > > >
>> > > > > > >
>> > > > >
>> > > > > >
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Sriram Subramanian <ra...@confluent.io>.
Radai,

I think it is important to understand the key requirements that we don’t
want to compromise. We can then understand the tradeoffs of the different
approaches. We did in fact start with the double journal approach couple of
years back. I will highlight the must have requirements first and then
explain the trade offs based on my understanding.

1. End to end latency for stream processing - This is probably one of the
biggest reasons to support transactions in Kafka. We would like to support
very low latency for end to end processing across steam topologies. This
means you would want your downstream processors to see the output of your
processing immediately. The low latency is a requirement even if we only
expose committed messages.

2. Speculative execution - We would like to go one step further for stream
processing. 99% of the transactions will always succeed. We would like to
take advantage of this and process the messages optimistically even if the
transactions are still unfinished. If the transactions abort, we would do a
cascading abort across the topology. This helps us to complete all the
processing and keep the output ready and expose them once the transactions
are committed. This will help us to significantly bring down the latency
for end to end stream processing and provide the ability to keep exactly
once as the default setting.

3. IO and memory constraints - We would want a solution that takes 2x the
number of writes. This will bring down broker utilization by half. I don’t
really understand the in memory solution (would be useful if you can
explain it more if you think it solves these goals) but the same resource
constraints apply. What has made Kafka successful is the ability to run
very high throughput clusters with very few machines. We would like to keep
this true when a cluster is largely dominated by stream processing
workloads.

4. Provide both read committed and read uncommitted isolation levels - This
is actually a desired feature. This is similar to database isolation levels
(except that we provide only two of them for now). Downstream systems that
need strong guarantees with some performance impact can choose read
committed isolation level. Systems that want to optimize for performance
and can live with approximations would choose read uncommitted options.
This helps to nicely decouple downstream users that would like to share
topics but have different end goals.

There are other obvious goals like correctness of the protocol and
simplicity of the design that needs to be true by default.

Given these goals, the double journal approach was a non starter to enable
low end to end latency and did not provide the ability to do speculative
execution in the future. We also found the resource constraints
(specifically IO/Network) to be unacceptable.

We did understand the complexity of the consumers but it was the best
tradeoff considering the other must have goals. We also thought of another
approach to push the consumer buffering to the broker side. This would
enable multiple consumer groups to share the same buffer pool for a
specific topic partition. However, in the worst case, you would need to
bring the entire log into memory to remove the aborted transaction (for a
consumer that is catching up from time 0). This would also make us loose
zero copy semantics.

I would be excited to hear an option that can solve our must have goals and
still keep the consumer really thin. The abstraction seems fine since we
allow the end users to pick the guarantees they need.

On Mon, Dec 19, 2016 at 10:56 AM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Radai,
>
> I'm not sure if I fully understand what you are proposing, but I
> interpreted it to be similar to a proposal we worked through back at
> LinkedIn. The proposal was to commit to a central txlog topic, and then
> recopy to the destination topic upon transaction commit. The observation on
> that approach at the time were the following:
>
>    1. It is cleaner since the output topics have only committed data!
>    2. You need full replication on the txlog topic to ensure atomicity. We
>    weren't able to come up with a solution where you buffer in memory or
> use
>    renaming tricks the way you are describing. The reason is that once you
>    begin committing you must ensure that the commit eventually succeeds to
>    guarantee atomicity. If you use a transient store you might commit some
>    data and then have a server failure that causes you to lose the rest of
> the
>    transaction.
>    3. Having a single log allows the reader to choose a "read uncommitted"
>    mode that hands out messages immediately. This is important for cases
> where
>    latency is important, especially for stream processing topologies where
>    these latencies stack up across multiple stages.
>
> For the stream processing use case, item (2) is a bit of a deal killer.
> This takes the cost of a transient message write (say the intermediate
> result of a stream processing topology) from 3x writes (assuming 3x
> replication) to 6x writes. This means you basically can't default it on. If
> we can in fact get the cost down to a single buffered write (i.e. 1x the
> data is written to memory and buffered to disk if the transaction is large)
> as in the KIP-98 proposal without too many other negative side effects I
> think that could be compelling.
>
> -Jay
>
>
>
> On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com> wrote:
>
> > regarding efficiency:
> >
> > I'd like to distinguish between server efficiency (resource utilization
> of
> > the broker machine alone) and overall network efficiency (resource
> > utilization on brokers, producers and consumers, including network
> > traffic).
> > my proposal is not as resource-efficient on the broker (although it can
> be,
> > depends on a few trade offs and implementation details). HOWEVER, if i
> look
> > at the overall efficiency:
> >
> >    1.clients would need to either buffer or double-read uncommitted msgs.
> > for N clients reading the stream M times (after re-starts and reconsumes)
> > this would mean a M*N factor in either network BW or disk/memory space
> > (depends on if buffer vs re-read). potentially N*M more broker-side reads
> > too.
> >    2 to reduce the broker side cost several things can be done (this is
> not
> > an either-or list, these are commulative):
> >       2.1 - keep TX logs in mem (+overflow to disk) - trades disk writes
> > for TX resiliency
> >       2.2 - when "appending" TX logs to real partitions - instead of
> > reading from (disk-based) TX log and writing to partition log (x2 disk
> > writes) the TX log can be made a segment file (so file rename, with
> > associated protocol changes). this would avoid double writing by simply
> > making the TX file part of the partition (for large enough TXs. smaller
> > ones can be rewritten).
> >       2.3 - the approach above could be combined with a background
> "defrag"
> > - similar in concept to compaction - to further reduce the total of
> > resulting number of files.
> >
> > I think my main issue with the current proposal, more important than
> > performance, is lack of proper "encapsulation" of transactions - I dont
> > think downstream consumers should see uncommitted msgs. ever.
> >
> >
> > On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com>
> wrote:
> >
> > > @Jason
> > >
> > > Yes, second thought on the number of messages included, the offset
> delta
> > > will probably be sufficient. The use case I encounter before for number
> > of
> > > messages in a message set is an embedded mirror maker on the
> destination
> > > broker side which fetches message directly from the source cluster.
> > Ideally
> > > the destination cluster only needs to check CRC and assign the offsets
> > > because all the message verification has been done by the source
> cluster,
> > > but due to the lack of the number of messages in the message set, we
> have
> > > to decompress the message set to increment offsets correctly. By
> knowing
> > > the number of the messages in the message set, we can avoid doing that.
> > The
> > > offset delta will also help. It's just then the offsets may have holes
> > for
> > > log compacted topics, but that may be fine.
> > >
> > > @Apurva
> > >
> > > I am not sure if it is true that the consumer will either deliver all
> the
> > > message for the entire transaction or none of them from one poll()
> call.
> > If
> > > we allow the transactions to be across partitions, unless the consumer
> > > consumes from all the partitions involved in a transactions, it seems
> > > impossible for it to deliver *all* the messages in a transaction,
> right?
> > A
> > > weaker guarantee is we will deliver all or none of the messages that
> > belong
> > > to the same transaction in ONE partition, but this would be different
> > from
> > > the guarantee from the producer side.
> > >
> > > My two cents on Radai's sideways partition design:
> > > 1. If we consider the producer side behavior as doing a two phase
> commit
> > > which including the committing the consumer offsets, it is a little
> > awkward
> > > that we allow uncommitted message goes into the main log and rely on
> the
> > > consumer to filter out. So semantic wise I think it would be better if
> we
> > > can avoid this. Radai's suggestion is actually intuitive because if the
> > > brokers do not want to expose uncommitted transactions to the consumer,
> > the
> > > brokers have to buffer it.
> > >
> > > 2. Regarding the efficiency. I think may be it worth looking at the
> > > efficiency cost v.s benefit. The efficiency includes both server side
> > > efficiency and consumer side efficiency.
> > >
> > > Regarding the server side efficiency, the current proposal would
> probably
> > > have better efficiency regardless of whether something goes wrong.
> > Radai's
> > > suggestion would put more burden on the server side. If nothing goes
> > wrong
> > > we always pay the cost of having double copy of the transactional
> > messages
> > > and do not get the semantic benefit. But if something goes wrong, the
> > > efficiency cost we pay we get us a better semantic.
> > >
> > > For the consumer side efficiency, because there is no need to buffer
> the
> > > uncommitted messages. The current proposal may have to potentially
> buffer
> > > uncommitted messages so it would be less efficient than Radai's
> > suggestion
> > > when a transaction aborts. When everything goes well, both design seems
> > > having the similar performance. However, it depends on whether we are
> > > willing to loosen the consumer side transaction guarantee that I
> > mentioned
> > > earlier to Apurva.
> > >
> > > Currently the biggest pressure on the consumer side is that it has to
> > > buffer incomplete transactions. There are two reasons for it,
> > > A. A transaction may be aborted so we cannot expose the messages to the
> > > users.
> > > B. We want to return all or none of the messages in a transaction in
> ONE
> > > partition.
> > >
> > > While reason A is mandatory, I think reason B may be discussable.
> Radai's
> > > design actually removes reason A because there is no uncommitted
> messages
> > > exposed to the consumers. This may potentially give us a chance to
> > > significantly improve consumer side efficiency in normal cases. It
> again
> > > depends on the use case, i.e. whether user can process a transaction
> > > progressively (message by message) or it has to be buffered and
> returned
> > > all together. If in most cases, users can process the transactions
> > message
> > > by message (most stream processing tasks probably can do so), then with
> > > Radai's proposal we don't need to buffer the transactions for the users
> > > anymore, which is a big difference. For the latter case, the consumer
> may
> > > have to buffer the incomplete transactions otherwise we are just
> throwing
> > > the burden onto the users.
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Yeah good point. I relent!
> > > >
> > > > -jay
> > > >
> > > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > Jay/Ismael,
> > > > >
> > > > >
> > > > >
> > > > > I agree that lazy initialization of metadata seems unavoidable.
> > > Ideally,
> > > > we
> > > > >
> > > > > could follow the same pattern for transactions, but remember that
> in
> > > the
> > > > >
> > > > > consumer+producer use case, the initialization needs to be
> completed
> > > > prior
> > > > >
> > > > > to setting the consumer's position. Otherwise we risk reading stale
> > > > >
> > > > > offsets. But it would be pretty awkward if you have to begin a
> > > > transaction
> > > > >
> > > > > first to ensure that your consumer can read the right offset from
> the
> > > > >
> > > > > consumer, right? It's a bit easier to explain that you should
> always
> > > call
> > > > >
> > > > > `producer.init()` prior to initializing the consumer. Users would
> > > > probably
> > > > >
> > > > > get this right without any special effort.
> > > > >
> > > > >
> > > > >
> > > > > -Jason
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <
> rsivaram@pivotal.io
> > >
> > > > > wrote:
> > > > >
> > > > >
> > > > >
> > > > > > Hi Apurva,
> > > > >
> > > > > >
> > > > >
> > > > > > Thank you for the answers. Just one follow-on.
> > > > >
> > > > > >
> > > > >
> > > > > > 15. Let me rephrase my original question. If all control messages
> > > > > (messages
> > > > >
> > > > > > to transaction logs and markers on user logs) were acknowledged
> > only
> > > > > after
> > > > >
> > > > > > flushing the log segment, will transactions become durable in the
> > > > >
> > > > > > traditional sense (i.e. not restricted to min.insync.replicas
> > > > failures) ?
> > > > >
> > > > > > This is not a suggestion to update the KIP. It seems to me that
> the
> > > > > design
> > > > >
> > > > > > enables full durability if required in the future with a rather
> > > > >
> > > > > > non-intrusive change. I just wanted to make sure I haven't missed
> > > > > anything
> > > > >
> > > > > > fundamental that prevents Kafka from doing this.
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > >
> > > > >
> > > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > > > >
> > > > > >
> > > > >
> > > > > > > Hi Apurva,
> > > > >
> > > > > > >
> > > > >
> > > > > > > Thanks for the detailed answers... and sorry for the late
> reply!
> > > > >
> > > > > > >
> > > > >
> > > > > > > It does sound like, if the input-partitions-to-app-id mapping
> > never
> > > > >
> > > > > > > changes, the existing fencing mechanisms should prevent
> > duplicates.
> > > > >
> > > > > > Great!
> > > > >
> > > > > > > I'm a bit concerned the proposed API will be delicate to
> program
> > > > > against
> > > > >
> > > > > > > successfully -- even in the simple case, we need to create a
> new
> > > > > producer
> > > > >
> > > > > > > instance per input partition, and anything fancier is going to
> > need
> > > > its
> > > > >
> > > > > > own
> > > > >
> > > > > > > implementation of the Streams/Samza-style 'task' idea -- but
> that
> > > may
> > > > > be
> > > > >
> > > > > > > fine for this sort of advanced feature.
> > > > >
> > > > > > >
> > > > >
> > > > > > > For the second question, I notice that Jason also elaborated on
> > > this
> > > > >
> > > > > > > downthread:
> > > > >
> > > > > > >
> > > > >
> > > > > > > > We also looked at removing the producer ID.
> > > > >
> > > > > > > > This was discussed somewhere above, but basically the idea is
> > to
> > > > > store
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > AppID in the message set header directly and avoid the
> mapping
> > to
> > > > >
> > > > > > > producer
> > > > >
> > > > > > > > ID altogether. As long as batching isn't too bad, the impact
> on
> > > > total
> > > > >
> > > > > > > size
> > > > >
> > > > > > > > may not be too bad, but we were ultimately more comfortable
> > with
> > > a
> > > > >
> > > > > > fixed
> > > > >
> > > > > > > > size ID.
> > > > >
> > > > > > >
> > > > >
> > > > > > > ...which suggests that the distinction is useful for
> performance,
> > > but
> > > > > not
> > > > >
> > > > > > > necessary for correctness, which makes good sense to me.
> (Would a
> > > > > 128-bid
> > > > >
> > > > > > > ID be a reasonable compromise? That's enough room for a UUID,
> or
> > a
> > > > >
> > > > > > > reasonable hash of an arbitrary string, and has only a marginal
> > > > > increase
> > > > >
> > > > > > on
> > > > >
> > > > > > > the message size.)
> > > > >
> > > > > > >
> > > > >
> > > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> > apurva@confluent.io
> > > >
> > > > >
> > > > > > wrote:
> > > > >
> > > > > > >
> > > > >
> > > > > > > > Hi Ben,
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Now, on to your first question of how deal with consumer
> > > > rebalances.
> > > > >
> > > > > > The
> > > > >
> > > > > > > > short answer is that the application needs to ensure that the
> > the
> > > > >
> > > > > > > > assignment of input partitions to appId is consistent across
> > > > >
> > > > > > rebalances.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > For Kafka streams, they already ensure that the mapping of
> > input
> > > > >
> > > > > > > partitions
> > > > >
> > > > > > > > to task Id is invariant across rebalances by implementing a
> > > custom
> > > > >
> > > > > > sticky
> > > > >
> > > > > > > > assignor. Other non-streams apps can trivially have one
> > producer
> > > > per
> > > > >
> > > > > > > input
> > > > >
> > > > > > > > partition and have the appId be the same as the partition
> > number
> > > to
> > > > >
> > > > > > > achieve
> > > > >
> > > > > > > > the same effect.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > With this precondition in place, we can maintain transactions
> > > > across
> > > > >
> > > > > > > > rebalances.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Hope this answers your question.
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > Thanks,
> > > > >
> > > > > > > > Apurva
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> > wrote:
> > > > >
> > > > > > > >
> > > > >
> > > > > > > > > Thanks for this! I'm looking forward to going through the
> > full
> > > > >
> > > > > > proposal
> > > > >
> > > > > > > > in
> > > > >
> > > > > > > > > detail soon; a few early questions:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > First: what happens when a consumer rebalances in the
> middle
> > > of a
> > > > >
> > > > > > > > > transaction? The full documentation suggests that such a
> > > > > transaction
> > > > >
> > > > > > > > ought
> > > > >
> > > > > > > > > to be rejected:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > > [...] if a rebalance has happened and this consumer
> > > > >
> > > > > > > > > > instance becomes a zombie, even if this offset message is
> > > > > appended
> > > > >
> > > > > > in
> > > > >
> > > > > > > > the
> > > > >
> > > > > > > > > > offset topic, the transaction will be rejected later on
> > when
> > > it
> > > > >
> > > > > > tries
> > > > >
> > > > > > > > to
> > > > >
> > > > > > > > > > commit the transaction via the EndTxnRequest.
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > ...but it's unclear to me how we ensure that a transaction
> > > can't
> > > > >
> > > > > > > complete
> > > > >
> > > > > > > > > if a rebalance has happened. (It's quite possible I'm
> missing
> > > > >
> > > > > > something
> > > > >
> > > > > > > > > obvious!)
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > As a concrete example: suppose a process with PID 1 adds
> > > offsets
> > > > > for
> > > > >
> > > > > > > some
> > > > >
> > > > > > > > > partition to a transaction; a consumer rebalance happens
> that
> > > > > assigns
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > > partition to a process with PID 2, which adds some offsets
> to
> > > its
> > > > >
> > > > > > > current
> > > > >
> > > > > > > > > transaction; both processes try and commit. Allowing both
> > > commits
> > > > >
> > > > > > would
> > > > >
> > > > > > > > > cause the messages to be processed twice -- how is that
> > > avoided?
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > Second: App IDs normally map to a single PID. It seems like
> > one
> > > > > could
> > > > >
> > > > > > > do
> > > > >
> > > > > > > > > away with the PID concept entirely, and just use App IDs in
> > > most
> > > > >
> > > > > > places
> > > > >
> > > > > > > > > that require a PID. This feels like it would be
> significantly
> > > > >
> > > > > > simpler,
> > > > >
> > > > > > > > > though it does increase the message size. Are there other
> > > reasons
> > > > > why
> > > > >
> > > > > > > the
> > > > >
> > > > > > > > > App ID / PID split is necessary?
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > >
> > > > > > > > wrote:
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > > > > Hi all,
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> > once
> > > > >
> > > > > > > delivery
> > > > >
> > > > > > > > > > semantics:
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >
> > > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >
> > > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > This KIP adds a transactional messaging mechanism along
> > with
> > > an
> > > > >
> > > > > > > > > idempotent
> > > > >
> > > > > > > > > > producer implementation to make sure that 1) duplicated
> > > > messages
> > > > >
> > > > > > sent
> > > > >
> > > > > > > > > from
> > > > >
> > > > > > > > > > the same identified producer can be detected on the
> broker
> > > > side,
> > > > >
> > > > > > and
> > > > >
> > > > > > > > 2) a
> > > > >
> > > > > > > > > > group of messages sent within a transaction will
> atomically
> > > be
> > > > >
> > > > > > either
> > > > >
> > > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > The above wiki page provides a high-level view of the
> > > proposed
> > > > >
> > > > > > > changes
> > > > >
> > > > > > > > as
> > > > >
> > > > > > > > > > well as summarized guarantees. Initial draft of the
> > detailed
> > > > >
> > > > > > > > > implementation
> > > > >
> > > > > > > > > > design is described in this Google doc:
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > > >
> > > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > >
> > > > > > > > > > 0wSw9ra8
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > Thanks,
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > > > -- Guozhang
> > > > >
> > > > > > > > > >
> > > > >
> > > > > > > > >
> > > > >
> > > > > > > >
> > > > >
> > > > > > >
> > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Radai,

I'm not sure if I fully understand what you are proposing, but I
interpreted it to be similar to a proposal we worked through back at
LinkedIn. The proposal was to commit to a central txlog topic, and then
recopy to the destination topic upon transaction commit. The observation on
that approach at the time were the following:

   1. It is cleaner since the output topics have only committed data!
   2. You need full replication on the txlog topic to ensure atomicity. We
   weren't able to come up with a solution where you buffer in memory or use
   renaming tricks the way you are describing. The reason is that once you
   begin committing you must ensure that the commit eventually succeeds to
   guarantee atomicity. If you use a transient store you might commit some
   data and then have a server failure that causes you to lose the rest of the
   transaction.
   3. Having a single log allows the reader to choose a "read uncommitted"
   mode that hands out messages immediately. This is important for cases where
   latency is important, especially for stream processing topologies where
   these latencies stack up across multiple stages.

For the stream processing use case, item (2) is a bit of a deal killer.
This takes the cost of a transient message write (say the intermediate
result of a stream processing topology) from 3x writes (assuming 3x
replication) to 6x writes. This means you basically can't default it on. If
we can in fact get the cost down to a single buffered write (i.e. 1x the
data is written to memory and buffered to disk if the transaction is large)
as in the KIP-98 proposal without too many other negative side effects I
think that could be compelling.

-Jay



On Mon, Dec 19, 2016 at 9:36 AM, radai <ra...@gmail.com> wrote:

> regarding efficiency:
>
> I'd like to distinguish between server efficiency (resource utilization of
> the broker machine alone) and overall network efficiency (resource
> utilization on brokers, producers and consumers, including network
> traffic).
> my proposal is not as resource-efficient on the broker (although it can be,
> depends on a few trade offs and implementation details). HOWEVER, if i look
> at the overall efficiency:
>
>    1.clients would need to either buffer or double-read uncommitted msgs.
> for N clients reading the stream M times (after re-starts and reconsumes)
> this would mean a M*N factor in either network BW or disk/memory space
> (depends on if buffer vs re-read). potentially N*M more broker-side reads
> too.
>    2 to reduce the broker side cost several things can be done (this is not
> an either-or list, these are commulative):
>       2.1 - keep TX logs in mem (+overflow to disk) - trades disk writes
> for TX resiliency
>       2.2 - when "appending" TX logs to real partitions - instead of
> reading from (disk-based) TX log and writing to partition log (x2 disk
> writes) the TX log can be made a segment file (so file rename, with
> associated protocol changes). this would avoid double writing by simply
> making the TX file part of the partition (for large enough TXs. smaller
> ones can be rewritten).
>       2.3 - the approach above could be combined with a background "defrag"
> - similar in concept to compaction - to further reduce the total of
> resulting number of files.
>
> I think my main issue with the current proposal, more important than
> performance, is lack of proper "encapsulation" of transactions - I dont
> think downstream consumers should see uncommitted msgs. ever.
>
>
> On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com> wrote:
>
> > @Jason
> >
> > Yes, second thought on the number of messages included, the offset delta
> > will probably be sufficient. The use case I encounter before for number
> of
> > messages in a message set is an embedded mirror maker on the destination
> > broker side which fetches message directly from the source cluster.
> Ideally
> > the destination cluster only needs to check CRC and assign the offsets
> > because all the message verification has been done by the source cluster,
> > but due to the lack of the number of messages in the message set, we have
> > to decompress the message set to increment offsets correctly. By knowing
> > the number of the messages in the message set, we can avoid doing that.
> The
> > offset delta will also help. It's just then the offsets may have holes
> for
> > log compacted topics, but that may be fine.
> >
> > @Apurva
> >
> > I am not sure if it is true that the consumer will either deliver all the
> > message for the entire transaction or none of them from one poll() call.
> If
> > we allow the transactions to be across partitions, unless the consumer
> > consumes from all the partitions involved in a transactions, it seems
> > impossible for it to deliver *all* the messages in a transaction, right?
> A
> > weaker guarantee is we will deliver all or none of the messages that
> belong
> > to the same transaction in ONE partition, but this would be different
> from
> > the guarantee from the producer side.
> >
> > My two cents on Radai's sideways partition design:
> > 1. If we consider the producer side behavior as doing a two phase commit
> > which including the committing the consumer offsets, it is a little
> awkward
> > that we allow uncommitted message goes into the main log and rely on the
> > consumer to filter out. So semantic wise I think it would be better if we
> > can avoid this. Radai's suggestion is actually intuitive because if the
> > brokers do not want to expose uncommitted transactions to the consumer,
> the
> > brokers have to buffer it.
> >
> > 2. Regarding the efficiency. I think may be it worth looking at the
> > efficiency cost v.s benefit. The efficiency includes both server side
> > efficiency and consumer side efficiency.
> >
> > Regarding the server side efficiency, the current proposal would probably
> > have better efficiency regardless of whether something goes wrong.
> Radai's
> > suggestion would put more burden on the server side. If nothing goes
> wrong
> > we always pay the cost of having double copy of the transactional
> messages
> > and do not get the semantic benefit. But if something goes wrong, the
> > efficiency cost we pay we get us a better semantic.
> >
> > For the consumer side efficiency, because there is no need to buffer the
> > uncommitted messages. The current proposal may have to potentially buffer
> > uncommitted messages so it would be less efficient than Radai's
> suggestion
> > when a transaction aborts. When everything goes well, both design seems
> > having the similar performance. However, it depends on whether we are
> > willing to loosen the consumer side transaction guarantee that I
> mentioned
> > earlier to Apurva.
> >
> > Currently the biggest pressure on the consumer side is that it has to
> > buffer incomplete transactions. There are two reasons for it,
> > A. A transaction may be aborted so we cannot expose the messages to the
> > users.
> > B. We want to return all or none of the messages in a transaction in ONE
> > partition.
> >
> > While reason A is mandatory, I think reason B may be discussable. Radai's
> > design actually removes reason A because there is no uncommitted messages
> > exposed to the consumers. This may potentially give us a chance to
> > significantly improve consumer side efficiency in normal cases. It again
> > depends on the use case, i.e. whether user can process a transaction
> > progressively (message by message) or it has to be buffered and returned
> > all together. If in most cases, users can process the transactions
> message
> > by message (most stream processing tasks probably can do so), then with
> > Radai's proposal we don't need to buffer the transactions for the users
> > anymore, which is a big difference. For the latter case, the consumer may
> > have to buffer the incomplete transactions otherwise we are just throwing
> > the burden onto the users.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Yeah good point. I relent!
> > >
> > > -jay
> > >
> > > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Jay/Ismael,
> > > >
> > > >
> > > >
> > > > I agree that lazy initialization of metadata seems unavoidable.
> > Ideally,
> > > we
> > > >
> > > > could follow the same pattern for transactions, but remember that in
> > the
> > > >
> > > > consumer+producer use case, the initialization needs to be completed
> > > prior
> > > >
> > > > to setting the consumer's position. Otherwise we risk reading stale
> > > >
> > > > offsets. But it would be pretty awkward if you have to begin a
> > > transaction
> > > >
> > > > first to ensure that your consumer can read the right offset from the
> > > >
> > > > consumer, right? It's a bit easier to explain that you should always
> > call
> > > >
> > > > `producer.init()` prior to initializing the consumer. Users would
> > > probably
> > > >
> > > > get this right without any special effort.
> > > >
> > > >
> > > >
> > > > -Jason
> > > >
> > > >
> > > >
> > > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rsivaram@pivotal.io
> >
> > > > wrote:
> > > >
> > > >
> > > >
> > > > > Hi Apurva,
> > > >
> > > > >
> > > >
> > > > > Thank you for the answers. Just one follow-on.
> > > >
> > > > >
> > > >
> > > > > 15. Let me rephrase my original question. If all control messages
> > > > (messages
> > > >
> > > > > to transaction logs and markers on user logs) were acknowledged
> only
> > > > after
> > > >
> > > > > flushing the log segment, will transactions become durable in the
> > > >
> > > > > traditional sense (i.e. not restricted to min.insync.replicas
> > > failures) ?
> > > >
> > > > > This is not a suggestion to update the KIP. It seems to me that the
> > > > design
> > > >
> > > > > enables full durability if required in the future with a rather
> > > >
> > > > > non-intrusive change. I just wanted to make sure I haven't missed
> > > > anything
> > > >
> > > > > fundamental that prevents Kafka from doing this.
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > >
> > > >
> > > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > > >
> > > > >
> > > >
> > > > > > Hi Apurva,
> > > >
> > > > > >
> > > >
> > > > > > Thanks for the detailed answers... and sorry for the late reply!
> > > >
> > > > > >
> > > >
> > > > > > It does sound like, if the input-partitions-to-app-id mapping
> never
> > > >
> > > > > > changes, the existing fencing mechanisms should prevent
> duplicates.
> > > >
> > > > > Great!
> > > >
> > > > > > I'm a bit concerned the proposed API will be delicate to program
> > > > against
> > > >
> > > > > > successfully -- even in the simple case, we need to create a new
> > > > producer
> > > >
> > > > > > instance per input partition, and anything fancier is going to
> need
> > > its
> > > >
> > > > > own
> > > >
> > > > > > implementation of the Streams/Samza-style 'task' idea -- but that
> > may
> > > > be
> > > >
> > > > > > fine for this sort of advanced feature.
> > > >
> > > > > >
> > > >
> > > > > > For the second question, I notice that Jason also elaborated on
> > this
> > > >
> > > > > > downthread:
> > > >
> > > > > >
> > > >
> > > > > > > We also looked at removing the producer ID.
> > > >
> > > > > > > This was discussed somewhere above, but basically the idea is
> to
> > > > store
> > > >
> > > > > > the
> > > >
> > > > > > > AppID in the message set header directly and avoid the mapping
> to
> > > >
> > > > > > producer
> > > >
> > > > > > > ID altogether. As long as batching isn't too bad, the impact on
> > > total
> > > >
> > > > > > size
> > > >
> > > > > > > may not be too bad, but we were ultimately more comfortable
> with
> > a
> > > >
> > > > > fixed
> > > >
> > > > > > > size ID.
> > > >
> > > > > >
> > > >
> > > > > > ...which suggests that the distinction is useful for performance,
> > but
> > > > not
> > > >
> > > > > > necessary for correctness, which makes good sense to me. (Would a
> > > > 128-bid
> > > >
> > > > > > ID be a reasonable compromise? That's enough room for a UUID, or
> a
> > > >
> > > > > > reasonable hash of an arbitrary string, and has only a marginal
> > > > increase
> > > >
> > > > > on
> > > >
> > > > > > the message size.)
> > > >
> > > > > >
> > > >
> > > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <
> apurva@confluent.io
> > >
> > > >
> > > > > wrote:
> > > >
> > > > > >
> > > >
> > > > > > > Hi Ben,
> > > >
> > > > > > >
> > > >
> > > > > > > Now, on to your first question of how deal with consumer
> > > rebalances.
> > > >
> > > > > The
> > > >
> > > > > > > short answer is that the application needs to ensure that the
> the
> > > >
> > > > > > > assignment of input partitions to appId is consistent across
> > > >
> > > > > rebalances.
> > > >
> > > > > > >
> > > >
> > > > > > > For Kafka streams, they already ensure that the mapping of
> input
> > > >
> > > > > > partitions
> > > >
> > > > > > > to task Id is invariant across rebalances by implementing a
> > custom
> > > >
> > > > > sticky
> > > >
> > > > > > > assignor. Other non-streams apps can trivially have one
> producer
> > > per
> > > >
> > > > > > input
> > > >
> > > > > > > partition and have the appId be the same as the partition
> number
> > to
> > > >
> > > > > > achieve
> > > >
> > > > > > > the same effect.
> > > >
> > > > > > >
> > > >
> > > > > > > With this precondition in place, we can maintain transactions
> > > across
> > > >
> > > > > > > rebalances.
> > > >
> > > > > > >
> > > >
> > > > > > > Hope this answers your question.
> > > >
> > > > > > >
> > > >
> > > > > > > Thanks,
> > > >
> > > > > > > Apurva
> > > >
> > > > > > >
> > > >
> > > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in>
> wrote:
> > > >
> > > > > > >
> > > >
> > > > > > > > Thanks for this! I'm looking forward to going through the
> full
> > > >
> > > > > proposal
> > > >
> > > > > > > in
> > > >
> > > > > > > > detail soon; a few early questions:
> > > >
> > > > > > > >
> > > >
> > > > > > > > First: what happens when a consumer rebalances in the middle
> > of a
> > > >
> > > > > > > > transaction? The full documentation suggests that such a
> > > > transaction
> > > >
> > > > > > > ought
> > > >
> > > > > > > > to be rejected:
> > > >
> > > > > > > >
> > > >
> > > > > > > > > [...] if a rebalance has happened and this consumer
> > > >
> > > > > > > > > instance becomes a zombie, even if this offset message is
> > > > appended
> > > >
> > > > > in
> > > >
> > > > > > > the
> > > >
> > > > > > > > > offset topic, the transaction will be rejected later on
> when
> > it
> > > >
> > > > > tries
> > > >
> > > > > > > to
> > > >
> > > > > > > > > commit the transaction via the EndTxnRequest.
> > > >
> > > > > > > >
> > > >
> > > > > > > > ...but it's unclear to me how we ensure that a transaction
> > can't
> > > >
> > > > > > complete
> > > >
> > > > > > > > if a rebalance has happened. (It's quite possible I'm missing
> > > >
> > > > > something
> > > >
> > > > > > > > obvious!)
> > > >
> > > > > > > >
> > > >
> > > > > > > > As a concrete example: suppose a process with PID 1 adds
> > offsets
> > > > for
> > > >
> > > > > > some
> > > >
> > > > > > > > partition to a transaction; a consumer rebalance happens that
> > > > assigns
> > > >
> > > > > > the
> > > >
> > > > > > > > partition to a process with PID 2, which adds some offsets to
> > its
> > > >
> > > > > > current
> > > >
> > > > > > > > transaction; both processes try and commit. Allowing both
> > commits
> > > >
> > > > > would
> > > >
> > > > > > > > cause the messages to be processed twice -- how is that
> > avoided?
> > > >
> > > > > > > >
> > > >
> > > > > > > > Second: App IDs normally map to a single PID. It seems like
> one
> > > > could
> > > >
> > > > > > do
> > > >
> > > > > > > > away with the PID concept entirely, and just use App IDs in
> > most
> > > >
> > > > > places
> > > >
> > > > > > > > that require a PID. This feels like it would be significantly
> > > >
> > > > > simpler,
> > > >
> > > > > > > > though it does increase the message size. Are there other
> > reasons
> > > > why
> > > >
> > > > > > the
> > > >
> > > > > > > > App ID / PID split is necessary?
> > > >
> > > > > > > >
> > > >
> > > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > > >
> > > >
> > > > > > > wrote:
> > > >
> > > > > > > >
> > > >
> > > > > > > > > Hi all,
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
> > > >
> > > > > > delivery
> > > >
> > > > > > > > > semantics:
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >
> > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >
> > > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > This KIP adds a transactional messaging mechanism along
> with
> > an
> > > >
> > > > > > > > idempotent
> > > >
> > > > > > > > > producer implementation to make sure that 1) duplicated
> > > messages
> > > >
> > > > > sent
> > > >
> > > > > > > > from
> > > >
> > > > > > > > > the same identified producer can be detected on the broker
> > > side,
> > > >
> > > > > and
> > > >
> > > > > > > 2) a
> > > >
> > > > > > > > > group of messages sent within a transaction will atomically
> > be
> > > >
> > > > > either
> > > >
> > > > > > > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > The above wiki page provides a high-level view of the
> > proposed
> > > >
> > > > > > changes
> > > >
> > > > > > > as
> > > >
> > > > > > > > > well as summarized guarantees. Initial draft of the
> detailed
> > > >
> > > > > > > > implementation
> > > >
> > > > > > > > > design is described in this Google doc:
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > https://docs.google.com/document/d/11Jqy_
> > > >
> > > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > >
> > > > > > > > > 0wSw9ra8
> > > >
> > > > > > > > >
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > We would love to hear your comments and suggestions.
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > Thanks,
> > > >
> > > > > > > > >
> > > >
> > > > > > > > > -- Guozhang
> > > >
> > > > > > > > >
> > > >
> > > > > > > >
> > > >
> > > > > > >
> > > >
> > > > > >
> > > >
> > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by radai <ra...@gmail.com>.
regarding efficiency:

I'd like to distinguish between server efficiency (resource utilization of
the broker machine alone) and overall network efficiency (resource
utilization on brokers, producers and consumers, including network traffic).
my proposal is not as resource-efficient on the broker (although it can be,
depends on a few trade offs and implementation details). HOWEVER, if i look
at the overall efficiency:

   1.clients would need to either buffer or double-read uncommitted msgs.
for N clients reading the stream M times (after re-starts and reconsumes)
this would mean a M*N factor in either network BW or disk/memory space
(depends on if buffer vs re-read). potentially N*M more broker-side reads
too.
   2 to reduce the broker side cost several things can be done (this is not
an either-or list, these are commulative):
      2.1 - keep TX logs in mem (+overflow to disk) - trades disk writes
for TX resiliency
      2.2 - when "appending" TX logs to real partitions - instead of
reading from (disk-based) TX log and writing to partition log (x2 disk
writes) the TX log can be made a segment file (so file rename, with
associated protocol changes). this would avoid double writing by simply
making the TX file part of the partition (for large enough TXs. smaller
ones can be rewritten).
      2.3 - the approach above could be combined with a background "defrag"
- similar in concept to compaction - to further reduce the total of
resulting number of files.

I think my main issue with the current proposal, more important than
performance, is lack of proper "encapsulation" of transactions - I dont
think downstream consumers should see uncommitted msgs. ever.


On Sun, Dec 18, 2016 at 10:19 PM, Becket Qin <be...@gmail.com> wrote:

> @Jason
>
> Yes, second thought on the number of messages included, the offset delta
> will probably be sufficient. The use case I encounter before for number of
> messages in a message set is an embedded mirror maker on the destination
> broker side which fetches message directly from the source cluster. Ideally
> the destination cluster only needs to check CRC and assign the offsets
> because all the message verification has been done by the source cluster,
> but due to the lack of the number of messages in the message set, we have
> to decompress the message set to increment offsets correctly. By knowing
> the number of the messages in the message set, we can avoid doing that. The
> offset delta will also help. It's just then the offsets may have holes for
> log compacted topics, but that may be fine.
>
> @Apurva
>
> I am not sure if it is true that the consumer will either deliver all the
> message for the entire transaction or none of them from one poll() call. If
> we allow the transactions to be across partitions, unless the consumer
> consumes from all the partitions involved in a transactions, it seems
> impossible for it to deliver *all* the messages in a transaction, right? A
> weaker guarantee is we will deliver all or none of the messages that belong
> to the same transaction in ONE partition, but this would be different from
> the guarantee from the producer side.
>
> My two cents on Radai's sideways partition design:
> 1. If we consider the producer side behavior as doing a two phase commit
> which including the committing the consumer offsets, it is a little awkward
> that we allow uncommitted message goes into the main log and rely on the
> consumer to filter out. So semantic wise I think it would be better if we
> can avoid this. Radai's suggestion is actually intuitive because if the
> brokers do not want to expose uncommitted transactions to the consumer, the
> brokers have to buffer it.
>
> 2. Regarding the efficiency. I think may be it worth looking at the
> efficiency cost v.s benefit. The efficiency includes both server side
> efficiency and consumer side efficiency.
>
> Regarding the server side efficiency, the current proposal would probably
> have better efficiency regardless of whether something goes wrong. Radai's
> suggestion would put more burden on the server side. If nothing goes wrong
> we always pay the cost of having double copy of the transactional messages
> and do not get the semantic benefit. But if something goes wrong, the
> efficiency cost we pay we get us a better semantic.
>
> For the consumer side efficiency, because there is no need to buffer the
> uncommitted messages. The current proposal may have to potentially buffer
> uncommitted messages so it would be less efficient than Radai's suggestion
> when a transaction aborts. When everything goes well, both design seems
> having the similar performance. However, it depends on whether we are
> willing to loosen the consumer side transaction guarantee that I mentioned
> earlier to Apurva.
>
> Currently the biggest pressure on the consumer side is that it has to
> buffer incomplete transactions. There are two reasons for it,
> A. A transaction may be aborted so we cannot expose the messages to the
> users.
> B. We want to return all or none of the messages in a transaction in ONE
> partition.
>
> While reason A is mandatory, I think reason B may be discussable. Radai's
> design actually removes reason A because there is no uncommitted messages
> exposed to the consumers. This may potentially give us a chance to
> significantly improve consumer side efficiency in normal cases. It again
> depends on the use case, i.e. whether user can process a transaction
> progressively (message by message) or it has to be buffered and returned
> all together. If in most cases, users can process the transactions message
> by message (most stream processing tasks probably can do so), then with
> Radai's proposal we don't need to buffer the transactions for the users
> anymore, which is a big difference. For the latter case, the consumer may
> have to buffer the incomplete transactions otherwise we are just throwing
> the burden onto the users.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Yeah good point. I relent!
> >
> > -jay
> >
> > On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Jay/Ismael,
> > >
> > >
> > >
> > > I agree that lazy initialization of metadata seems unavoidable.
> Ideally,
> > we
> > >
> > > could follow the same pattern for transactions, but remember that in
> the
> > >
> > > consumer+producer use case, the initialization needs to be completed
> > prior
> > >
> > > to setting the consumer's position. Otherwise we risk reading stale
> > >
> > > offsets. But it would be pretty awkward if you have to begin a
> > transaction
> > >
> > > first to ensure that your consumer can read the right offset from the
> > >
> > > consumer, right? It's a bit easier to explain that you should always
> call
> > >
> > > `producer.init()` prior to initializing the consumer. Users would
> > probably
> > >
> > > get this right without any special effort.
> > >
> > >
> > >
> > > -Jason
> > >
> > >
> > >
> > > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> > > wrote:
> > >
> > >
> > >
> > > > Hi Apurva,
> > >
> > > >
> > >
> > > > Thank you for the answers. Just one follow-on.
> > >
> > > >
> > >
> > > > 15. Let me rephrase my original question. If all control messages
> > > (messages
> > >
> > > > to transaction logs and markers on user logs) were acknowledged only
> > > after
> > >
> > > > flushing the log segment, will transactions become durable in the
> > >
> > > > traditional sense (i.e. not restricted to min.insync.replicas
> > failures) ?
> > >
> > > > This is not a suggestion to update the KIP. It seems to me that the
> > > design
> > >
> > > > enables full durability if required in the future with a rather
> > >
> > > > non-intrusive change. I just wanted to make sure I haven't missed
> > > anything
> > >
> > > > fundamental that prevents Kafka from doing this.
> > >
> > > >
> > >
> > > >
> > >
> > > >
> > >
> > > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> > >
> > > >
> > >
> > > > > Hi Apurva,
> > >
> > > > >
> > >
> > > > > Thanks for the detailed answers... and sorry for the late reply!
> > >
> > > > >
> > >
> > > > > It does sound like, if the input-partitions-to-app-id mapping never
> > >
> > > > > changes, the existing fencing mechanisms should prevent duplicates.
> > >
> > > > Great!
> > >
> > > > > I'm a bit concerned the proposed API will be delicate to program
> > > against
> > >
> > > > > successfully -- even in the simple case, we need to create a new
> > > producer
> > >
> > > > > instance per input partition, and anything fancier is going to need
> > its
> > >
> > > > own
> > >
> > > > > implementation of the Streams/Samza-style 'task' idea -- but that
> may
> > > be
> > >
> > > > > fine for this sort of advanced feature.
> > >
> > > > >
> > >
> > > > > For the second question, I notice that Jason also elaborated on
> this
> > >
> > > > > downthread:
> > >
> > > > >
> > >
> > > > > > We also looked at removing the producer ID.
> > >
> > > > > > This was discussed somewhere above, but basically the idea is to
> > > store
> > >
> > > > > the
> > >
> > > > > > AppID in the message set header directly and avoid the mapping to
> > >
> > > > > producer
> > >
> > > > > > ID altogether. As long as batching isn't too bad, the impact on
> > total
> > >
> > > > > size
> > >
> > > > > > may not be too bad, but we were ultimately more comfortable with
> a
> > >
> > > > fixed
> > >
> > > > > > size ID.
> > >
> > > > >
> > >
> > > > > ...which suggests that the distinction is useful for performance,
> but
> > > not
> > >
> > > > > necessary for correctness, which makes good sense to me. (Would a
> > > 128-bid
> > >
> > > > > ID be a reasonable compromise? That's enough room for a UUID, or a
> > >
> > > > > reasonable hash of an arbitrary string, and has only a marginal
> > > increase
> > >
> > > > on
> > >
> > > > > the message size.)
> > >
> > > > >
> > >
> > > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <apurva@confluent.io
> >
> > >
> > > > wrote:
> > >
> > > > >
> > >
> > > > > > Hi Ben,
> > >
> > > > > >
> > >
> > > > > > Now, on to your first question of how deal with consumer
> > rebalances.
> > >
> > > > The
> > >
> > > > > > short answer is that the application needs to ensure that the the
> > >
> > > > > > assignment of input partitions to appId is consistent across
> > >
> > > > rebalances.
> > >
> > > > > >
> > >
> > > > > > For Kafka streams, they already ensure that the mapping of input
> > >
> > > > > partitions
> > >
> > > > > > to task Id is invariant across rebalances by implementing a
> custom
> > >
> > > > sticky
> > >
> > > > > > assignor. Other non-streams apps can trivially have one producer
> > per
> > >
> > > > > input
> > >
> > > > > > partition and have the appId be the same as the partition number
> to
> > >
> > > > > achieve
> > >
> > > > > > the same effect.
> > >
> > > > > >
> > >
> > > > > > With this precondition in place, we can maintain transactions
> > across
> > >
> > > > > > rebalances.
> > >
> > > > > >
> > >
> > > > > > Hope this answers your question.
> > >
> > > > > >
> > >
> > > > > > Thanks,
> > >
> > > > > > Apurva
> > >
> > > > > >
> > >
> > > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> > >
> > > > > >
> > >
> > > > > > > Thanks for this! I'm looking forward to going through the full
> > >
> > > > proposal
> > >
> > > > > > in
> > >
> > > > > > > detail soon; a few early questions:
> > >
> > > > > > >
> > >
> > > > > > > First: what happens when a consumer rebalances in the middle
> of a
> > >
> > > > > > > transaction? The full documentation suggests that such a
> > > transaction
> > >
> > > > > > ought
> > >
> > > > > > > to be rejected:
> > >
> > > > > > >
> > >
> > > > > > > > [...] if a rebalance has happened and this consumer
> > >
> > > > > > > > instance becomes a zombie, even if this offset message is
> > > appended
> > >
> > > > in
> > >
> > > > > > the
> > >
> > > > > > > > offset topic, the transaction will be rejected later on when
> it
> > >
> > > > tries
> > >
> > > > > > to
> > >
> > > > > > > > commit the transaction via the EndTxnRequest.
> > >
> > > > > > >
> > >
> > > > > > > ...but it's unclear to me how we ensure that a transaction
> can't
> > >
> > > > > complete
> > >
> > > > > > > if a rebalance has happened. (It's quite possible I'm missing
> > >
> > > > something
> > >
> > > > > > > obvious!)
> > >
> > > > > > >
> > >
> > > > > > > As a concrete example: suppose a process with PID 1 adds
> offsets
> > > for
> > >
> > > > > some
> > >
> > > > > > > partition to a transaction; a consumer rebalance happens that
> > > assigns
> > >
> > > > > the
> > >
> > > > > > > partition to a process with PID 2, which adds some offsets to
> its
> > >
> > > > > current
> > >
> > > > > > > transaction; both processes try and commit. Allowing both
> commits
> > >
> > > > would
> > >
> > > > > > > cause the messages to be processed twice -- how is that
> avoided?
> > >
> > > > > > >
> > >
> > > > > > > Second: App IDs normally map to a single PID. It seems like one
> > > could
> > >
> > > > > do
> > >
> > > > > > > away with the PID concept entirely, and just use App IDs in
> most
> > >
> > > > places
> > >
> > > > > > > that require a PID. This feels like it would be significantly
> > >
> > > > simpler,
> > >
> > > > > > > though it does increase the message size. Are there other
> reasons
> > > why
> > >
> > > > > the
> > >
> > > > > > > App ID / PID split is necessary?
> > >
> > > > > > >
> > >
> > > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > >
> > > > > > wrote:
> > >
> > > > > > >
> > >
> > > > > > > > Hi all,
> > >
> > > > > > > >
> > >
> > > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > >
> > > > > delivery
> > >
> > > > > > > > semantics:
> > >
> > > > > > > >
> > >
> > > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >
> > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >
> > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >
> > > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > >
> > > > > > > >
> > >
> > > > > > > > This KIP adds a transactional messaging mechanism along with
> an
> > >
> > > > > > > idempotent
> > >
> > > > > > > > producer implementation to make sure that 1) duplicated
> > messages
> > >
> > > > sent
> > >
> > > > > > > from
> > >
> > > > > > > > the same identified producer can be detected on the broker
> > side,
> > >
> > > > and
> > >
> > > > > > 2) a
> > >
> > > > > > > > group of messages sent within a transaction will atomically
> be
> > >
> > > > either
> > >
> > > > > > > > reflected and fetchable to consumers or not as a whole.
> > >
> > > > > > > >
> > >
> > > > > > > > The above wiki page provides a high-level view of the
> proposed
> > >
> > > > > changes
> > >
> > > > > > as
> > >
> > > > > > > > well as summarized guarantees. Initial draft of the detailed
> > >
> > > > > > > implementation
> > >
> > > > > > > > design is described in this Google doc:
> > >
> > > > > > > >
> > >
> > > > > > > > https://docs.google.com/document/d/11Jqy_
> > >
> > > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > >
> > > > > > > > 0wSw9ra8
> > >
> > > > > > > >
> > >
> > > > > > > >
> > >
> > > > > > > > We would love to hear your comments and suggestions.
> > >
> > > > > > > >
> > >
> > > > > > > > Thanks,
> > >
> > > > > > > >
> > >
> > > > > > > > -- Guozhang
> > >
> > > > > > > >
> > >
> > > > > > >
> > >
> > > > > >
> > >
> > > > >
> > >
> > > >
> > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Becket Qin <be...@gmail.com>.
@Jason

Yes, second thought on the number of messages included, the offset delta
will probably be sufficient. The use case I encounter before for number of
messages in a message set is an embedded mirror maker on the destination
broker side which fetches message directly from the source cluster. Ideally
the destination cluster only needs to check CRC and assign the offsets
because all the message verification has been done by the source cluster,
but due to the lack of the number of messages in the message set, we have
to decompress the message set to increment offsets correctly. By knowing
the number of the messages in the message set, we can avoid doing that. The
offset delta will also help. It's just then the offsets may have holes for
log compacted topics, but that may be fine.

@Apurva

I am not sure if it is true that the consumer will either deliver all the
message for the entire transaction or none of them from one poll() call. If
we allow the transactions to be across partitions, unless the consumer
consumes from all the partitions involved in a transactions, it seems
impossible for it to deliver *all* the messages in a transaction, right? A
weaker guarantee is we will deliver all or none of the messages that belong
to the same transaction in ONE partition, but this would be different from
the guarantee from the producer side.

My two cents on Radai's sideways partition design:
1. If we consider the producer side behavior as doing a two phase commit
which including the committing the consumer offsets, it is a little awkward
that we allow uncommitted message goes into the main log and rely on the
consumer to filter out. So semantic wise I think it would be better if we
can avoid this. Radai's suggestion is actually intuitive because if the
brokers do not want to expose uncommitted transactions to the consumer, the
brokers have to buffer it.

2. Regarding the efficiency. I think may be it worth looking at the
efficiency cost v.s benefit. The efficiency includes both server side
efficiency and consumer side efficiency.

Regarding the server side efficiency, the current proposal would probably
have better efficiency regardless of whether something goes wrong. Radai's
suggestion would put more burden on the server side. If nothing goes wrong
we always pay the cost of having double copy of the transactional messages
and do not get the semantic benefit. But if something goes wrong, the
efficiency cost we pay we get us a better semantic.

For the consumer side efficiency, because there is no need to buffer the
uncommitted messages. The current proposal may have to potentially buffer
uncommitted messages so it would be less efficient than Radai's suggestion
when a transaction aborts. When everything goes well, both design seems
having the similar performance. However, it depends on whether we are
willing to loosen the consumer side transaction guarantee that I mentioned
earlier to Apurva.

Currently the biggest pressure on the consumer side is that it has to
buffer incomplete transactions. There are two reasons for it,
A. A transaction may be aborted so we cannot expose the messages to the
users.
B. We want to return all or none of the messages in a transaction in ONE
partition.

While reason A is mandatory, I think reason B may be discussable. Radai's
design actually removes reason A because there is no uncommitted messages
exposed to the consumers. This may potentially give us a chance to
significantly improve consumer side efficiency in normal cases. It again
depends on the use case, i.e. whether user can process a transaction
progressively (message by message) or it has to be buffered and returned
all together. If in most cases, users can process the transactions message
by message (most stream processing tasks probably can do so), then with
Radai's proposal we don't need to buffer the transactions for the users
anymore, which is a big difference. For the latter case, the consumer may
have to buffer the incomplete transactions otherwise we are just throwing
the burden onto the users.

Thanks,

Jiangjie (Becket) Qin

On Fri, Dec 16, 2016 at 4:56 PM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah good point. I relent!
>
> -jay
>
> On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Jay/Ismael,
> >
> >
> >
> > I agree that lazy initialization of metadata seems unavoidable. Ideally,
> we
> >
> > could follow the same pattern for transactions, but remember that in the
> >
> > consumer+producer use case, the initialization needs to be completed
> prior
> >
> > to setting the consumer's position. Otherwise we risk reading stale
> >
> > offsets. But it would be pretty awkward if you have to begin a
> transaction
> >
> > first to ensure that your consumer can read the right offset from the
> >
> > consumer, right? It's a bit easier to explain that you should always call
> >
> > `producer.init()` prior to initializing the consumer. Users would
> probably
> >
> > get this right without any special effort.
> >
> >
> >
> > -Jason
> >
> >
> >
> > On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> > wrote:
> >
> >
> >
> > > Hi Apurva,
> >
> > >
> >
> > > Thank you for the answers. Just one follow-on.
> >
> > >
> >
> > > 15. Let me rephrase my original question. If all control messages
> > (messages
> >
> > > to transaction logs and markers on user logs) were acknowledged only
> > after
> >
> > > flushing the log segment, will transactions become durable in the
> >
> > > traditional sense (i.e. not restricted to min.insync.replicas
> failures) ?
> >
> > > This is not a suggestion to update the KIP. It seems to me that the
> > design
> >
> > > enables full durability if required in the future with a rather
> >
> > > non-intrusive change. I just wanted to make sure I haven't missed
> > anything
> >
> > > fundamental that prevents Kafka from doing this.
> >
> > >
> >
> > >
> >
> > >
> >
> > > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
> >
> > >
> >
> > > > Hi Apurva,
> >
> > > >
> >
> > > > Thanks for the detailed answers... and sorry for the late reply!
> >
> > > >
> >
> > > > It does sound like, if the input-partitions-to-app-id mapping never
> >
> > > > changes, the existing fencing mechanisms should prevent duplicates.
> >
> > > Great!
> >
> > > > I'm a bit concerned the proposed API will be delicate to program
> > against
> >
> > > > successfully -- even in the simple case, we need to create a new
> > producer
> >
> > > > instance per input partition, and anything fancier is going to need
> its
> >
> > > own
> >
> > > > implementation of the Streams/Samza-style 'task' idea -- but that may
> > be
> >
> > > > fine for this sort of advanced feature.
> >
> > > >
> >
> > > > For the second question, I notice that Jason also elaborated on this
> >
> > > > downthread:
> >
> > > >
> >
> > > > > We also looked at removing the producer ID.
> >
> > > > > This was discussed somewhere above, but basically the idea is to
> > store
> >
> > > > the
> >
> > > > > AppID in the message set header directly and avoid the mapping to
> >
> > > > producer
> >
> > > > > ID altogether. As long as batching isn't too bad, the impact on
> total
> >
> > > > size
> >
> > > > > may not be too bad, but we were ultimately more comfortable with a
> >
> > > fixed
> >
> > > > > size ID.
> >
> > > >
> >
> > > > ...which suggests that the distinction is useful for performance, but
> > not
> >
> > > > necessary for correctness, which makes good sense to me. (Would a
> > 128-bid
> >
> > > > ID be a reasonable compromise? That's enough room for a UUID, or a
> >
> > > > reasonable hash of an arbitrary string, and has only a marginal
> > increase
> >
> > > on
> >
> > > > the message size.)
> >
> > > >
> >
> > > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
> >
> > > wrote:
> >
> > > >
> >
> > > > > Hi Ben,
> >
> > > > >
> >
> > > > > Now, on to your first question of how deal with consumer
> rebalances.
> >
> > > The
> >
> > > > > short answer is that the application needs to ensure that the the
> >
> > > > > assignment of input partitions to appId is consistent across
> >
> > > rebalances.
> >
> > > > >
> >
> > > > > For Kafka streams, they already ensure that the mapping of input
> >
> > > > partitions
> >
> > > > > to task Id is invariant across rebalances by implementing a custom
> >
> > > sticky
> >
> > > > > assignor. Other non-streams apps can trivially have one producer
> per
> >
> > > > input
> >
> > > > > partition and have the appId be the same as the partition number to
> >
> > > > achieve
> >
> > > > > the same effect.
> >
> > > > >
> >
> > > > > With this precondition in place, we can maintain transactions
> across
> >
> > > > > rebalances.
> >
> > > > >
> >
> > > > > Hope this answers your question.
> >
> > > > >
> >
> > > > > Thanks,
> >
> > > > > Apurva
> >
> > > > >
> >
> > > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> >
> > > > >
> >
> > > > > > Thanks for this! I'm looking forward to going through the full
> >
> > > proposal
> >
> > > > > in
> >
> > > > > > detail soon; a few early questions:
> >
> > > > > >
> >
> > > > > > First: what happens when a consumer rebalances in the middle of a
> >
> > > > > > transaction? The full documentation suggests that such a
> > transaction
> >
> > > > > ought
> >
> > > > > > to be rejected:
> >
> > > > > >
> >
> > > > > > > [...] if a rebalance has happened and this consumer
> >
> > > > > > > instance becomes a zombie, even if this offset message is
> > appended
> >
> > > in
> >
> > > > > the
> >
> > > > > > > offset topic, the transaction will be rejected later on when it
> >
> > > tries
> >
> > > > > to
> >
> > > > > > > commit the transaction via the EndTxnRequest.
> >
> > > > > >
> >
> > > > > > ...but it's unclear to me how we ensure that a transaction can't
> >
> > > > complete
> >
> > > > > > if a rebalance has happened. (It's quite possible I'm missing
> >
> > > something
> >
> > > > > > obvious!)
> >
> > > > > >
> >
> > > > > > As a concrete example: suppose a process with PID 1 adds offsets
> > for
> >
> > > > some
> >
> > > > > > partition to a transaction; a consumer rebalance happens that
> > assigns
> >
> > > > the
> >
> > > > > > partition to a process with PID 2, which adds some offsets to its
> >
> > > > current
> >
> > > > > > transaction; both processes try and commit. Allowing both commits
> >
> > > would
> >
> > > > > > cause the messages to be processed twice -- how is that avoided?
> >
> > > > > >
> >
> > > > > > Second: App IDs normally map to a single PID. It seems like one
> > could
> >
> > > > do
> >
> > > > > > away with the PID concept entirely, and just use App IDs in most
> >
> > > places
> >
> > > > > > that require a PID. This feels like it would be significantly
> >
> > > simpler,
> >
> > > > > > though it does increase the message size. Are there other reasons
> > why
> >
> > > > the
> >
> > > > > > App ID / PID split is necessary?
> >
> > > > > >
> >
> > > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> >
> > > > > wrote:
> >
> > > > > >
> >
> > > > > > > Hi all,
> >
> > > > > > >
> >
> > > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> >
> > > > delivery
> >
> > > > > > > semantics:
> >
> > > > > > >
> >
> > > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> >
> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >
> > > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > > > > > >
> >
> > > > > > > This KIP adds a transactional messaging mechanism along with an
> >
> > > > > > idempotent
> >
> > > > > > > producer implementation to make sure that 1) duplicated
> messages
> >
> > > sent
> >
> > > > > > from
> >
> > > > > > > the same identified producer can be detected on the broker
> side,
> >
> > > and
> >
> > > > > 2) a
> >
> > > > > > > group of messages sent within a transaction will atomically be
> >
> > > either
> >
> > > > > > > reflected and fetchable to consumers or not as a whole.
> >
> > > > > > >
> >
> > > > > > > The above wiki page provides a high-level view of the proposed
> >
> > > > changes
> >
> > > > > as
> >
> > > > > > > well as summarized guarantees. Initial draft of the detailed
> >
> > > > > > implementation
> >
> > > > > > > design is described in this Google doc:
> >
> > > > > > >
> >
> > > > > > > https://docs.google.com/document/d/11Jqy_
> >
> > > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> >
> > > > > > > 0wSw9ra8
> >
> > > > > > >
> >
> > > > > > >
> >
> > > > > > > We would love to hear your comments and suggestions.
> >
> > > > > > >
> >
> > > > > > > Thanks,
> >
> > > > > > >
> >
> > > > > > > -- Guozhang
> >
> > > > > > >
> >
> > > > > >
> >
> > > > >
> >
> > > >
> >
> > >
> >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Yeah good point. I relent!

-jay

On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson <ja...@confluent.io> wrote:

> Jay/Ismael,
>
>
>
> I agree that lazy initialization of metadata seems unavoidable. Ideally, we
>
> could follow the same pattern for transactions, but remember that in the
>
> consumer+producer use case, the initialization needs to be completed prior
>
> to setting the consumer's position. Otherwise we risk reading stale
>
> offsets. But it would be pretty awkward if you have to begin a transaction
>
> first to ensure that your consumer can read the right offset from the
>
> consumer, right? It's a bit easier to explain that you should always call
>
> `producer.init()` prior to initializing the consumer. Users would probably
>
> get this right without any special effort.
>
>
>
> -Jason
>
>
>
> On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io>
> wrote:
>
>
>
> > Hi Apurva,
>
> >
>
> > Thank you for the answers. Just one follow-on.
>
> >
>
> > 15. Let me rephrase my original question. If all control messages
> (messages
>
> > to transaction logs and markers on user logs) were acknowledged only
> after
>
> > flushing the log segment, will transactions become durable in the
>
> > traditional sense (i.e. not restricted to min.insync.replicas failures) ?
>
> > This is not a suggestion to update the KIP. It seems to me that the
> design
>
> > enables full durability if required in the future with a rather
>
> > non-intrusive change. I just wanted to make sure I haven't missed
> anything
>
> > fundamental that prevents Kafka from doing this.
>
> >
>
> >
>
> >
>
> > On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
>
> >
>
> > > Hi Apurva,
>
> > >
>
> > > Thanks for the detailed answers... and sorry for the late reply!
>
> > >
>
> > > It does sound like, if the input-partitions-to-app-id mapping never
>
> > > changes, the existing fencing mechanisms should prevent duplicates.
>
> > Great!
>
> > > I'm a bit concerned the proposed API will be delicate to program
> against
>
> > > successfully -- even in the simple case, we need to create a new
> producer
>
> > > instance per input partition, and anything fancier is going to need its
>
> > own
>
> > > implementation of the Streams/Samza-style 'task' idea -- but that may
> be
>
> > > fine for this sort of advanced feature.
>
> > >
>
> > > For the second question, I notice that Jason also elaborated on this
>
> > > downthread:
>
> > >
>
> > > > We also looked at removing the producer ID.
>
> > > > This was discussed somewhere above, but basically the idea is to
> store
>
> > > the
>
> > > > AppID in the message set header directly and avoid the mapping to
>
> > > producer
>
> > > > ID altogether. As long as batching isn't too bad, the impact on total
>
> > > size
>
> > > > may not be too bad, but we were ultimately more comfortable with a
>
> > fixed
>
> > > > size ID.
>
> > >
>
> > > ...which suggests that the distinction is useful for performance, but
> not
>
> > > necessary for correctness, which makes good sense to me. (Would a
> 128-bid
>
> > > ID be a reasonable compromise? That's enough room for a UUID, or a
>
> > > reasonable hash of an arbitrary string, and has only a marginal
> increase
>
> > on
>
> > > the message size.)
>
> > >
>
> > > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
>
> > wrote:
>
> > >
>
> > > > Hi Ben,
>
> > > >
>
> > > > Now, on to your first question of how deal with consumer rebalances.
>
> > The
>
> > > > short answer is that the application needs to ensure that the the
>
> > > > assignment of input partitions to appId is consistent across
>
> > rebalances.
>
> > > >
>
> > > > For Kafka streams, they already ensure that the mapping of input
>
> > > partitions
>
> > > > to task Id is invariant across rebalances by implementing a custom
>
> > sticky
>
> > > > assignor. Other non-streams apps can trivially have one producer per
>
> > > input
>
> > > > partition and have the appId be the same as the partition number to
>
> > > achieve
>
> > > > the same effect.
>
> > > >
>
> > > > With this precondition in place, we can maintain transactions across
>
> > > > rebalances.
>
> > > >
>
> > > > Hope this answers your question.
>
> > > >
>
> > > > Thanks,
>
> > > > Apurva
>
> > > >
>
> > > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
>
> > > >
>
> > > > > Thanks for this! I'm looking forward to going through the full
>
> > proposal
>
> > > > in
>
> > > > > detail soon; a few early questions:
>
> > > > >
>
> > > > > First: what happens when a consumer rebalances in the middle of a
>
> > > > > transaction? The full documentation suggests that such a
> transaction
>
> > > > ought
>
> > > > > to be rejected:
>
> > > > >
>
> > > > > > [...] if a rebalance has happened and this consumer
>
> > > > > > instance becomes a zombie, even if this offset message is
> appended
>
> > in
>
> > > > the
>
> > > > > > offset topic, the transaction will be rejected later on when it
>
> > tries
>
> > > > to
>
> > > > > > commit the transaction via the EndTxnRequest.
>
> > > > >
>
> > > > > ...but it's unclear to me how we ensure that a transaction can't
>
> > > complete
>
> > > > > if a rebalance has happened. (It's quite possible I'm missing
>
> > something
>
> > > > > obvious!)
>
> > > > >
>
> > > > > As a concrete example: suppose a process with PID 1 adds offsets
> for
>
> > > some
>
> > > > > partition to a transaction; a consumer rebalance happens that
> assigns
>
> > > the
>
> > > > > partition to a process with PID 2, which adds some offsets to its
>
> > > current
>
> > > > > transaction; both processes try and commit. Allowing both commits
>
> > would
>
> > > > > cause the messages to be processed twice -- how is that avoided?
>
> > > > >
>
> > > > > Second: App IDs normally map to a single PID. It seems like one
> could
>
> > > do
>
> > > > > away with the PID concept entirely, and just use App IDs in most
>
> > places
>
> > > > > that require a PID. This feels like it would be significantly
>
> > simpler,
>
> > > > > though it does increase the message size. Are there other reasons
> why
>
> > > the
>
> > > > > App ID / PID split is necessary?
>
> > > > >
>
> > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wangguoz@gmail.com
> >
>
> > > > wrote:
>
> > > > >
>
> > > > > > Hi all,
>
> > > > > >
>
> > > > > > I have just created KIP-98 to enhance Kafka with exactly once
>
> > > delivery
>
> > > > > > semantics:
>
> > > > > >
>
> > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>
> > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>
> > > > > >
>
> > > > > > This KIP adds a transactional messaging mechanism along with an
>
> > > > > idempotent
>
> > > > > > producer implementation to make sure that 1) duplicated messages
>
> > sent
>
> > > > > from
>
> > > > > > the same identified producer can be detected on the broker side,
>
> > and
>
> > > > 2) a
>
> > > > > > group of messages sent within a transaction will atomically be
>
> > either
>
> > > > > > reflected and fetchable to consumers or not as a whole.
>
> > > > > >
>
> > > > > > The above wiki page provides a high-level view of the proposed
>
> > > changes
>
> > > > as
>
> > > > > > well as summarized guarantees. Initial draft of the detailed
>
> > > > > implementation
>
> > > > > > design is described in this Google doc:
>
> > > > > >
>
> > > > > > https://docs.google.com/document/d/11Jqy_
>
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>
> > > > > > 0wSw9ra8
>
> > > > > >
>
> > > > > >
>
> > > > > > We would love to hear your comments and suggestions.
>
> > > > > >
>
> > > > > > Thanks,
>
> > > > > >
>
> > > > > > -- Guozhang
>
> > > > > >
>
> > > > >
>
> > > >
>
> > >
>
> >
>
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Jay/Ismael,

I agree that lazy initialization of metadata seems unavoidable. Ideally, we
could follow the same pattern for transactions, but remember that in the
consumer+producer use case, the initialization needs to be completed prior
to setting the consumer's position. Otherwise we risk reading stale
offsets. But it would be pretty awkward if you have to begin a transaction
first to ensure that your consumer can read the right offset from the
consumer, right? It's a bit easier to explain that you should always call
`producer.init()` prior to initializing the consumer. Users would probably
get this right without any special effort.

-Jason

On Wed, Dec 14, 2016 at 1:52 AM, Rajini Sivaram <rs...@pivotal.io> wrote:

> Hi Apurva,
>
> Thank you for the answers. Just one follow-on.
>
> 15. Let me rephrase my original question. If all control messages (messages
> to transaction logs and markers on user logs) were acknowledged only after
> flushing the log segment, will transactions become durable in the
> traditional sense (i.e. not restricted to min.insync.replicas failures) ?
> This is not a suggestion to update the KIP. It seems to me that the design
> enables full durability if required in the future with a rather
> non-intrusive change. I just wanted to make sure I haven't missed anything
> fundamental that prevents Kafka from doing this.
>
>
>
> On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:
>
> > Hi Apurva,
> >
> > Thanks for the detailed answers... and sorry for the late reply!
> >
> > It does sound like, if the input-partitions-to-app-id mapping never
> > changes, the existing fencing mechanisms should prevent duplicates.
> Great!
> > I'm a bit concerned the proposed API will be delicate to program against
> > successfully -- even in the simple case, we need to create a new producer
> > instance per input partition, and anything fancier is going to need its
> own
> > implementation of the Streams/Samza-style 'task' idea -- but that may be
> > fine for this sort of advanced feature.
> >
> > For the second question, I notice that Jason also elaborated on this
> > downthread:
> >
> > > We also looked at removing the producer ID.
> > > This was discussed somewhere above, but basically the idea is to store
> > the
> > > AppID in the message set header directly and avoid the mapping to
> > producer
> > > ID altogether. As long as batching isn't too bad, the impact on total
> > size
> > > may not be too bad, but we were ultimately more comfortable with a
> fixed
> > > size ID.
> >
> > ...which suggests that the distinction is useful for performance, but not
> > necessary for correctness, which makes good sense to me. (Would a 128-bid
> > ID be a reasonable compromise? That's enough room for a UUID, or a
> > reasonable hash of an arbitrary string, and has only a marginal increase
> on
> > the message size.)
> >
> > On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> > > Hi Ben,
> > >
> > > Now, on to your first question of how deal with consumer rebalances.
> The
> > > short answer is that the application needs to ensure that the the
> > > assignment of input partitions to appId is consistent across
> rebalances.
> > >
> > > For Kafka streams, they already ensure that the mapping of input
> > partitions
> > > to task Id is invariant across rebalances by implementing a custom
> sticky
> > > assignor. Other non-streams apps can trivially have one producer per
> > input
> > > partition and have the appId be the same as the partition number to
> > achieve
> > > the same effect.
> > >
> > > With this precondition in place, we can maintain transactions across
> > > rebalances.
> > >
> > > Hope this answers your question.
> > >
> > > Thanks,
> > > Apurva
> > >
> > > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> > >
> > > > Thanks for this! I'm looking forward to going through the full
> proposal
> > > in
> > > > detail soon; a few early questions:
> > > >
> > > > First: what happens when a consumer rebalances in the middle of a
> > > > transaction? The full documentation suggests that such a transaction
> > > ought
> > > > to be rejected:
> > > >
> > > > > [...] if a rebalance has happened and this consumer
> > > > > instance becomes a zombie, even if this offset message is appended
> in
> > > the
> > > > > offset topic, the transaction will be rejected later on when it
> tries
> > > to
> > > > > commit the transaction via the EndTxnRequest.
> > > >
> > > > ...but it's unclear to me how we ensure that a transaction can't
> > complete
> > > > if a rebalance has happened. (It's quite possible I'm missing
> something
> > > > obvious!)
> > > >
> > > > As a concrete example: suppose a process with PID 1 adds offsets for
> > some
> > > > partition to a transaction; a consumer rebalance happens that assigns
> > the
> > > > partition to a process with PID 2, which adds some offsets to its
> > current
> > > > transaction; both processes try and commit. Allowing both commits
> would
> > > > cause the messages to be processed twice -- how is that avoided?
> > > >
> > > > Second: App IDs normally map to a single PID. It seems like one could
> > do
> > > > away with the PID concept entirely, and just use App IDs in most
> places
> > > > that require a PID. This feels like it would be significantly
> simpler,
> > > > though it does increase the message size. Are there other reasons why
> > the
> > > > App ID / PID split is necessary?
> > > >
> > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > delivery
> > > > > semantics:
> > > > >
> > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > >
> > > > > This KIP adds a transactional messaging mechanism along with an
> > > > idempotent
> > > > > producer implementation to make sure that 1) duplicated messages
> sent
> > > > from
> > > > > the same identified producer can be detected on the broker side,
> and
> > > 2) a
> > > > > group of messages sent within a transaction will atomically be
> either
> > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > The above wiki page provides a high-level view of the proposed
> > changes
> > > as
> > > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > > design is described in this Google doc:
> > > > >
> > > > > https://docs.google.com/document/d/11Jqy_
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8
> > > > >
> > > > >
> > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Rajini Sivaram <rs...@pivotal.io>.
Hi Apurva,

Thank you for the answers. Just one follow-on.

15. Let me rephrase my original question. If all control messages (messages
to transaction logs and markers on user logs) were acknowledged only after
flushing the log segment, will transactions become durable in the
traditional sense (i.e. not restricted to min.insync.replicas failures) ?
This is not a suggestion to update the KIP. It seems to me that the design
enables full durability if required in the future with a rather
non-intrusive change. I just wanted to make sure I haven't missed anything
fundamental that prevents Kafka from doing this.



On Wed, Dec 14, 2016 at 5:30 AM, Ben Kirwin <be...@kirw.in> wrote:

> Hi Apurva,
>
> Thanks for the detailed answers... and sorry for the late reply!
>
> It does sound like, if the input-partitions-to-app-id mapping never
> changes, the existing fencing mechanisms should prevent duplicates. Great!
> I'm a bit concerned the proposed API will be delicate to program against
> successfully -- even in the simple case, we need to create a new producer
> instance per input partition, and anything fancier is going to need its own
> implementation of the Streams/Samza-style 'task' idea -- but that may be
> fine for this sort of advanced feature.
>
> For the second question, I notice that Jason also elaborated on this
> downthread:
>
> > We also looked at removing the producer ID.
> > This was discussed somewhere above, but basically the idea is to store
> the
> > AppID in the message set header directly and avoid the mapping to
> producer
> > ID altogether. As long as batching isn't too bad, the impact on total
> size
> > may not be too bad, but we were ultimately more comfortable with a fixed
> > size ID.
>
> ...which suggests that the distinction is useful for performance, but not
> necessary for correctness, which makes good sense to me. (Would a 128-bid
> ID be a reasonable compromise? That's enough room for a UUID, or a
> reasonable hash of an arbitrary string, and has only a marginal increase on
> the message size.)
>
> On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io> wrote:
>
> > Hi Ben,
> >
> > Now, on to your first question of how deal with consumer rebalances. The
> > short answer is that the application needs to ensure that the the
> > assignment of input partitions to appId is consistent across rebalances.
> >
> > For Kafka streams, they already ensure that the mapping of input
> partitions
> > to task Id is invariant across rebalances by implementing a custom sticky
> > assignor. Other non-streams apps can trivially have one producer per
> input
> > partition and have the appId be the same as the partition number to
> achieve
> > the same effect.
> >
> > With this precondition in place, we can maintain transactions across
> > rebalances.
> >
> > Hope this answers your question.
> >
> > Thanks,
> > Apurva
> >
> > On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
> >
> > > Thanks for this! I'm looking forward to going through the full proposal
> > in
> > > detail soon; a few early questions:
> > >
> > > First: what happens when a consumer rebalances in the middle of a
> > > transaction? The full documentation suggests that such a transaction
> > ought
> > > to be rejected:
> > >
> > > > [...] if a rebalance has happened and this consumer
> > > > instance becomes a zombie, even if this offset message is appended in
> > the
> > > > offset topic, the transaction will be rejected later on when it tries
> > to
> > > > commit the transaction via the EndTxnRequest.
> > >
> > > ...but it's unclear to me how we ensure that a transaction can't
> complete
> > > if a rebalance has happened. (It's quite possible I'm missing something
> > > obvious!)
> > >
> > > As a concrete example: suppose a process with PID 1 adds offsets for
> some
> > > partition to a transaction; a consumer rebalance happens that assigns
> the
> > > partition to a process with PID 2, which adds some offsets to its
> current
> > > transaction; both processes try and commit. Allowing both commits would
> > > cause the messages to be processed twice -- how is that avoided?
> > >
> > > Second: App IDs normally map to a single PID. It seems like one could
> do
> > > away with the PID concept entirely, and just use App IDs in most places
> > > that require a PID. This feels like it would be significantly simpler,
> > > though it does increase the message size. Are there other reasons why
> the
> > > App ID / PID split is necessary?
> > >
> > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ben Kirwin <be...@kirw.in>.
Hi Apurva,

Thanks for the detailed answers... and sorry for the late reply!

It does sound like, if the input-partitions-to-app-id mapping never
changes, the existing fencing mechanisms should prevent duplicates. Great!
I'm a bit concerned the proposed API will be delicate to program against
successfully -- even in the simple case, we need to create a new producer
instance per input partition, and anything fancier is going to need its own
implementation of the Streams/Samza-style 'task' idea -- but that may be
fine for this sort of advanced feature.

For the second question, I notice that Jason also elaborated on this
downthread:

> We also looked at removing the producer ID.
> This was discussed somewhere above, but basically the idea is to store the
> AppID in the message set header directly and avoid the mapping to producer
> ID altogether. As long as batching isn't too bad, the impact on total size
> may not be too bad, but we were ultimately more comfortable with a fixed
> size ID.

...which suggests that the distinction is useful for performance, but not
necessary for correctness, which makes good sense to me. (Would a 128-bid
ID be a reasonable compromise? That's enough room for a UUID, or a
reasonable hash of an arbitrary string, and has only a marginal increase on
the message size.)

On Tue, Dec 6, 2016 at 11:44 PM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Ben,
>
> Now, on to your first question of how deal with consumer rebalances. The
> short answer is that the application needs to ensure that the the
> assignment of input partitions to appId is consistent across rebalances.
>
> For Kafka streams, they already ensure that the mapping of input partitions
> to task Id is invariant across rebalances by implementing a custom sticky
> assignor. Other non-streams apps can trivially have one producer per input
> partition and have the appId be the same as the partition number to achieve
> the same effect.
>
> With this precondition in place, we can maintain transactions across
> rebalances.
>
> Hope this answers your question.
>
> Thanks,
> Apurva
>
> On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:
>
> > Thanks for this! I'm looking forward to going through the full proposal
> in
> > detail soon; a few early questions:
> >
> > First: what happens when a consumer rebalances in the middle of a
> > transaction? The full documentation suggests that such a transaction
> ought
> > to be rejected:
> >
> > > [...] if a rebalance has happened and this consumer
> > > instance becomes a zombie, even if this offset message is appended in
> the
> > > offset topic, the transaction will be rejected later on when it tries
> to
> > > commit the transaction via the EndTxnRequest.
> >
> > ...but it's unclear to me how we ensure that a transaction can't complete
> > if a rebalance has happened. (It's quite possible I'm missing something
> > obvious!)
> >
> > As a concrete example: suppose a process with PID 1 adds offsets for some
> > partition to a transaction; a consumer rebalance happens that assigns the
> > partition to a process with PID 2, which adds some offsets to its current
> > transaction; both processes try and commit. Allowing both commits would
> > cause the messages to be processed twice -- how is that avoided?
> >
> > Second: App IDs normally map to a single PID. It seems like one could do
> > away with the PID concept entirely, and just use App IDs in most places
> > that require a PID. This feels like it would be significantly simpler,
> > though it does increase the message size. Are there other reasons why the
> > App ID / PID split is necessary?
> >
> > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Ben,

Now, on to your first question of how deal with consumer rebalances. The
short answer is that the application needs to ensure that the the
assignment of input partitions to appId is consistent across rebalances.

For Kafka streams, they already ensure that the mapping of input partitions
to task Id is invariant across rebalances by implementing a custom sticky
assignor. Other non-streams apps can trivially have one producer per input
partition and have the appId be the same as the partition number to achieve
the same effect.

With this precondition in place, we can maintain transactions across
rebalances.

Hope this answers your question.

Thanks,
Apurva

On Tue, Dec 6, 2016 at 3:22 PM, Ben Kirwin <be...@kirw.in> wrote:

> Thanks for this! I'm looking forward to going through the full proposal in
> detail soon; a few early questions:
>
> First: what happens when a consumer rebalances in the middle of a
> transaction? The full documentation suggests that such a transaction ought
> to be rejected:
>
> > [...] if a rebalance has happened and this consumer
> > instance becomes a zombie, even if this offset message is appended in the
> > offset topic, the transaction will be rejected later on when it tries to
> > commit the transaction via the EndTxnRequest.
>
> ...but it's unclear to me how we ensure that a transaction can't complete
> if a rebalance has happened. (It's quite possible I'm missing something
> obvious!)
>
> As a concrete example: suppose a process with PID 1 adds offsets for some
> partition to a transaction; a consumer rebalance happens that assigns the
> partition to a process with PID 2, which adds some offsets to its current
> transaction; both processes try and commit. Allowing both commits would
> cause the messages to be processed twice -- how is that avoided?
>
> Second: App IDs normally map to a single PID. It seems like one could do
> away with the PID concept entirely, and just use App IDs in most places
> that require a PID. This feels like it would be significantly simpler,
> though it does increase the message size. Are there other reasons why the
> App ID / PID split is necessary?
>
> On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ben Kirwin <be...@kirw.in>.
Thanks for this! I'm looking forward to going through the full proposal in
detail soon; a few early questions:

First: what happens when a consumer rebalances in the middle of a
transaction? The full documentation suggests that such a transaction ought
to be rejected:

> [...] if a rebalance has happened and this consumer
> instance becomes a zombie, even if this offset message is appended in the
> offset topic, the transaction will be rejected later on when it tries to
> commit the transaction via the EndTxnRequest.

...but it's unclear to me how we ensure that a transaction can't complete
if a rebalance has happened. (It's quite possible I'm missing something
obvious!)

As a concrete example: suppose a process with PID 1 adds offsets for some
partition to a transaction; a consumer rebalance happens that assigns the
partition to a process with PID 2, which adds some offsets to its current
transaction; both processes try and commit. Allowing both commits would
cause the messages to be processed twice -- how is that avoided?

Second: App IDs normally map to a single PID. It seems like one could do
away with the PID concept entirely, and just use App IDs in most places
that require a PID. This feels like it would be significantly simpler,
though it does increase the message size. Are there other reasons why the
App ID / PID split is necessary?

On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Jay,

Thanks for your comments. Answers to some of your points are below:

2. There have been long debates about the necessity of the initTransactions
method. Let's consider the options for doing without the initTransactions
method:

   - If we do it on demand, we have to consider that the InitPIDRequest
   sent by initTransactions actually does transaction recovery. It would be
   slightly counterintuitive for the previous transactions to be recovered
   _only_ when new data is produced.
   - We did think about doing the initialization in the constructor, but
   then this would introduce potentially long blocking calls to the
   constructor in the case of transaction recovery. In particular, the
   coordinator would have to write the commit markers to all the partitions
   that are part of the transactions during recovery.
   - We did think about doing it in the 'beginTransaction' method, but then
   this suffers the same critique as doing it on demand: transaction recovery
   only happens when the next transaction is started.

These were the considerations that motivated the initTransactions method.
Personally, I am not opposed to doing transaction recovery in the
constructor and would be interested in others' opinions on this. Of course,
it would be even better if there were a third or fourth option!

* *

3. That nomenclature makes sense. We can adopt it unless others have
objections.

* *

4. I think the PID allocation with only 4 byte PIDs may become tricky. In
particular, the current algorithm for PID allocation is to have each
coordinator carve out some part of the PID space and then allocate from its
pool. The available PIDs would be stored in zookeeper and parceled out to
coordinators on demand. With 64bit PIDs we can be sure that we would have
PIDs available till eternity.

With a 32 bit PID, we are less certain because our PID expiration algorithm
doesn't 'restore' expired PIDs. If we did restore PIDs, then I think 32bit
PIDs would be sufficient, since the space would only have to be big enough
to handle all concurrent producers. The latter is worth thinking about
given the storage savings. We will explore restoring PIDs on expiration,
and reducing to 32bit PIDs.

* *

Thanks,
Apurva

On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:

> Looks great!
>
> A few questions:
>
>    1. What is the relationship between transaction.app.id and the existing
>    config application.id in streams?
>    2. The initTransactions() call is a little annoying. Can we get rid of
>    that and call it automatically if you set a transaction.app.id when we
>    do the first message send as we do with metadata? Arguably we should
> have
>    included a general connect() or init() call in the producer, but given
> that
>    we didn't do this it seems weird that the cluster metadata initializes
>    automatically on demand and the transaction metadata doesn't.
>    3. The equivalent concept of what we call "fetch.mode" in databases is
>    called "isolation level" and takes values like "serializable", "read
>    committed", "read uncommitted". Since we went with transaction as the
> name
>    for the thing in between the begin/commit might make sense to use this
>    terminology for the concept and levels? I think the behavior we are
>    planning is "read committed" and the alternative re-ordering behavior is
>    equivalent to "serializable"?
>    4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
>    billion concurrent producers should be enough for anyone, right?
>    5. One implication of factoring out the message set seems to be you
>    can't ever "repack" messages to improve compression beyond what is done
> by
>    the producer. We'd talked about doing this either by buffering when
> writing
>    or during log cleaning. This isn't a show stopper but I think one
>    implication is that we won't be able to do this. Furthermore with log
>    cleaning you'd assume that over time ALL messages would collapse down
> to a
>    single wrapper as compaction removes the others.
>
> -Jay
>
> On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Jay,

The expiration of PIDs is tied to the retention of each partition. Once a
PID "falls off the log," then it is considered expired for that partition
and we take it out of the sequence table. This is intuitive because we
always consider the log as the source of truth for the PID sequence
mapping. If there is no data remaining in a given partition for some PID,
then we've lost its sequence as well. The problem then is that it's
difficult to tell when it is safe to reuse a PID without some mechanism for
global coordination. We need to be sure that the PID has expired on all
partitions. This could be done through Zk perhaps, but we didn't really
want to have any per-producer state stored there. I honestly don't have a
good sense whether 4 bytes is enough given this expiration scheme. My
feeling is that it's probably fine for most use cases: it seems unlikely
you'd have a single producer which lasts as long as 4 billion other
producers (assuming an unsigned PID). But it doesn't give me the order of
magnitude level of comfort that I'd hope for.

We've also discussed some other options. One thought was to let the
ProduceRequest include a flag to indicate when a PID is used for the first
time. Then we might be able to have the broker force the expiration of the
old PID (if there is one). That would be safer than just using the PID and
hoping for the best. We've also discussed options for global expiration of
a PID, but it tends to get a little hairy.

Thanks,
Jason

On Tue, Dec 6, 2016 at 11:01 AM, Sriram Subramanian <ra...@confluent.io>
wrote:

> @Jay
>
> 1. I totally agree on the naming. The appid for transactions is really an
> instance id. Any recommendation for a name is appreciated. We had thought
> of instance id, session id or app id and went with app id.
> 2. We also discussed about init() method but that could add its own set of
> confusion to existing users (should I update my existing usage to call
> init()? Why should I have this extra step instead of the constructor doing
> it?). Transactions is going to be used by a subset of users (probably
> small) and it made sense to add the burden of calling
> initTransactions/recoverTransactions to only that subset. We are actually
> open to suggestions here in terms of naming as well.
>
> @Jonathan
> I am not sure it adds more complexity unless you use them. We have
> explicitly named them for transactions and the current usage of the
> producer remains unchanged.
>
> @Michael
> If you look at our idempotent producer implementation in the kip/design,
> this is exactly what we do except that the deduplication happens on the
> server. We started with separate KIPs but it made it very confusing to read
> since there were interdependencies between the concepts.
>
>
>
> On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <Mi...@ig.com>
> wrote:
>
> > For dealing with exactly once delivery.
> >
> > As an alternative option has it been considered to have a message uuid in
> > the record, that then is deduped on consumption?
> >
> > Similar to
> > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-detection.html
> >
> > Agreed this does not deal with transaction support.
> >
> > But should the two concerns be separated anyhow into two kips?
> >
> > Cheers
> > Mike
> >
> >
> > Sent using OWA for iPhone
> > ________________________________________
> > From: Jay Kreps <ja...@confluent.io>
> > Sent: Tuesday, December 6, 2016 4:47:55 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> > Hey Guozhang,
> >
> >
> >    1. My point is that it is a bit confusing to have two things called
> >    application id that have different meanings, right? Won't the streams
> > user
> >    end up specifying two different application ids?
> >    2. Makes sense. My two complaints are
> >       1. At this point we've jumped through quite a lot of hoops to make
> >       the producer lazily initialize, seems sad to get rid of that now.
> >       2. The call initTransactions doesn't really make sense to the user
> >       unless they understand the details of the protocol (which they
> > won't). When
> >       do i call this? How many times? etc. Maybe two additional
> > options would be
> >       to just add a general init() call that could cover metadata
> > initialization
> >       as well as this and potentially future things or continue to do it
> > lazily.
> >    3. Yeah I get that you need an expiry scheme to limit it to 4 bytes.
> Is
> >    there a mechanism to expire them, and hence shrink it?
> >
> > -Jay
> >
> > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > @Jay
> > >
> > > 1. Stream's applicationId is shared among all instances for the app,
> and
> > is
> > > used as part of the consumer group id, while "app.id" is per producer
> > > instance. So a Streams app that has a single "applicationID" config
> will
> > > likely contain multiple producers each with a different appID based on
> > > their corresponding taskIDs.
> > >
> > > 2. Another motivation besides the one pointed out by Jason for making
> > sure
> > > transaction-involved offsets have been committed before resuming, is
> that
> > > we also want to separate the "app.id" config with the transactional
> > > mechanism. More concretely, if a user does specify the "app.id" config
> > and
> > > without using transaction functions (i.e. initTransactions, beginTxn,
> > etc),
> > > they can still get idempotency guarantee across multiple sessions of
> the
> > > producer identified by the app.id.
> > >
> > > 4. We thought about the PID length, note that since we do not expire
> > PIDs,
> > > we are expecting it to cover all possible producers that we have ever
> > seen,
> > > and producers without an "app.id" can come and go with different PIDs.
> > > That
> > > is why we feel 4 billion may not be sufficient.
> > >
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > Hey Jay,
> > > >
> > > > Thanks for the questions! Let me take a couple of them.
> > > >
> > > > 2. The initTransactions() call is a little annoying. Can we get rid
> of
> > > > >    that and call it automatically if you set a transaction.app.id
> > when
> > > > we
> > > > >    do the first message send as we do with metadata? Arguably we
> > should
> > > > > have
> > > > >    included a general connect() or init() call in the producer, but
> > > given
> > > > > that
> > > > >    we didn't do this it seems weird that the cluster metadata
> > > initializes
> > > > >    automatically on demand and the transaction metadata doesn't.
> > > >
> > > >
> > > > The purpose of this call is to fence off any producer with the same
> > AppID
> > > > and await the completion of any pending transactions. When it
> returns,
> > > you
> > > > know that your producer is safe to resume work. Take the the "consume
> > and
> > > > produce" use case as an example. We send the offset commits as part
> of
> > > the
> > > > producer's transaction (approximating the idea that it is "just
> another
> > > > write to a partition"). When you first initialize the application,
> you
> > > have
> > > > to determine when it's safe for the consumer to read those offsets.
> > > > Otherwise, you may read stale offsets before a transaction which is
> > > rolling
> > > > forward is able to write the marker to __consumer_offsets. So we
> can't
> > do
> > > > the initialization in send() because that would assume that we had
> > > already
> > > > read data from the consumer, which we can't do until we've
> initialized
> > > the
> > > > producer. Does that make sense?
> > > >
> > > > (For what it's worth, we're not married to this name or any of the
> > > others,
> > > > so anyone can feel free to suggest alternatives.)
> > > >
> > > >
> > > > 5. One implication of factoring out the message set seems to be you
> > > > >    can't ever "repack" messages to improve compression beyond what
> is
> > > > done
> > > > > by
> > > > >    the producer. We'd talked about doing this either by buffering
> > when
> > > > > writing
> > > > >    or during log cleaning. This isn't a show stopper but I think
> one
> > > > >    implication is that we won't be able to do this. Furthermore
> with
> > > log
> > > > >    cleaning you'd assume that over time ALL messages would collapse
> > > down
> > > > > to a
> > > > >    single wrapper as compaction removes the others.
> > > >
> > > >
> > > > Yeah, that's a fair point. You may still be able to do some merging
> if
> > > > adjacent message sets have the same PID, but the potential savings
> > might
> > > > not be worth the cost of implementation. My gut feeling is that
> merging
> > > > message sets from different producers may not be a great idea anyway
> > > (you'd
> > > > have to accept the fact that you always need "deep iteration" to find
> > the
> > > > PIDs contained within the message set), but I haven't thought a ton
> > about
> > > > it. Ultimately we'll have to decide if the potential for savings in
> the
> > > > future is worth some loss in efficiency now (for what it's worth, I
> > think
> > > > the work that Ben has been looking at also hopes to bundle some more
> > > > information into the message set header).
> > > >
> > > > On a purely pragmatic development level, after spending a ton of
> recent
> > > > time working with that code, I can say that the benefit of having a
> > > > conceptually simpler message format is huge. It allows you to
> converge
> > > the
> > > > paths for validation of message sets on the broker, for example.
> > > Currently,
> > > > we pretty much need two separate paths everywhere we process
> messages.
> > It
> > > > can be tricky just to tell if the message you're dealing with is the
> > > inner
> > > > or outer message, and whether it matters or not. Also, the fact that
> > the
> > > > inner and outer messages share common fields makes bugs like
> KAFKA-4298
> > > > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The
> risk
> > of
> > > > these bugs is much smaller when you can clearly separate the fields
> > > allowed
> > > > in the message set header and those in the messages.
> > > >
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > > > Looks great!
> > > > >
> > > > > A few questions:
> > > > >
> > > > >    1. What is the relationship between transaction.app.id and the
> > > > existing
> > > > >    config application.id in streams?
> > > > >    2. The initTransactions() call is a little annoying. Can we get
> > rid
> > > of
> > > > >    that and call it automatically if you set a transaction.app.id
> > when
> > > > we
> > > > >    do the first message send as we do with metadata? Arguably we
> > should
> > > > > have
> > > > >    included a general connect() or init() call in the producer, but
> > > given
> > > > > that
> > > > >    we didn't do this it seems weird that the cluster metadata
> > > initializes
> > > > >    automatically on demand and the transaction metadata doesn't.
> > > > >    3. The equivalent concept of what we call "fetch.mode" in
> > databases
> > > is
> > > > >    called "isolation level" and takes values like "serializable",
> > "read
> > > > >    committed", "read uncommitted". Since we went with transaction
> as
> > > the
> > > > > name
> > > > >    for the thing in between the begin/commit might make sense to
> use
> > > this
> > > > >    terminology for the concept and levels? I think the behavior we
> > are
> > > > >    planning is "read committed" and the alternative re-ordering
> > > behavior
> > > > is
> > > > >    equivalent to "serializable"?
> > > > >    4. Can the PID be made 4 bytes if we handle roll-over
> gracefully?
> > 2
> > > > >    billion concurrent producers should be enough for anyone, right?
> > > > >    5. One implication of factoring out the message set seems to be
> > you
> > > > >    can't ever "repack" messages to improve compression beyond what
> is
> > > > done
> > > > > by
> > > > >    the producer. We'd talked about doing this either by buffering
> > when
> > > > > writing
> > > > >    or during log cleaning. This isn't a show stopper but I think
> one
> > > > >    implication is that we won't be able to do this. Furthermore
> with
> > > log
> > > > >    cleaning you'd assume that over time ALL messages would collapse
> > > down
> > > > > to a
> > > > >    single wrapper as compaction removes the others.
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > delivery
> > > > > > semantics:
> > > > > >
> > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > > >
> > > > > > This KIP adds a transactional messaging mechanism along with an
> > > > > idempotent
> > > > > > producer implementation to make sure that 1) duplicated messages
> > sent
> > > > > from
> > > > > > the same identified producer can be detected on the broker side,
> > and
> > > > 2) a
> > > > > > group of messages sent within a transaction will atomically be
> > either
> > > > > > reflected and fetchable to consumers or not as a whole.
> > > > > >
> > > > > > The above wiki page provides a high-level view of the proposed
> > > changes
> > > > as
> > > > > > well as summarized guarantees. Initial draft of the detailed
> > > > > implementation
> > > > > > design is described in this Google doc:
> > > > > >
> > > > > > https://docs.google.com/document/d/11Jqy_
> > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > > 0wSw9ra8
> > > > > >
> > > > > >
> > > > > > We would love to hear your comments and suggestions.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Like wise I think handling transactions separately is important as we should consider supporting or at least a how in future session transactions, xa transactions etc

Sent using OWA for iPhone
________________________________________
From: Michael Pearce <Mi...@ig.com>
Sent: Thursday, December 8, 2016 6:12:09 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hi

For me delivering the once delivery story should not require transactions. Your last statement makes me feel you think they do.

Atm reading the kip I see only once delivery working end to end if I have also transactions enabled on the consumer also. I shouldn't have to do this.

Whilst having an idenpodent producer helps it doesn't solve the end to end problem. This only solves deliver into the broker. Not back out or any mirroring that may occur.

Some work is also needed on the consumer for message dupe detection. To achieve this without transactions.

This is why I really feel these should be tackled seperatly. So that there's a concentrated effort to ensure that we can deliver once only story without transactions end to end.

Like everything timestamps and now the further kip cases going in there is resuse and advancements. But they're still storied and tackle the problem independently


Cheers
Mike







Sent using OWA for iPhone
________________________________________
From: Jason Gustafson <ja...@confluent.io>
Sent: Thursday, December 8, 2016 5:46:43 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

There is no overhead from transactions if you don't use them, and they are
not needed to leverage the capabilities of the idempotent producer.
However, some of the underlying mechanics are shared, such as the message
format changes and the InitPidRequest. We see them as the two essential
ingredients needed to complete the exactly-once story, so we prefer to keep
them together.

-Jason

On Thu, Dec 8, 2016 at 12:36 AM, Michael Pearce <Mi...@ig.com>
wrote:

> My apologies, my computer is auto spellchecking and I didn’t notice before
> sending.
>
> *@Sriram
>
> Thanks
> Mike
>
> On 08/12/2016, 08:35, "Michael Pearce" <Mi...@ig.com> wrote:
>
>     @Shiram,
>     I would like to be able to have though exactly once delivery without
> the need to use more heavy weight transactions, this is why I’m proposing
> separate KIP these are not entirely orthogonal.
>
>     Agreed the uuid could be re-used in supporting transactionality, but I
> wouldn’t want to have to have full transactions and the additional
> performance costs. Note this on activemq/artemis implementation, that I can
> get/achieve the dedupe, but it is not tied into having a full transaction
> the solutions are de-coupled.
>
>
>     On 06/12/2016, 19:01, "Sriram Subramanian" <ra...@confluent.io> wrote:
>
>         @Jay
>
>         1. I totally agree on the naming. The appid for transactions is
> really an
>         instance id. Any recommendation for a name is appreciated. We had
> thought
>         of instance id, session id or app id and went with app id.
>         2. We also discussed about init() method but that could add its
> own set of
>         confusion to existing users (should I update my existing usage to
> call
>         init()? Why should I have this extra step instead of the
> constructor doing
>         it?). Transactions is going to be used by a subset of users
> (probably
>         small) and it made sense to add the burden of calling
>         initTransactions/recoverTransactions to only that subset. We are
> actually
>         open to suggestions here in terms of naming as well.
>
>         @Jonathan
>         I am not sure it adds more complexity unless you use them. We have
>         explicitly named them for transactions and the current usage of the
>         producer remains unchanged.
>
>         @Michael
>         If you look at our idempotent producer implementation in the
> kip/design,
>         this is exactly what we do except that the deduplication happens
> on the
>         server. We started with separate KIPs but it made it very
> confusing to read
>         since there were interdependencies between the concepts.
>
>
>
>         On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <
> Michael.Pearce@ig.com>
>         wrote:
>
>         > For dealing with exactly once delivery.
>         >
>         > As an alternative option has it been considered to have a
> message uuid in
>         > the record, that then is deduped on consumption?
>         >
>         > Similar to
>         > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-
> detection.html
>         >
>         > Agreed this does not deal with transaction support.
>         >
>         > But should the two concerns be separated anyhow into two kips?
>         >
>         > Cheers
>         > Mike
>         >
>         >
>         > Sent using OWA for iPhone
>         > ________________________________________
>         > From: Jay Kreps <ja...@confluent.io>
>         > Sent: Tuesday, December 6, 2016 4:47:55 PM
>         > To: dev@kafka.apache.org
>         > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
>         > Messaging
>         >
>         > Hey Guozhang,
>         >
>         >
>         >    1. My point is that it is a bit confusing to have two things
> called
>         >    application id that have different meanings, right? Won't the
> streams
>         > user
>         >    end up specifying two different application ids?
>         >    2. Makes sense. My two complaints are
>         >       1. At this point we've jumped through quite a lot of hoops
> to make
>         >       the producer lazily initialize, seems sad to get rid of
> that now.
>         >       2. The call initTransactions doesn't really make sense to
> the user
>         >       unless they understand the details of the protocol (which
> they
>         > won't). When
>         >       do i call this? How many times? etc. Maybe two additional
>         > options would be
>         >       to just add a general init() call that could cover metadata
>         > initialization
>         >       as well as this and potentially future things or continue
> to do it
>         > lazily.
>         >    3. Yeah I get that you need an expiry scheme to limit it to 4
> bytes. Is
>         >    there a mechanism to expire them, and hence shrink it?
>         >
>         > -Jay
>         >
>         > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <
> wangguoz@gmail.com> wrote:
>         >
>         > > @Jay
>         > >
>         > > 1. Stream's applicationId is shared among all instances for
> the app, and
>         > is
>         > > used as part of the consumer group id, while "app.id" is per
> producer
>         > > instance. So a Streams app that has a single "applicationID"
> config will
>         > > likely contain multiple producers each with a different appID
> based on
>         > > their corresponding taskIDs.
>         > >
>         > > 2. Another motivation besides the one pointed out by Jason for
> making
>         > sure
>         > > transaction-involved offsets have been committed before
> resuming, is that
>         > > we also want to separate the "app.id" config with the
> transactional
>         > > mechanism. More concretely, if a user does specify the "app.id"
> config
>         > and
>         > > without using transaction functions (i.e. initTransactions,
> beginTxn,
>         > etc),
>         > > they can still get idempotency guarantee across multiple
> sessions of the
>         > > producer identified by the app.id.
>         > >
>         > > 4. We thought about the PID length, note that since we do not
> expire
>         > PIDs,
>         > > we are expecting it to cover all possible producers that we
> have ever
>         > seen,
>         > > and producers without an "app.id" can come and go with
> different PIDs.
>         > > That
>         > > is why we feel 4 billion may not be sufficient.
>         > >
>         > >
>         > >
>         > > Guozhang
>         > >
>         > >
>         > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <
> jason@confluent.io>
>         > > wrote:
>         > >
>         > > > Hey Jay,
>         > > >
>         > > > Thanks for the questions! Let me take a couple of them.
>         > > >
>         > > > 2. The initTransactions() call is a little annoying. Can we
> get rid of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > >
>         > > >
>         > > > The purpose of this call is to fence off any producer with
> the same
>         > AppID
>         > > > and await the completion of any pending transactions. When
> it returns,
>         > > you
>         > > > know that your producer is safe to resume work. Take the the
> "consume
>         > and
>         > > > produce" use case as an example. We send the offset commits
> as part of
>         > > the
>         > > > producer's transaction (approximating the idea that it is
> "just another
>         > > > write to a partition"). When you first initialize the
> application, you
>         > > have
>         > > > to determine when it's safe for the consumer to read those
> offsets.
>         > > > Otherwise, you may read stale offsets before a transaction
> which is
>         > > rolling
>         > > > forward is able to write the marker to __consumer_offsets.
> So we can't
>         > do
>         > > > the initialization in send() because that would assume that
> we had
>         > > already
>         > > > read data from the consumer, which we can't do until we've
> initialized
>         > > the
>         > > > producer. Does that make sense?
>         > > >
>         > > > (For what it's worth, we're not married to this name or any
> of the
>         > > others,
>         > > > so anyone can feel free to suggest alternatives.)
>         > > >
>         > > >
>         > > > 5. One implication of factoring out the message set seems to
> be you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > >
>         > > >
>         > > > Yeah, that's a fair point. You may still be able to do some
> merging if
>         > > > adjacent message sets have the same PID, but the potential
> savings
>         > might
>         > > > not be worth the cost of implementation. My gut feeling is
> that merging
>         > > > message sets from different producers may not be a great
> idea anyway
>         > > (you'd
>         > > > have to accept the fact that you always need "deep
> iteration" to find
>         > the
>         > > > PIDs contained within the message set), but I haven't
> thought a ton
>         > about
>         > > > it. Ultimately we'll have to decide if the potential for
> savings in the
>         > > > future is worth some loss in efficiency now (for what it's
> worth, I
>         > think
>         > > > the work that Ben has been looking at also hopes to bundle
> some more
>         > > > information into the message set header).
>         > > >
>         > > > On a purely pragmatic development level, after spending a
> ton of recent
>         > > > time working with that code, I can say that the benefit of
> having a
>         > > > conceptually simpler message format is huge. It allows you
> to converge
>         > > the
>         > > > paths for validation of message sets on the broker, for
> example.
>         > > Currently,
>         > > > we pretty much need two separate paths everywhere we process
> messages.
>         > It
>         > > > can be tricky just to tell if the message you're dealing
> with is the
>         > > inner
>         > > > or outer message, and whether it matters or not. Also, the
> fact that
>         > the
>         > > > inner and outer messages share common fields makes bugs like
> KAFKA-4298
>         > > > <https://issues.apache.org/jira/browse/KAFKA-4298>
> possible. The risk
>         > of
>         > > > these bugs is much smaller when you can clearly separate the
> fields
>         > > allowed
>         > > > in the message set header and those in the messages.
>         > > >
>         > > >
>         > > > Thanks,
>         > > > Jason
>         > > >
>         > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io>
> wrote:
>         > > >
>         > > > > Looks great!
>         > > > >
>         > > > > A few questions:
>         > > > >
>         > > > >    1. What is the relationship between transaction.app.id
> and the
>         > > > existing
>         > > > >    config application.id in streams?
>         > > > >    2. The initTransactions() call is a little annoying.
> Can we get
>         > rid
>         > > of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > > >    3. The equivalent concept of what we call "fetch.mode"
> in
>         > databases
>         > > is
>         > > > >    called "isolation level" and takes values like
> "serializable",
>         > "read
>         > > > >    committed", "read uncommitted". Since we went with
> transaction as
>         > > the
>         > > > > name
>         > > > >    for the thing in between the begin/commit might make
> sense to use
>         > > this
>         > > > >    terminology for the concept and levels? I think the
> behavior we
>         > are
>         > > > >    planning is "read committed" and the alternative
> re-ordering
>         > > behavior
>         > > > is
>         > > > >    equivalent to "serializable"?
>         > > > >    4. Can the PID be made 4 bytes if we handle roll-over
> gracefully?
>         > 2
>         > > > >    billion concurrent producers should be enough for
> anyone, right?
>         > > > >    5. One implication of factoring out the message set
> seems to be
>         > you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > > >
>         > > > > -Jay
>         > > > >
>         > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> wangguoz@gmail.com>
>         > > > wrote:
>         > > > >
>         > > > > > Hi all,
>         > > > > >
>         > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
>         > > delivery
>         > > > > > semantics:
>         > > > > >
>         > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>         > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>         > > > > >
>         > > > > > This KIP adds a transactional messaging mechanism along
> with an
>         > > > > idempotent
>         > > > > > producer implementation to make sure that 1) duplicated
> messages
>         > sent
>         > > > > from
>         > > > > > the same identified producer can be detected on the
> broker side,
>         > and
>         > > > 2) a
>         > > > > > group of messages sent within a transaction will
> atomically be
>         > either
>         > > > > > reflected and fetchable to consumers or not as a whole.
>         > > > > >
>         > > > > > The above wiki page provides a high-level view of the
> proposed
>         > > changes
>         > > > as
>         > > > > > well as summarized guarantees. Initial draft of the
> detailed
>         > > > > implementation
>         > > > > > design is described in this Google doc:
>         > > > > >
>         > > > > > https://docs.google.com/document/d/11Jqy_
>         > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>         > > > > > 0wSw9ra8
>         > > > > >
>         > > > > >
>         > > > > > We would love to hear your comments and suggestions.
>         > > > > >
>         > > > > > Thanks,
>         > > > > >
>         > > > > > -- Guozhang
>         > > > > >
>         > > > >
>         > > >
>         > >
>         > >
>         > >
>         > > --
>         > > -- Guozhang
>         > >
>         > The information contained in this email is strictly confidential
> and for
>         > the use of the addressee only, unless otherwise indicated. If
> you are not
>         > the intended recipient, please do not read, copy, use or
> disclose to others
>         > this message or any attachment. Please also notify the sender by
> replying
>         > to this email or by telephone (+44(020 7896 0011) and then
> delete the email
>         > and any copies of it. Opinions, conclusion (etc) that do not
> relate to the
>         > official business of this company shall be understood as neither
> given nor
>         > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
>         > registered in England and Wales, company number 04008957) and IG
> Index
>         > Limited (a company registered in England and Wales, company
> number
>         > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
>         > London EC4R 2YA. Both IG Markets Limited (register number
> 195355) and IG
>         > Index Limited (register number 114059) are authorised and
> regulated by the
>         > Financial Conduct Authority.
>         >
>
>
>
>
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
Hi

For me delivering the once delivery story should not require transactions. Your last statement makes me feel you think they do.

Atm reading the kip I see only once delivery working end to end if I have also transactions enabled on the consumer also. I shouldn't have to do this.

Whilst having an idenpodent producer helps it doesn't solve the end to end problem. This only solves deliver into the broker. Not back out or any mirroring that may occur.

Some work is also needed on the consumer for message dupe detection. To achieve this without transactions.

This is why I really feel these should be tackled seperatly. So that there's a concentrated effort to ensure that we can deliver once only story without transactions end to end.

Like everything timestamps and now the further kip cases going in there is resuse and advancements. But they're still storied and tackle the problem independently


Cheers
Mike







Sent using OWA for iPhone
________________________________________
From: Jason Gustafson <ja...@confluent.io>
Sent: Thursday, December 8, 2016 5:46:43 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

There is no overhead from transactions if you don't use them, and they are
not needed to leverage the capabilities of the idempotent producer.
However, some of the underlying mechanics are shared, such as the message
format changes and the InitPidRequest. We see them as the two essential
ingredients needed to complete the exactly-once story, so we prefer to keep
them together.

-Jason

On Thu, Dec 8, 2016 at 12:36 AM, Michael Pearce <Mi...@ig.com>
wrote:

> My apologies, my computer is auto spellchecking and I didn’t notice before
> sending.
>
> *@Sriram
>
> Thanks
> Mike
>
> On 08/12/2016, 08:35, "Michael Pearce" <Mi...@ig.com> wrote:
>
>     @Shiram,
>     I would like to be able to have though exactly once delivery without
> the need to use more heavy weight transactions, this is why I’m proposing
> separate KIP these are not entirely orthogonal.
>
>     Agreed the uuid could be re-used in supporting transactionality, but I
> wouldn’t want to have to have full transactions and the additional
> performance costs. Note this on activemq/artemis implementation, that I can
> get/achieve the dedupe, but it is not tied into having a full transaction
> the solutions are de-coupled.
>
>
>     On 06/12/2016, 19:01, "Sriram Subramanian" <ra...@confluent.io> wrote:
>
>         @Jay
>
>         1. I totally agree on the naming. The appid for transactions is
> really an
>         instance id. Any recommendation for a name is appreciated. We had
> thought
>         of instance id, session id or app id and went with app id.
>         2. We also discussed about init() method but that could add its
> own set of
>         confusion to existing users (should I update my existing usage to
> call
>         init()? Why should I have this extra step instead of the
> constructor doing
>         it?). Transactions is going to be used by a subset of users
> (probably
>         small) and it made sense to add the burden of calling
>         initTransactions/recoverTransactions to only that subset. We are
> actually
>         open to suggestions here in terms of naming as well.
>
>         @Jonathan
>         I am not sure it adds more complexity unless you use them. We have
>         explicitly named them for transactions and the current usage of the
>         producer remains unchanged.
>
>         @Michael
>         If you look at our idempotent producer implementation in the
> kip/design,
>         this is exactly what we do except that the deduplication happens
> on the
>         server. We started with separate KIPs but it made it very
> confusing to read
>         since there were interdependencies between the concepts.
>
>
>
>         On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <
> Michael.Pearce@ig.com>
>         wrote:
>
>         > For dealing with exactly once delivery.
>         >
>         > As an alternative option has it been considered to have a
> message uuid in
>         > the record, that then is deduped on consumption?
>         >
>         > Similar to
>         > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-
> detection.html
>         >
>         > Agreed this does not deal with transaction support.
>         >
>         > But should the two concerns be separated anyhow into two kips?
>         >
>         > Cheers
>         > Mike
>         >
>         >
>         > Sent using OWA for iPhone
>         > ________________________________________
>         > From: Jay Kreps <ja...@confluent.io>
>         > Sent: Tuesday, December 6, 2016 4:47:55 PM
>         > To: dev@kafka.apache.org
>         > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
>         > Messaging
>         >
>         > Hey Guozhang,
>         >
>         >
>         >    1. My point is that it is a bit confusing to have two things
> called
>         >    application id that have different meanings, right? Won't the
> streams
>         > user
>         >    end up specifying two different application ids?
>         >    2. Makes sense. My two complaints are
>         >       1. At this point we've jumped through quite a lot of hoops
> to make
>         >       the producer lazily initialize, seems sad to get rid of
> that now.
>         >       2. The call initTransactions doesn't really make sense to
> the user
>         >       unless they understand the details of the protocol (which
> they
>         > won't). When
>         >       do i call this? How many times? etc. Maybe two additional
>         > options would be
>         >       to just add a general init() call that could cover metadata
>         > initialization
>         >       as well as this and potentially future things or continue
> to do it
>         > lazily.
>         >    3. Yeah I get that you need an expiry scheme to limit it to 4
> bytes. Is
>         >    there a mechanism to expire them, and hence shrink it?
>         >
>         > -Jay
>         >
>         > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <
> wangguoz@gmail.com> wrote:
>         >
>         > > @Jay
>         > >
>         > > 1. Stream's applicationId is shared among all instances for
> the app, and
>         > is
>         > > used as part of the consumer group id, while "app.id" is per
> producer
>         > > instance. So a Streams app that has a single "applicationID"
> config will
>         > > likely contain multiple producers each with a different appID
> based on
>         > > their corresponding taskIDs.
>         > >
>         > > 2. Another motivation besides the one pointed out by Jason for
> making
>         > sure
>         > > transaction-involved offsets have been committed before
> resuming, is that
>         > > we also want to separate the "app.id" config with the
> transactional
>         > > mechanism. More concretely, if a user does specify the "app.id"
> config
>         > and
>         > > without using transaction functions (i.e. initTransactions,
> beginTxn,
>         > etc),
>         > > they can still get idempotency guarantee across multiple
> sessions of the
>         > > producer identified by the app.id.
>         > >
>         > > 4. We thought about the PID length, note that since we do not
> expire
>         > PIDs,
>         > > we are expecting it to cover all possible producers that we
> have ever
>         > seen,
>         > > and producers without an "app.id" can come and go with
> different PIDs.
>         > > That
>         > > is why we feel 4 billion may not be sufficient.
>         > >
>         > >
>         > >
>         > > Guozhang
>         > >
>         > >
>         > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <
> jason@confluent.io>
>         > > wrote:
>         > >
>         > > > Hey Jay,
>         > > >
>         > > > Thanks for the questions! Let me take a couple of them.
>         > > >
>         > > > 2. The initTransactions() call is a little annoying. Can we
> get rid of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > >
>         > > >
>         > > > The purpose of this call is to fence off any producer with
> the same
>         > AppID
>         > > > and await the completion of any pending transactions. When
> it returns,
>         > > you
>         > > > know that your producer is safe to resume work. Take the the
> "consume
>         > and
>         > > > produce" use case as an example. We send the offset commits
> as part of
>         > > the
>         > > > producer's transaction (approximating the idea that it is
> "just another
>         > > > write to a partition"). When you first initialize the
> application, you
>         > > have
>         > > > to determine when it's safe for the consumer to read those
> offsets.
>         > > > Otherwise, you may read stale offsets before a transaction
> which is
>         > > rolling
>         > > > forward is able to write the marker to __consumer_offsets.
> So we can't
>         > do
>         > > > the initialization in send() because that would assume that
> we had
>         > > already
>         > > > read data from the consumer, which we can't do until we've
> initialized
>         > > the
>         > > > producer. Does that make sense?
>         > > >
>         > > > (For what it's worth, we're not married to this name or any
> of the
>         > > others,
>         > > > so anyone can feel free to suggest alternatives.)
>         > > >
>         > > >
>         > > > 5. One implication of factoring out the message set seems to
> be you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > >
>         > > >
>         > > > Yeah, that's a fair point. You may still be able to do some
> merging if
>         > > > adjacent message sets have the same PID, but the potential
> savings
>         > might
>         > > > not be worth the cost of implementation. My gut feeling is
> that merging
>         > > > message sets from different producers may not be a great
> idea anyway
>         > > (you'd
>         > > > have to accept the fact that you always need "deep
> iteration" to find
>         > the
>         > > > PIDs contained within the message set), but I haven't
> thought a ton
>         > about
>         > > > it. Ultimately we'll have to decide if the potential for
> savings in the
>         > > > future is worth some loss in efficiency now (for what it's
> worth, I
>         > think
>         > > > the work that Ben has been looking at also hopes to bundle
> some more
>         > > > information into the message set header).
>         > > >
>         > > > On a purely pragmatic development level, after spending a
> ton of recent
>         > > > time working with that code, I can say that the benefit of
> having a
>         > > > conceptually simpler message format is huge. It allows you
> to converge
>         > > the
>         > > > paths for validation of message sets on the broker, for
> example.
>         > > Currently,
>         > > > we pretty much need two separate paths everywhere we process
> messages.
>         > It
>         > > > can be tricky just to tell if the message you're dealing
> with is the
>         > > inner
>         > > > or outer message, and whether it matters or not. Also, the
> fact that
>         > the
>         > > > inner and outer messages share common fields makes bugs like
> KAFKA-4298
>         > > > <https://issues.apache.org/jira/browse/KAFKA-4298>
> possible. The risk
>         > of
>         > > > these bugs is much smaller when you can clearly separate the
> fields
>         > > allowed
>         > > > in the message set header and those in the messages.
>         > > >
>         > > >
>         > > > Thanks,
>         > > > Jason
>         > > >
>         > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io>
> wrote:
>         > > >
>         > > > > Looks great!
>         > > > >
>         > > > > A few questions:
>         > > > >
>         > > > >    1. What is the relationship between transaction.app.id
> and the
>         > > > existing
>         > > > >    config application.id in streams?
>         > > > >    2. The initTransactions() call is a little annoying.
> Can we get
>         > rid
>         > > of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > > >    3. The equivalent concept of what we call "fetch.mode"
> in
>         > databases
>         > > is
>         > > > >    called "isolation level" and takes values like
> "serializable",
>         > "read
>         > > > >    committed", "read uncommitted". Since we went with
> transaction as
>         > > the
>         > > > > name
>         > > > >    for the thing in between the begin/commit might make
> sense to use
>         > > this
>         > > > >    terminology for the concept and levels? I think the
> behavior we
>         > are
>         > > > >    planning is "read committed" and the alternative
> re-ordering
>         > > behavior
>         > > > is
>         > > > >    equivalent to "serializable"?
>         > > > >    4. Can the PID be made 4 bytes if we handle roll-over
> gracefully?
>         > 2
>         > > > >    billion concurrent producers should be enough for
> anyone, right?
>         > > > >    5. One implication of factoring out the message set
> seems to be
>         > you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > > >
>         > > > > -Jay
>         > > > >
>         > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> wangguoz@gmail.com>
>         > > > wrote:
>         > > > >
>         > > > > > Hi all,
>         > > > > >
>         > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
>         > > delivery
>         > > > > > semantics:
>         > > > > >
>         > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>         > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>         > > > > >
>         > > > > > This KIP adds a transactional messaging mechanism along
> with an
>         > > > > idempotent
>         > > > > > producer implementation to make sure that 1) duplicated
> messages
>         > sent
>         > > > > from
>         > > > > > the same identified producer can be detected on the
> broker side,
>         > and
>         > > > 2) a
>         > > > > > group of messages sent within a transaction will
> atomically be
>         > either
>         > > > > > reflected and fetchable to consumers or not as a whole.
>         > > > > >
>         > > > > > The above wiki page provides a high-level view of the
> proposed
>         > > changes
>         > > > as
>         > > > > > well as summarized guarantees. Initial draft of the
> detailed
>         > > > > implementation
>         > > > > > design is described in this Google doc:
>         > > > > >
>         > > > > > https://docs.google.com/document/d/11Jqy_
>         > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>         > > > > > 0wSw9ra8
>         > > > > >
>         > > > > >
>         > > > > > We would love to hear your comments and suggestions.
>         > > > > >
>         > > > > > Thanks,
>         > > > > >
>         > > > > > -- Guozhang
>         > > > > >
>         > > > >
>         > > >
>         > >
>         > >
>         > >
>         > > --
>         > > -- Guozhang
>         > >
>         > The information contained in this email is strictly confidential
> and for
>         > the use of the addressee only, unless otherwise indicated. If
> you are not
>         > the intended recipient, please do not read, copy, use or
> disclose to others
>         > this message or any attachment. Please also notify the sender by
> replying
>         > to this email or by telephone (+44(020 7896 0011) and then
> delete the email
>         > and any copies of it. Opinions, conclusion (etc) that do not
> relate to the
>         > official business of this company shall be understood as neither
> given nor
>         > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
>         > registered in England and Wales, company number 04008957) and IG
> Index
>         > Limited (a company registered in England and Wales, company
> number
>         > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
>         > London EC4R 2YA. Both IG Markets Limited (register number
> 195355) and IG
>         > Index Limited (register number 114059) are authorised and
> regulated by the
>         > Financial Conduct Authority.
>         >
>
>
>
>
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Michael,

There is no overhead from transactions if you don't use them, and they are
not needed to leverage the capabilities of the idempotent producer.
However, some of the underlying mechanics are shared, such as the message
format changes and the InitPidRequest. We see them as the two essential
ingredients needed to complete the exactly-once story, so we prefer to keep
them together.

-Jason

On Thu, Dec 8, 2016 at 12:36 AM, Michael Pearce <Mi...@ig.com>
wrote:

> My apologies, my computer is auto spellchecking and I didn’t notice before
> sending.
>
> *@Sriram
>
> Thanks
> Mike
>
> On 08/12/2016, 08:35, "Michael Pearce" <Mi...@ig.com> wrote:
>
>     @Shiram,
>     I would like to be able to have though exactly once delivery without
> the need to use more heavy weight transactions, this is why I’m proposing
> separate KIP these are not entirely orthogonal.
>
>     Agreed the uuid could be re-used in supporting transactionality, but I
> wouldn’t want to have to have full transactions and the additional
> performance costs. Note this on activemq/artemis implementation, that I can
> get/achieve the dedupe, but it is not tied into having a full transaction
> the solutions are de-coupled.
>
>
>     On 06/12/2016, 19:01, "Sriram Subramanian" <ra...@confluent.io> wrote:
>
>         @Jay
>
>         1. I totally agree on the naming. The appid for transactions is
> really an
>         instance id. Any recommendation for a name is appreciated. We had
> thought
>         of instance id, session id or app id and went with app id.
>         2. We also discussed about init() method but that could add its
> own set of
>         confusion to existing users (should I update my existing usage to
> call
>         init()? Why should I have this extra step instead of the
> constructor doing
>         it?). Transactions is going to be used by a subset of users
> (probably
>         small) and it made sense to add the burden of calling
>         initTransactions/recoverTransactions to only that subset. We are
> actually
>         open to suggestions here in terms of naming as well.
>
>         @Jonathan
>         I am not sure it adds more complexity unless you use them. We have
>         explicitly named them for transactions and the current usage of the
>         producer remains unchanged.
>
>         @Michael
>         If you look at our idempotent producer implementation in the
> kip/design,
>         this is exactly what we do except that the deduplication happens
> on the
>         server. We started with separate KIPs but it made it very
> confusing to read
>         since there were interdependencies between the concepts.
>
>
>
>         On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <
> Michael.Pearce@ig.com>
>         wrote:
>
>         > For dealing with exactly once delivery.
>         >
>         > As an alternative option has it been considered to have a
> message uuid in
>         > the record, that then is deduped on consumption?
>         >
>         > Similar to
>         > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-
> detection.html
>         >
>         > Agreed this does not deal with transaction support.
>         >
>         > But should the two concerns be separated anyhow into two kips?
>         >
>         > Cheers
>         > Mike
>         >
>         >
>         > Sent using OWA for iPhone
>         > ________________________________________
>         > From: Jay Kreps <ja...@confluent.io>
>         > Sent: Tuesday, December 6, 2016 4:47:55 PM
>         > To: dev@kafka.apache.org
>         > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
>         > Messaging
>         >
>         > Hey Guozhang,
>         >
>         >
>         >    1. My point is that it is a bit confusing to have two things
> called
>         >    application id that have different meanings, right? Won't the
> streams
>         > user
>         >    end up specifying two different application ids?
>         >    2. Makes sense. My two complaints are
>         >       1. At this point we've jumped through quite a lot of hoops
> to make
>         >       the producer lazily initialize, seems sad to get rid of
> that now.
>         >       2. The call initTransactions doesn't really make sense to
> the user
>         >       unless they understand the details of the protocol (which
> they
>         > won't). When
>         >       do i call this? How many times? etc. Maybe two additional
>         > options would be
>         >       to just add a general init() call that could cover metadata
>         > initialization
>         >       as well as this and potentially future things or continue
> to do it
>         > lazily.
>         >    3. Yeah I get that you need an expiry scheme to limit it to 4
> bytes. Is
>         >    there a mechanism to expire them, and hence shrink it?
>         >
>         > -Jay
>         >
>         > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <
> wangguoz@gmail.com> wrote:
>         >
>         > > @Jay
>         > >
>         > > 1. Stream's applicationId is shared among all instances for
> the app, and
>         > is
>         > > used as part of the consumer group id, while "app.id" is per
> producer
>         > > instance. So a Streams app that has a single "applicationID"
> config will
>         > > likely contain multiple producers each with a different appID
> based on
>         > > their corresponding taskIDs.
>         > >
>         > > 2. Another motivation besides the one pointed out by Jason for
> making
>         > sure
>         > > transaction-involved offsets have been committed before
> resuming, is that
>         > > we also want to separate the "app.id" config with the
> transactional
>         > > mechanism. More concretely, if a user does specify the "app.id"
> config
>         > and
>         > > without using transaction functions (i.e. initTransactions,
> beginTxn,
>         > etc),
>         > > they can still get idempotency guarantee across multiple
> sessions of the
>         > > producer identified by the app.id.
>         > >
>         > > 4. We thought about the PID length, note that since we do not
> expire
>         > PIDs,
>         > > we are expecting it to cover all possible producers that we
> have ever
>         > seen,
>         > > and producers without an "app.id" can come and go with
> different PIDs.
>         > > That
>         > > is why we feel 4 billion may not be sufficient.
>         > >
>         > >
>         > >
>         > > Guozhang
>         > >
>         > >
>         > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <
> jason@confluent.io>
>         > > wrote:
>         > >
>         > > > Hey Jay,
>         > > >
>         > > > Thanks for the questions! Let me take a couple of them.
>         > > >
>         > > > 2. The initTransactions() call is a little annoying. Can we
> get rid of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > >
>         > > >
>         > > > The purpose of this call is to fence off any producer with
> the same
>         > AppID
>         > > > and await the completion of any pending transactions. When
> it returns,
>         > > you
>         > > > know that your producer is safe to resume work. Take the the
> "consume
>         > and
>         > > > produce" use case as an example. We send the offset commits
> as part of
>         > > the
>         > > > producer's transaction (approximating the idea that it is
> "just another
>         > > > write to a partition"). When you first initialize the
> application, you
>         > > have
>         > > > to determine when it's safe for the consumer to read those
> offsets.
>         > > > Otherwise, you may read stale offsets before a transaction
> which is
>         > > rolling
>         > > > forward is able to write the marker to __consumer_offsets.
> So we can't
>         > do
>         > > > the initialization in send() because that would assume that
> we had
>         > > already
>         > > > read data from the consumer, which we can't do until we've
> initialized
>         > > the
>         > > > producer. Does that make sense?
>         > > >
>         > > > (For what it's worth, we're not married to this name or any
> of the
>         > > others,
>         > > > so anyone can feel free to suggest alternatives.)
>         > > >
>         > > >
>         > > > 5. One implication of factoring out the message set seems to
> be you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > >
>         > > >
>         > > > Yeah, that's a fair point. You may still be able to do some
> merging if
>         > > > adjacent message sets have the same PID, but the potential
> savings
>         > might
>         > > > not be worth the cost of implementation. My gut feeling is
> that merging
>         > > > message sets from different producers may not be a great
> idea anyway
>         > > (you'd
>         > > > have to accept the fact that you always need "deep
> iteration" to find
>         > the
>         > > > PIDs contained within the message set), but I haven't
> thought a ton
>         > about
>         > > > it. Ultimately we'll have to decide if the potential for
> savings in the
>         > > > future is worth some loss in efficiency now (for what it's
> worth, I
>         > think
>         > > > the work that Ben has been looking at also hopes to bundle
> some more
>         > > > information into the message set header).
>         > > >
>         > > > On a purely pragmatic development level, after spending a
> ton of recent
>         > > > time working with that code, I can say that the benefit of
> having a
>         > > > conceptually simpler message format is huge. It allows you
> to converge
>         > > the
>         > > > paths for validation of message sets on the broker, for
> example.
>         > > Currently,
>         > > > we pretty much need two separate paths everywhere we process
> messages.
>         > It
>         > > > can be tricky just to tell if the message you're dealing
> with is the
>         > > inner
>         > > > or outer message, and whether it matters or not. Also, the
> fact that
>         > the
>         > > > inner and outer messages share common fields makes bugs like
> KAFKA-4298
>         > > > <https://issues.apache.org/jira/browse/KAFKA-4298>
> possible. The risk
>         > of
>         > > > these bugs is much smaller when you can clearly separate the
> fields
>         > > allowed
>         > > > in the message set header and those in the messages.
>         > > >
>         > > >
>         > > > Thanks,
>         > > > Jason
>         > > >
>         > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io>
> wrote:
>         > > >
>         > > > > Looks great!
>         > > > >
>         > > > > A few questions:
>         > > > >
>         > > > >    1. What is the relationship between transaction.app.id
> and the
>         > > > existing
>         > > > >    config application.id in streams?
>         > > > >    2. The initTransactions() call is a little annoying.
> Can we get
>         > rid
>         > > of
>         > > > >    that and call it automatically if you set a
> transaction.app.id
>         > when
>         > > > we
>         > > > >    do the first message send as we do with metadata?
> Arguably we
>         > should
>         > > > > have
>         > > > >    included a general connect() or init() call in the
> producer, but
>         > > given
>         > > > > that
>         > > > >    we didn't do this it seems weird that the cluster
> metadata
>         > > initializes
>         > > > >    automatically on demand and the transaction metadata
> doesn't.
>         > > > >    3. The equivalent concept of what we call "fetch.mode"
> in
>         > databases
>         > > is
>         > > > >    called "isolation level" and takes values like
> "serializable",
>         > "read
>         > > > >    committed", "read uncommitted". Since we went with
> transaction as
>         > > the
>         > > > > name
>         > > > >    for the thing in between the begin/commit might make
> sense to use
>         > > this
>         > > > >    terminology for the concept and levels? I think the
> behavior we
>         > are
>         > > > >    planning is "read committed" and the alternative
> re-ordering
>         > > behavior
>         > > > is
>         > > > >    equivalent to "serializable"?
>         > > > >    4. Can the PID be made 4 bytes if we handle roll-over
> gracefully?
>         > 2
>         > > > >    billion concurrent producers should be enough for
> anyone, right?
>         > > > >    5. One implication of factoring out the message set
> seems to be
>         > you
>         > > > >    can't ever "repack" messages to improve compression
> beyond what is
>         > > > done
>         > > > > by
>         > > > >    the producer. We'd talked about doing this either by
> buffering
>         > when
>         > > > > writing
>         > > > >    or during log cleaning. This isn't a show stopper but I
> think one
>         > > > >    implication is that we won't be able to do this.
> Furthermore with
>         > > log
>         > > > >    cleaning you'd assume that over time ALL messages would
> collapse
>         > > down
>         > > > > to a
>         > > > >    single wrapper as compaction removes the others.
>         > > > >
>         > > > > -Jay
>         > > > >
>         > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <
> wangguoz@gmail.com>
>         > > > wrote:
>         > > > >
>         > > > > > Hi all,
>         > > > > >
>         > > > > > I have just created KIP-98 to enhance Kafka with exactly
> once
>         > > delivery
>         > > > > > semantics:
>         > > > > >
>         > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>         > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>         > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>         > > > > >
>         > > > > > This KIP adds a transactional messaging mechanism along
> with an
>         > > > > idempotent
>         > > > > > producer implementation to make sure that 1) duplicated
> messages
>         > sent
>         > > > > from
>         > > > > > the same identified producer can be detected on the
> broker side,
>         > and
>         > > > 2) a
>         > > > > > group of messages sent within a transaction will
> atomically be
>         > either
>         > > > > > reflected and fetchable to consumers or not as a whole.
>         > > > > >
>         > > > > > The above wiki page provides a high-level view of the
> proposed
>         > > changes
>         > > > as
>         > > > > > well as summarized guarantees. Initial draft of the
> detailed
>         > > > > implementation
>         > > > > > design is described in this Google doc:
>         > > > > >
>         > > > > > https://docs.google.com/document/d/11Jqy_
>         > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
>         > > > > > 0wSw9ra8
>         > > > > >
>         > > > > >
>         > > > > > We would love to hear your comments and suggestions.
>         > > > > >
>         > > > > > Thanks,
>         > > > > >
>         > > > > > -- Guozhang
>         > > > > >
>         > > > >
>         > > >
>         > >
>         > >
>         > >
>         > > --
>         > > -- Guozhang
>         > >
>         > The information contained in this email is strictly confidential
> and for
>         > the use of the addressee only, unless otherwise indicated. If
> you are not
>         > the intended recipient, please do not read, copy, use or
> disclose to others
>         > this message or any attachment. Please also notify the sender by
> replying
>         > to this email or by telephone (+44(020 7896 0011) and then
> delete the email
>         > and any copies of it. Opinions, conclusion (etc) that do not
> relate to the
>         > official business of this company shall be understood as neither
> given nor
>         > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
>         > registered in England and Wales, company number 04008957) and IG
> Index
>         > Limited (a company registered in England and Wales, company
> number
>         > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
>         > London EC4R 2YA. Both IG Markets Limited (register number
> 195355) and IG
>         > Index Limited (register number 114059) are authorised and
> regulated by the
>         > Financial Conduct Authority.
>         >
>
>
>
>
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
My apologies, my computer is auto spellchecking and I didn’t notice before sending.

*@Sriram

Thanks
Mike

On 08/12/2016, 08:35, "Michael Pearce" <Mi...@ig.com> wrote:

    @Shiram,
    I would like to be able to have though exactly once delivery without the need to use more heavy weight transactions, this is why I’m proposing separate KIP these are not entirely orthogonal.

    Agreed the uuid could be re-used in supporting transactionality, but I wouldn’t want to have to have full transactions and the additional performance costs. Note this on activemq/artemis implementation, that I can get/achieve the dedupe, but it is not tied into having a full transaction the solutions are de-coupled.


    On 06/12/2016, 19:01, "Sriram Subramanian" <ra...@confluent.io> wrote:

        @Jay

        1. I totally agree on the naming. The appid for transactions is really an
        instance id. Any recommendation for a name is appreciated. We had thought
        of instance id, session id or app id and went with app id.
        2. We also discussed about init() method but that could add its own set of
        confusion to existing users (should I update my existing usage to call
        init()? Why should I have this extra step instead of the constructor doing
        it?). Transactions is going to be used by a subset of users (probably
        small) and it made sense to add the burden of calling
        initTransactions/recoverTransactions to only that subset. We are actually
        open to suggestions here in terms of naming as well.

        @Jonathan
        I am not sure it adds more complexity unless you use them. We have
        explicitly named them for transactions and the current usage of the
        producer remains unchanged.

        @Michael
        If you look at our idempotent producer implementation in the kip/design,
        this is exactly what we do except that the deduplication happens on the
        server. We started with separate KIPs but it made it very confusing to read
        since there were interdependencies between the concepts.



        On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <Mi...@ig.com>
        wrote:

        > For dealing with exactly once delivery.
        >
        > As an alternative option has it been considered to have a message uuid in
        > the record, that then is deduped on consumption?
        >
        > Similar to
        > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-detection.html
        >
        > Agreed this does not deal with transaction support.
        >
        > But should the two concerns be separated anyhow into two kips?
        >
        > Cheers
        > Mike
        >
        >
        > Sent using OWA for iPhone
        > ________________________________________
        > From: Jay Kreps <ja...@confluent.io>
        > Sent: Tuesday, December 6, 2016 4:47:55 PM
        > To: dev@kafka.apache.org
        > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
        > Messaging
        >
        > Hey Guozhang,
        >
        >
        >    1. My point is that it is a bit confusing to have two things called
        >    application id that have different meanings, right? Won't the streams
        > user
        >    end up specifying two different application ids?
        >    2. Makes sense. My two complaints are
        >       1. At this point we've jumped through quite a lot of hoops to make
        >       the producer lazily initialize, seems sad to get rid of that now.
        >       2. The call initTransactions doesn't really make sense to the user
        >       unless they understand the details of the protocol (which they
        > won't). When
        >       do i call this? How many times? etc. Maybe two additional
        > options would be
        >       to just add a general init() call that could cover metadata
        > initialization
        >       as well as this and potentially future things or continue to do it
        > lazily.
        >    3. Yeah I get that you need an expiry scheme to limit it to 4 bytes. Is
        >    there a mechanism to expire them, and hence shrink it?
        >
        > -Jay
        >
        > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com> wrote:
        >
        > > @Jay
        > >
        > > 1. Stream's applicationId is shared among all instances for the app, and
        > is
        > > used as part of the consumer group id, while "app.id" is per producer
        > > instance. So a Streams app that has a single "applicationID" config will
        > > likely contain multiple producers each with a different appID based on
        > > their corresponding taskIDs.
        > >
        > > 2. Another motivation besides the one pointed out by Jason for making
        > sure
        > > transaction-involved offsets have been committed before resuming, is that
        > > we also want to separate the "app.id" config with the transactional
        > > mechanism. More concretely, if a user does specify the "app.id" config
        > and
        > > without using transaction functions (i.e. initTransactions, beginTxn,
        > etc),
        > > they can still get idempotency guarantee across multiple sessions of the
        > > producer identified by the app.id.
        > >
        > > 4. We thought about the PID length, note that since we do not expire
        > PIDs,
        > > we are expecting it to cover all possible producers that we have ever
        > seen,
        > > and producers without an "app.id" can come and go with different PIDs.
        > > That
        > > is why we feel 4 billion may not be sufficient.
        > >
        > >
        > >
        > > Guozhang
        > >
        > >
        > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
        > > wrote:
        > >
        > > > Hey Jay,
        > > >
        > > > Thanks for the questions! Let me take a couple of them.
        > > >
        > > > 2. The initTransactions() call is a little annoying. Can we get rid of
        > > > >    that and call it automatically if you set a transaction.app.id
        > when
        > > > we
        > > > >    do the first message send as we do with metadata? Arguably we
        > should
        > > > > have
        > > > >    included a general connect() or init() call in the producer, but
        > > given
        > > > > that
        > > > >    we didn't do this it seems weird that the cluster metadata
        > > initializes
        > > > >    automatically on demand and the transaction metadata doesn't.
        > > >
        > > >
        > > > The purpose of this call is to fence off any producer with the same
        > AppID
        > > > and await the completion of any pending transactions. When it returns,
        > > you
        > > > know that your producer is safe to resume work. Take the the "consume
        > and
        > > > produce" use case as an example. We send the offset commits as part of
        > > the
        > > > producer's transaction (approximating the idea that it is "just another
        > > > write to a partition"). When you first initialize the application, you
        > > have
        > > > to determine when it's safe for the consumer to read those offsets.
        > > > Otherwise, you may read stale offsets before a transaction which is
        > > rolling
        > > > forward is able to write the marker to __consumer_offsets. So we can't
        > do
        > > > the initialization in send() because that would assume that we had
        > > already
        > > > read data from the consumer, which we can't do until we've initialized
        > > the
        > > > producer. Does that make sense?
        > > >
        > > > (For what it's worth, we're not married to this name or any of the
        > > others,
        > > > so anyone can feel free to suggest alternatives.)
        > > >
        > > >
        > > > 5. One implication of factoring out the message set seems to be you
        > > > >    can't ever "repack" messages to improve compression beyond what is
        > > > done
        > > > > by
        > > > >    the producer. We'd talked about doing this either by buffering
        > when
        > > > > writing
        > > > >    or during log cleaning. This isn't a show stopper but I think one
        > > > >    implication is that we won't be able to do this. Furthermore with
        > > log
        > > > >    cleaning you'd assume that over time ALL messages would collapse
        > > down
        > > > > to a
        > > > >    single wrapper as compaction removes the others.
        > > >
        > > >
        > > > Yeah, that's a fair point. You may still be able to do some merging if
        > > > adjacent message sets have the same PID, but the potential savings
        > might
        > > > not be worth the cost of implementation. My gut feeling is that merging
        > > > message sets from different producers may not be a great idea anyway
        > > (you'd
        > > > have to accept the fact that you always need "deep iteration" to find
        > the
        > > > PIDs contained within the message set), but I haven't thought a ton
        > about
        > > > it. Ultimately we'll have to decide if the potential for savings in the
        > > > future is worth some loss in efficiency now (for what it's worth, I
        > think
        > > > the work that Ben has been looking at also hopes to bundle some more
        > > > information into the message set header).
        > > >
        > > > On a purely pragmatic development level, after spending a ton of recent
        > > > time working with that code, I can say that the benefit of having a
        > > > conceptually simpler message format is huge. It allows you to converge
        > > the
        > > > paths for validation of message sets on the broker, for example.
        > > Currently,
        > > > we pretty much need two separate paths everywhere we process messages.
        > It
        > > > can be tricky just to tell if the message you're dealing with is the
        > > inner
        > > > or outer message, and whether it matters or not. Also, the fact that
        > the
        > > > inner and outer messages share common fields makes bugs like KAFKA-4298
        > > > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk
        > of
        > > > these bugs is much smaller when you can clearly separate the fields
        > > allowed
        > > > in the message set header and those in the messages.
        > > >
        > > >
        > > > Thanks,
        > > > Jason
        > > >
        > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
        > > >
        > > > > Looks great!
        > > > >
        > > > > A few questions:
        > > > >
        > > > >    1. What is the relationship between transaction.app.id and the
        > > > existing
        > > > >    config application.id in streams?
        > > > >    2. The initTransactions() call is a little annoying. Can we get
        > rid
        > > of
        > > > >    that and call it automatically if you set a transaction.app.id
        > when
        > > > we
        > > > >    do the first message send as we do with metadata? Arguably we
        > should
        > > > > have
        > > > >    included a general connect() or init() call in the producer, but
        > > given
        > > > > that
        > > > >    we didn't do this it seems weird that the cluster metadata
        > > initializes
        > > > >    automatically on demand and the transaction metadata doesn't.
        > > > >    3. The equivalent concept of what we call "fetch.mode" in
        > databases
        > > is
        > > > >    called "isolation level" and takes values like "serializable",
        > "read
        > > > >    committed", "read uncommitted". Since we went with transaction as
        > > the
        > > > > name
        > > > >    for the thing in between the begin/commit might make sense to use
        > > this
        > > > >    terminology for the concept and levels? I think the behavior we
        > are
        > > > >    planning is "read committed" and the alternative re-ordering
        > > behavior
        > > > is
        > > > >    equivalent to "serializable"?
        > > > >    4. Can the PID be made 4 bytes if we handle roll-over gracefully?
        > 2
        > > > >    billion concurrent producers should be enough for anyone, right?
        > > > >    5. One implication of factoring out the message set seems to be
        > you
        > > > >    can't ever "repack" messages to improve compression beyond what is
        > > > done
        > > > > by
        > > > >    the producer. We'd talked about doing this either by buffering
        > when
        > > > > writing
        > > > >    or during log cleaning. This isn't a show stopper but I think one
        > > > >    implication is that we won't be able to do this. Furthermore with
        > > log
        > > > >    cleaning you'd assume that over time ALL messages would collapse
        > > down
        > > > > to a
        > > > >    single wrapper as compaction removes the others.
        > > > >
        > > > > -Jay
        > > > >
        > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
        > > > wrote:
        > > > >
        > > > > > Hi all,
        > > > > >
        > > > > > I have just created KIP-98 to enhance Kafka with exactly once
        > > delivery
        > > > > > semantics:
        > > > > >
        > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
        > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
        > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
        > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
        > > > > >
        > > > > > This KIP adds a transactional messaging mechanism along with an
        > > > > idempotent
        > > > > > producer implementation to make sure that 1) duplicated messages
        > sent
        > > > > from
        > > > > > the same identified producer can be detected on the broker side,
        > and
        > > > 2) a
        > > > > > group of messages sent within a transaction will atomically be
        > either
        > > > > > reflected and fetchable to consumers or not as a whole.
        > > > > >
        > > > > > The above wiki page provides a high-level view of the proposed
        > > changes
        > > > as
        > > > > > well as summarized guarantees. Initial draft of the detailed
        > > > > implementation
        > > > > > design is described in this Google doc:
        > > > > >
        > > > > > https://docs.google.com/document/d/11Jqy_
        > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
        > > > > > 0wSw9ra8
        > > > > >
        > > > > >
        > > > > > We would love to hear your comments and suggestions.
        > > > > >
        > > > > > Thanks,
        > > > > >
        > > > > > -- Guozhang
        > > > > >
        > > > >
        > > >
        > >
        > >
        > >
        > > --
        > > -- Guozhang
        > >
        > The information contained in this email is strictly confidential and for
        > the use of the addressee only, unless otherwise indicated. If you are not
        > the intended recipient, please do not read, copy, use or disclose to others
        > this message or any attachment. Please also notify the sender by replying
        > to this email or by telephone (+44(020 7896 0011) and then delete the email
        > and any copies of it. Opinions, conclusion (etc) that do not relate to the
        > official business of this company shall be understood as neither given nor
        > endorsed by it. IG is a trading name of IG Markets Limited (a company
        > registered in England and Wales, company number 04008957) and IG Index
        > Limited (a company registered in England and Wales, company number
        > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
        > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
        > Index Limited (register number 114059) are authorised and regulated by the
        > Financial Conduct Authority.
        >




The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
@Shiram,
I would like to be able to have though exactly once delivery without the need to use more heavy weight transactions, this is why I’m proposing separate KIP these are not entirely orthogonal.

Agreed the uuid could be re-used in supporting transactionality, but I wouldn’t want to have to have full transactions and the additional performance costs. Note this on activemq/artemis implementation, that I can get/achieve the dedupe, but it is not tied into having a full transaction the solutions are de-coupled.


On 06/12/2016, 19:01, "Sriram Subramanian" <ra...@confluent.io> wrote:

    @Jay

    1. I totally agree on the naming. The appid for transactions is really an
    instance id. Any recommendation for a name is appreciated. We had thought
    of instance id, session id or app id and went with app id.
    2. We also discussed about init() method but that could add its own set of
    confusion to existing users (should I update my existing usage to call
    init()? Why should I have this extra step instead of the constructor doing
    it?). Transactions is going to be used by a subset of users (probably
    small) and it made sense to add the burden of calling
    initTransactions/recoverTransactions to only that subset. We are actually
    open to suggestions here in terms of naming as well.

    @Jonathan
    I am not sure it adds more complexity unless you use them. We have
    explicitly named them for transactions and the current usage of the
    producer remains unchanged.

    @Michael
    If you look at our idempotent producer implementation in the kip/design,
    this is exactly what we do except that the deduplication happens on the
    server. We started with separate KIPs but it made it very confusing to read
    since there were interdependencies between the concepts.



    On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <Mi...@ig.com>
    wrote:

    > For dealing with exactly once delivery.
    >
    > As an alternative option has it been considered to have a message uuid in
    > the record, that then is deduped on consumption?
    >
    > Similar to
    > https://activemq.apache.org/artemis/docs/1.0.0/duplicate-detection.html
    >
    > Agreed this does not deal with transaction support.
    >
    > But should the two concerns be separated anyhow into two kips?
    >
    > Cheers
    > Mike
    >
    >
    > Sent using OWA for iPhone
    > ________________________________________
    > From: Jay Kreps <ja...@confluent.io>
    > Sent: Tuesday, December 6, 2016 4:47:55 PM
    > To: dev@kafka.apache.org
    > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
    > Messaging
    >
    > Hey Guozhang,
    >
    >
    >    1. My point is that it is a bit confusing to have two things called
    >    application id that have different meanings, right? Won't the streams
    > user
    >    end up specifying two different application ids?
    >    2. Makes sense. My two complaints are
    >       1. At this point we've jumped through quite a lot of hoops to make
    >       the producer lazily initialize, seems sad to get rid of that now.
    >       2. The call initTransactions doesn't really make sense to the user
    >       unless they understand the details of the protocol (which they
    > won't). When
    >       do i call this? How many times? etc. Maybe two additional
    > options would be
    >       to just add a general init() call that could cover metadata
    > initialization
    >       as well as this and potentially future things or continue to do it
    > lazily.
    >    3. Yeah I get that you need an expiry scheme to limit it to 4 bytes. Is
    >    there a mechanism to expire them, and hence shrink it?
    >
    > -Jay
    >
    > On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com> wrote:
    >
    > > @Jay
    > >
    > > 1. Stream's applicationId is shared among all instances for the app, and
    > is
    > > used as part of the consumer group id, while "app.id" is per producer
    > > instance. So a Streams app that has a single "applicationID" config will
    > > likely contain multiple producers each with a different appID based on
    > > their corresponding taskIDs.
    > >
    > > 2. Another motivation besides the one pointed out by Jason for making
    > sure
    > > transaction-involved offsets have been committed before resuming, is that
    > > we also want to separate the "app.id" config with the transactional
    > > mechanism. More concretely, if a user does specify the "app.id" config
    > and
    > > without using transaction functions (i.e. initTransactions, beginTxn,
    > etc),
    > > they can still get idempotency guarantee across multiple sessions of the
    > > producer identified by the app.id.
    > >
    > > 4. We thought about the PID length, note that since we do not expire
    > PIDs,
    > > we are expecting it to cover all possible producers that we have ever
    > seen,
    > > and producers without an "app.id" can come and go with different PIDs.
    > > That
    > > is why we feel 4 billion may not be sufficient.
    > >
    > >
    > >
    > > Guozhang
    > >
    > >
    > > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
    > > wrote:
    > >
    > > > Hey Jay,
    > > >
    > > > Thanks for the questions! Let me take a couple of them.
    > > >
    > > > 2. The initTransactions() call is a little annoying. Can we get rid of
    > > > >    that and call it automatically if you set a transaction.app.id
    > when
    > > > we
    > > > >    do the first message send as we do with metadata? Arguably we
    > should
    > > > > have
    > > > >    included a general connect() or init() call in the producer, but
    > > given
    > > > > that
    > > > >    we didn't do this it seems weird that the cluster metadata
    > > initializes
    > > > >    automatically on demand and the transaction metadata doesn't.
    > > >
    > > >
    > > > The purpose of this call is to fence off any producer with the same
    > AppID
    > > > and await the completion of any pending transactions. When it returns,
    > > you
    > > > know that your producer is safe to resume work. Take the the "consume
    > and
    > > > produce" use case as an example. We send the offset commits as part of
    > > the
    > > > producer's transaction (approximating the idea that it is "just another
    > > > write to a partition"). When you first initialize the application, you
    > > have
    > > > to determine when it's safe for the consumer to read those offsets.
    > > > Otherwise, you may read stale offsets before a transaction which is
    > > rolling
    > > > forward is able to write the marker to __consumer_offsets. So we can't
    > do
    > > > the initialization in send() because that would assume that we had
    > > already
    > > > read data from the consumer, which we can't do until we've initialized
    > > the
    > > > producer. Does that make sense?
    > > >
    > > > (For what it's worth, we're not married to this name or any of the
    > > others,
    > > > so anyone can feel free to suggest alternatives.)
    > > >
    > > >
    > > > 5. One implication of factoring out the message set seems to be you
    > > > >    can't ever "repack" messages to improve compression beyond what is
    > > > done
    > > > > by
    > > > >    the producer. We'd talked about doing this either by buffering
    > when
    > > > > writing
    > > > >    or during log cleaning. This isn't a show stopper but I think one
    > > > >    implication is that we won't be able to do this. Furthermore with
    > > log
    > > > >    cleaning you'd assume that over time ALL messages would collapse
    > > down
    > > > > to a
    > > > >    single wrapper as compaction removes the others.
    > > >
    > > >
    > > > Yeah, that's a fair point. You may still be able to do some merging if
    > > > adjacent message sets have the same PID, but the potential savings
    > might
    > > > not be worth the cost of implementation. My gut feeling is that merging
    > > > message sets from different producers may not be a great idea anyway
    > > (you'd
    > > > have to accept the fact that you always need "deep iteration" to find
    > the
    > > > PIDs contained within the message set), but I haven't thought a ton
    > about
    > > > it. Ultimately we'll have to decide if the potential for savings in the
    > > > future is worth some loss in efficiency now (for what it's worth, I
    > think
    > > > the work that Ben has been looking at also hopes to bundle some more
    > > > information into the message set header).
    > > >
    > > > On a purely pragmatic development level, after spending a ton of recent
    > > > time working with that code, I can say that the benefit of having a
    > > > conceptually simpler message format is huge. It allows you to converge
    > > the
    > > > paths for validation of message sets on the broker, for example.
    > > Currently,
    > > > we pretty much need two separate paths everywhere we process messages.
    > It
    > > > can be tricky just to tell if the message you're dealing with is the
    > > inner
    > > > or outer message, and whether it matters or not. Also, the fact that
    > the
    > > > inner and outer messages share common fields makes bugs like KAFKA-4298
    > > > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk
    > of
    > > > these bugs is much smaller when you can clearly separate the fields
    > > allowed
    > > > in the message set header and those in the messages.
    > > >
    > > >
    > > > Thanks,
    > > > Jason
    > > >
    > > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
    > > >
    > > > > Looks great!
    > > > >
    > > > > A few questions:
    > > > >
    > > > >    1. What is the relationship between transaction.app.id and the
    > > > existing
    > > > >    config application.id in streams?
    > > > >    2. The initTransactions() call is a little annoying. Can we get
    > rid
    > > of
    > > > >    that and call it automatically if you set a transaction.app.id
    > when
    > > > we
    > > > >    do the first message send as we do with metadata? Arguably we
    > should
    > > > > have
    > > > >    included a general connect() or init() call in the producer, but
    > > given
    > > > > that
    > > > >    we didn't do this it seems weird that the cluster metadata
    > > initializes
    > > > >    automatically on demand and the transaction metadata doesn't.
    > > > >    3. The equivalent concept of what we call "fetch.mode" in
    > databases
    > > is
    > > > >    called "isolation level" and takes values like "serializable",
    > "read
    > > > >    committed", "read uncommitted". Since we went with transaction as
    > > the
    > > > > name
    > > > >    for the thing in between the begin/commit might make sense to use
    > > this
    > > > >    terminology for the concept and levels? I think the behavior we
    > are
    > > > >    planning is "read committed" and the alternative re-ordering
    > > behavior
    > > > is
    > > > >    equivalent to "serializable"?
    > > > >    4. Can the PID be made 4 bytes if we handle roll-over gracefully?
    > 2
    > > > >    billion concurrent producers should be enough for anyone, right?
    > > > >    5. One implication of factoring out the message set seems to be
    > you
    > > > >    can't ever "repack" messages to improve compression beyond what is
    > > > done
    > > > > by
    > > > >    the producer. We'd talked about doing this either by buffering
    > when
    > > > > writing
    > > > >    or during log cleaning. This isn't a show stopper but I think one
    > > > >    implication is that we won't be able to do this. Furthermore with
    > > log
    > > > >    cleaning you'd assume that over time ALL messages would collapse
    > > down
    > > > > to a
    > > > >    single wrapper as compaction removes the others.
    > > > >
    > > > > -Jay
    > > > >
    > > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
    > > > wrote:
    > > > >
    > > > > > Hi all,
    > > > > >
    > > > > > I have just created KIP-98 to enhance Kafka with exactly once
    > > delivery
    > > > > > semantics:
    > > > > >
    > > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
    > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
    > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
    > > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
    > > > > >
    > > > > > This KIP adds a transactional messaging mechanism along with an
    > > > > idempotent
    > > > > > producer implementation to make sure that 1) duplicated messages
    > sent
    > > > > from
    > > > > > the same identified producer can be detected on the broker side,
    > and
    > > > 2) a
    > > > > > group of messages sent within a transaction will atomically be
    > either
    > > > > > reflected and fetchable to consumers or not as a whole.
    > > > > >
    > > > > > The above wiki page provides a high-level view of the proposed
    > > changes
    > > > as
    > > > > > well as summarized guarantees. Initial draft of the detailed
    > > > > implementation
    > > > > > design is described in this Google doc:
    > > > > >
    > > > > > https://docs.google.com/document/d/11Jqy_
    > > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
    > > > > > 0wSw9ra8
    > > > > >
    > > > > >
    > > > > > We would love to hear your comments and suggestions.
    > > > > >
    > > > > > Thanks,
    > > > > >
    > > > > > -- Guozhang
    > > > > >
    > > > >
    > > >
    > >
    > >
    > >
    > > --
    > > -- Guozhang
    > >
    > The information contained in this email is strictly confidential and for
    > the use of the addressee only, unless otherwise indicated. If you are not
    > the intended recipient, please do not read, copy, use or disclose to others
    > this message or any attachment. Please also notify the sender by replying
    > to this email or by telephone (+44(020 7896 0011) and then delete the email
    > and any copies of it. Opinions, conclusion (etc) that do not relate to the
    > official business of this company shall be understood as neither given nor
    > endorsed by it. IG is a trading name of IG Markets Limited (a company
    > registered in England and Wales, company number 04008957) and IG Index
    > Limited (a company registered in England and Wales, company number
    > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
    > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
    > Index Limited (register number 114059) are authorised and regulated by the
    > Financial Conduct Authority.
    >


The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Sriram Subramanian <ra...@confluent.io>.
@Jay

1. I totally agree on the naming. The appid for transactions is really an
instance id. Any recommendation for a name is appreciated. We had thought
of instance id, session id or app id and went with app id.
2. We also discussed about init() method but that could add its own set of
confusion to existing users (should I update my existing usage to call
init()? Why should I have this extra step instead of the constructor doing
it?). Transactions is going to be used by a subset of users (probably
small) and it made sense to add the burden of calling
initTransactions/recoverTransactions to only that subset. We are actually
open to suggestions here in terms of naming as well.

@Jonathan
I am not sure it adds more complexity unless you use them. We have
explicitly named them for transactions and the current usage of the
producer remains unchanged.

@Michael
If you look at our idempotent producer implementation in the kip/design,
this is exactly what we do except that the deduplication happens on the
server. We started with separate KIPs but it made it very confusing to read
since there were interdependencies between the concepts.



On Tue, Dec 6, 2016 at 9:04 AM, Michael Pearce <Mi...@ig.com>
wrote:

> For dealing with exactly once delivery.
>
> As an alternative option has it been considered to have a message uuid in
> the record, that then is deduped on consumption?
>
> Similar to
> https://activemq.apache.org/artemis/docs/1.0.0/duplicate-detection.html
>
> Agreed this does not deal with transaction support.
>
> But should the two concerns be separated anyhow into two kips?
>
> Cheers
> Mike
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <ja...@confluent.io>
> Sent: Tuesday, December 6, 2016 4:47:55 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Guozhang,
>
>
>    1. My point is that it is a bit confusing to have two things called
>    application id that have different meanings, right? Won't the streams
> user
>    end up specifying two different application ids?
>    2. Makes sense. My two complaints are
>       1. At this point we've jumped through quite a lot of hoops to make
>       the producer lazily initialize, seems sad to get rid of that now.
>       2. The call initTransactions doesn't really make sense to the user
>       unless they understand the details of the protocol (which they
> won't). When
>       do i call this? How many times? etc. Maybe two additional
> options would be
>       to just add a general init() call that could cover metadata
> initialization
>       as well as this and potentially future things or continue to do it
> lazily.
>    3. Yeah I get that you need an expiry scheme to limit it to 4 bytes. Is
>    there a mechanism to expire them, and hence shrink it?
>
> -Jay
>
> On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > @Jay
> >
> > 1. Stream's applicationId is shared among all instances for the app, and
> is
> > used as part of the consumer group id, while "app.id" is per producer
> > instance. So a Streams app that has a single "applicationID" config will
> > likely contain multiple producers each with a different appID based on
> > their corresponding taskIDs.
> >
> > 2. Another motivation besides the one pointed out by Jason for making
> sure
> > transaction-involved offsets have been committed before resuming, is that
> > we also want to separate the "app.id" config with the transactional
> > mechanism. More concretely, if a user does specify the "app.id" config
> and
> > without using transaction functions (i.e. initTransactions, beginTxn,
> etc),
> > they can still get idempotency guarantee across multiple sessions of the
> > producer identified by the app.id.
> >
> > 4. We thought about the PID length, note that since we do not expire
> PIDs,
> > we are expecting it to cover all possible producers that we have ever
> seen,
> > and producers without an "app.id" can come and go with different PIDs.
> > That
> > is why we feel 4 billion may not be sufficient.
> >
> >
> >
> > Guozhang
> >
> >
> > On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > Hey Jay,
> > >
> > > Thanks for the questions! Let me take a couple of them.
> > >
> > > 2. The initTransactions() call is a little annoying. Can we get rid of
> > > >    that and call it automatically if you set a transaction.app.id
> when
> > > we
> > > >    do the first message send as we do with metadata? Arguably we
> should
> > > > have
> > > >    included a general connect() or init() call in the producer, but
> > given
> > > > that
> > > >    we didn't do this it seems weird that the cluster metadata
> > initializes
> > > >    automatically on demand and the transaction metadata doesn't.
> > >
> > >
> > > The purpose of this call is to fence off any producer with the same
> AppID
> > > and await the completion of any pending transactions. When it returns,
> > you
> > > know that your producer is safe to resume work. Take the the "consume
> and
> > > produce" use case as an example. We send the offset commits as part of
> > the
> > > producer's transaction (approximating the idea that it is "just another
> > > write to a partition"). When you first initialize the application, you
> > have
> > > to determine when it's safe for the consumer to read those offsets.
> > > Otherwise, you may read stale offsets before a transaction which is
> > rolling
> > > forward is able to write the marker to __consumer_offsets. So we can't
> do
> > > the initialization in send() because that would assume that we had
> > already
> > > read data from the consumer, which we can't do until we've initialized
> > the
> > > producer. Does that make sense?
> > >
> > > (For what it's worth, we're not married to this name or any of the
> > others,
> > > so anyone can feel free to suggest alternatives.)
> > >
> > >
> > > 5. One implication of factoring out the message set seems to be you
> > > >    can't ever "repack" messages to improve compression beyond what is
> > > done
> > > > by
> > > >    the producer. We'd talked about doing this either by buffering
> when
> > > > writing
> > > >    or during log cleaning. This isn't a show stopper but I think one
> > > >    implication is that we won't be able to do this. Furthermore with
> > log
> > > >    cleaning you'd assume that over time ALL messages would collapse
> > down
> > > > to a
> > > >    single wrapper as compaction removes the others.
> > >
> > >
> > > Yeah, that's a fair point. You may still be able to do some merging if
> > > adjacent message sets have the same PID, but the potential savings
> might
> > > not be worth the cost of implementation. My gut feeling is that merging
> > > message sets from different producers may not be a great idea anyway
> > (you'd
> > > have to accept the fact that you always need "deep iteration" to find
> the
> > > PIDs contained within the message set), but I haven't thought a ton
> about
> > > it. Ultimately we'll have to decide if the potential for savings in the
> > > future is worth some loss in efficiency now (for what it's worth, I
> think
> > > the work that Ben has been looking at also hopes to bundle some more
> > > information into the message set header).
> > >
> > > On a purely pragmatic development level, after spending a ton of recent
> > > time working with that code, I can say that the benefit of having a
> > > conceptually simpler message format is huge. It allows you to converge
> > the
> > > paths for validation of message sets on the broker, for example.
> > Currently,
> > > we pretty much need two separate paths everywhere we process messages.
> It
> > > can be tricky just to tell if the message you're dealing with is the
> > inner
> > > or outer message, and whether it matters or not. Also, the fact that
> the
> > > inner and outer messages share common fields makes bugs like KAFKA-4298
> > > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk
> of
> > > these bugs is much smaller when you can clearly separate the fields
> > allowed
> > > in the message set header and those in the messages.
> > >
> > >
> > > Thanks,
> > > Jason
> > >
> > > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Looks great!
> > > >
> > > > A few questions:
> > > >
> > > >    1. What is the relationship between transaction.app.id and the
> > > existing
> > > >    config application.id in streams?
> > > >    2. The initTransactions() call is a little annoying. Can we get
> rid
> > of
> > > >    that and call it automatically if you set a transaction.app.id
> when
> > > we
> > > >    do the first message send as we do with metadata? Arguably we
> should
> > > > have
> > > >    included a general connect() or init() call in the producer, but
> > given
> > > > that
> > > >    we didn't do this it seems weird that the cluster metadata
> > initializes
> > > >    automatically on demand and the transaction metadata doesn't.
> > > >    3. The equivalent concept of what we call "fetch.mode" in
> databases
> > is
> > > >    called "isolation level" and takes values like "serializable",
> "read
> > > >    committed", "read uncommitted". Since we went with transaction as
> > the
> > > > name
> > > >    for the thing in between the begin/commit might make sense to use
> > this
> > > >    terminology for the concept and levels? I think the behavior we
> are
> > > >    planning is "read committed" and the alternative re-ordering
> > behavior
> > > is
> > > >    equivalent to "serializable"?
> > > >    4. Can the PID be made 4 bytes if we handle roll-over gracefully?
> 2
> > > >    billion concurrent producers should be enough for anyone, right?
> > > >    5. One implication of factoring out the message set seems to be
> you
> > > >    can't ever "repack" messages to improve compression beyond what is
> > > done
> > > > by
> > > >    the producer. We'd talked about doing this either by buffering
> when
> > > > writing
> > > >    or during log cleaning. This isn't a show stopper but I think one
> > > >    implication is that we won't be able to do this. Furthermore with
> > log
> > > >    cleaning you'd assume that over time ALL messages would collapse
> > down
> > > > to a
> > > >    single wrapper as compaction removes the others.
> > > >
> > > > -Jay
> > > >
> > > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I have just created KIP-98 to enhance Kafka with exactly once
> > delivery
> > > > > semantics:
> > > > >
> > > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > > >
> > > > > This KIP adds a transactional messaging mechanism along with an
> > > > idempotent
> > > > > producer implementation to make sure that 1) duplicated messages
> sent
> > > > from
> > > > > the same identified producer can be detected on the broker side,
> and
> > > 2) a
> > > > > group of messages sent within a transaction will atomically be
> either
> > > > > reflected and fetchable to consumers or not as a whole.
> > > > >
> > > > > The above wiki page provides a high-level view of the proposed
> > changes
> > > as
> > > > > well as summarized guarantees. Initial draft of the detailed
> > > > implementation
> > > > > design is described in this Google doc:
> > > > >
> > > > > https://docs.google.com/document/d/11Jqy_
> > > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > > 0wSw9ra8
> > > > >
> > > > >
> > > > > We would love to hear your comments and suggestions.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
> The information contained in this email is strictly confidential and for
> the use of the addressee only, unless otherwise indicated. If you are not
> the intended recipient, please do not read, copy, use or disclose to others
> this message or any attachment. Please also notify the sender by replying
> to this email or by telephone (+44(020 7896 0011) and then delete the email
> and any copies of it. Opinions, conclusion (etc) that do not relate to the
> official business of this company shall be understood as neither given nor
> endorsed by it. IG is a trading name of IG Markets Limited (a company
> registered in England and Wales, company number 04008957) and IG Index
> Limited (a company registered in England and Wales, company number
> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> Index Limited (register number 114059) are authorised and regulated by the
> Financial Conduct Authority.
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Michael Pearce <Mi...@ig.com>.
For dealing with exactly once delivery.

As an alternative option has it been considered to have a message uuid in the record, that then is deduped on consumption?

Similar to
https://activemq.apache.org/artemis/docs/1.0.0/duplicate-detection.html

Agreed this does not deal with transaction support.

But should the two concerns be separated anyhow into two kips?

Cheers
Mike


Sent using OWA for iPhone
________________________________________
From: Jay Kreps <ja...@confluent.io>
Sent: Tuesday, December 6, 2016 4:47:55 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Guozhang,


   1. My point is that it is a bit confusing to have two things called
   application id that have different meanings, right? Won't the streams user
   end up specifying two different application ids?
   2. Makes sense. My two complaints are
      1. At this point we've jumped through quite a lot of hoops to make
      the producer lazily initialize, seems sad to get rid of that now.
      2. The call initTransactions doesn't really make sense to the user
      unless they understand the details of the protocol (which they
won't). When
      do i call this? How many times? etc. Maybe two additional
options would be
      to just add a general init() call that could cover metadata
initialization
      as well as this and potentially future things or continue to do it lazily.
   3. Yeah I get that you need an expiry scheme to limit it to 4 bytes. Is
   there a mechanism to expire them, and hence shrink it?

-Jay

On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com> wrote:

> @Jay
>
> 1. Stream's applicationId is shared among all instances for the app, and is
> used as part of the consumer group id, while "app.id" is per producer
> instance. So a Streams app that has a single "applicationID" config will
> likely contain multiple producers each with a different appID based on
> their corresponding taskIDs.
>
> 2. Another motivation besides the one pointed out by Jason for making sure
> transaction-involved offsets have been committed before resuming, is that
> we also want to separate the "app.id" config with the transactional
> mechanism. More concretely, if a user does specify the "app.id" config and
> without using transaction functions (i.e. initTransactions, beginTxn, etc),
> they can still get idempotency guarantee across multiple sessions of the
> producer identified by the app.id.
>
> 4. We thought about the PID length, note that since we do not expire PIDs,
> we are expecting it to cover all possible producers that we have ever seen,
> and producers without an "app.id" can come and go with different PIDs.
> That
> is why we feel 4 billion may not be sufficient.
>
>
>
> Guozhang
>
>
> On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Jay,
> >
> > Thanks for the questions! Let me take a couple of them.
> >
> > 2. The initTransactions() call is a little annoying. Can we get rid of
> > >    that and call it automatically if you set a transaction.app.id when
> > we
> > >    do the first message send as we do with metadata? Arguably we should
> > > have
> > >    included a general connect() or init() call in the producer, but
> given
> > > that
> > >    we didn't do this it seems weird that the cluster metadata
> initializes
> > >    automatically on demand and the transaction metadata doesn't.
> >
> >
> > The purpose of this call is to fence off any producer with the same AppID
> > and await the completion of any pending transactions. When it returns,
> you
> > know that your producer is safe to resume work. Take the the "consume and
> > produce" use case as an example. We send the offset commits as part of
> the
> > producer's transaction (approximating the idea that it is "just another
> > write to a partition"). When you first initialize the application, you
> have
> > to determine when it's safe for the consumer to read those offsets.
> > Otherwise, you may read stale offsets before a transaction which is
> rolling
> > forward is able to write the marker to __consumer_offsets. So we can't do
> > the initialization in send() because that would assume that we had
> already
> > read data from the consumer, which we can't do until we've initialized
> the
> > producer. Does that make sense?
> >
> > (For what it's worth, we're not married to this name or any of the
> others,
> > so anyone can feel free to suggest alternatives.)
> >
> >
> > 5. One implication of factoring out the message set seems to be you
> > >    can't ever "repack" messages to improve compression beyond what is
> > done
> > > by
> > >    the producer. We'd talked about doing this either by buffering when
> > > writing
> > >    or during log cleaning. This isn't a show stopper but I think one
> > >    implication is that we won't be able to do this. Furthermore with
> log
> > >    cleaning you'd assume that over time ALL messages would collapse
> down
> > > to a
> > >    single wrapper as compaction removes the others.
> >
> >
> > Yeah, that's a fair point. You may still be able to do some merging if
> > adjacent message sets have the same PID, but the potential savings might
> > not be worth the cost of implementation. My gut feeling is that merging
> > message sets from different producers may not be a great idea anyway
> (you'd
> > have to accept the fact that you always need "deep iteration" to find the
> > PIDs contained within the message set), but I haven't thought a ton about
> > it. Ultimately we'll have to decide if the potential for savings in the
> > future is worth some loss in efficiency now (for what it's worth, I think
> > the work that Ben has been looking at also hopes to bundle some more
> > information into the message set header).
> >
> > On a purely pragmatic development level, after spending a ton of recent
> > time working with that code, I can say that the benefit of having a
> > conceptually simpler message format is huge. It allows you to converge
> the
> > paths for validation of message sets on the broker, for example.
> Currently,
> > we pretty much need two separate paths everywhere we process messages. It
> > can be tricky just to tell if the message you're dealing with is the
> inner
> > or outer message, and whether it matters or not. Also, the fact that the
> > inner and outer messages share common fields makes bugs like KAFKA-4298
> > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk of
> > these bugs is much smaller when you can clearly separate the fields
> allowed
> > in the message set header and those in the messages.
> >
> >
> > Thanks,
> > Jason
> >
> > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Looks great!
> > >
> > > A few questions:
> > >
> > >    1. What is the relationship between transaction.app.id and the
> > existing
> > >    config application.id in streams?
> > >    2. The initTransactions() call is a little annoying. Can we get rid
> of
> > >    that and call it automatically if you set a transaction.app.id when
> > we
> > >    do the first message send as we do with metadata? Arguably we should
> > > have
> > >    included a general connect() or init() call in the producer, but
> given
> > > that
> > >    we didn't do this it seems weird that the cluster metadata
> initializes
> > >    automatically on demand and the transaction metadata doesn't.
> > >    3. The equivalent concept of what we call "fetch.mode" in databases
> is
> > >    called "isolation level" and takes values like "serializable", "read
> > >    committed", "read uncommitted". Since we went with transaction as
> the
> > > name
> > >    for the thing in between the begin/commit might make sense to use
> this
> > >    terminology for the concept and levels? I think the behavior we are
> > >    planning is "read committed" and the alternative re-ordering
> behavior
> > is
> > >    equivalent to "serializable"?
> > >    4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
> > >    billion concurrent producers should be enough for anyone, right?
> > >    5. One implication of factoring out the message set seems to be you
> > >    can't ever "repack" messages to improve compression beyond what is
> > done
> > > by
> > >    the producer. We'd talked about doing this either by buffering when
> > > writing
> > >    or during log cleaning. This isn't a show stopper but I think one
> > >    implication is that we won't be able to do this. Furthermore with
> log
> > >    cleaning you'd assume that over time ALL messages would collapse
> down
> > > to a
> > >    single wrapper as compaction removes the others.
> > >
> > > -Jay
> > >
> > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>
The information contained in this email is strictly confidential and for the use of the addressee only, unless otherwise indicated. If you are not the intended recipient, please do not read, copy, use or disclose to others this message or any attachment. Please also notify the sender by replying to this email or by telephone (+44(020 7896 0011) and then delete the email and any copies of it. Opinions, conclusion (etc) that do not relate to the official business of this company shall be understood as neither given nor endorsed by it. IG is a trading name of IG Markets Limited (a company registered in England and Wales, company number 04008957) and IG Index Limited (a company registered in England and Wales, company number 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG Index Limited (register number 114059) are authorised and regulated by the Financial Conduct Authority.

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Hey Guozhang,


   1. My point is that it is a bit confusing to have two things called
   application id that have different meanings, right? Won't the streams user
   end up specifying two different application ids?
   2. Makes sense. My two complaints are
      1. At this point we've jumped through quite a lot of hoops to make
      the producer lazily initialize, seems sad to get rid of that now.
      2. The call initTransactions doesn't really make sense to the user
      unless they understand the details of the protocol (which they
won't). When
      do i call this? How many times? etc. Maybe two additional
options would be
      to just add a general init() call that could cover metadata
initialization
      as well as this and potentially future things or continue to do it lazily.
   3. Yeah I get that you need an expiry scheme to limit it to 4 bytes. Is
   there a mechanism to expire them, and hence shrink it?

-Jay

On Fri, Dec 2, 2016 at 12:04 PM, Guozhang Wang <wa...@gmail.com> wrote:

> @Jay
>
> 1. Stream's applicationId is shared among all instances for the app, and is
> used as part of the consumer group id, while "app.id" is per producer
> instance. So a Streams app that has a single "applicationID" config will
> likely contain multiple producers each with a different appID based on
> their corresponding taskIDs.
>
> 2. Another motivation besides the one pointed out by Jason for making sure
> transaction-involved offsets have been committed before resuming, is that
> we also want to separate the "app.id" config with the transactional
> mechanism. More concretely, if a user does specify the "app.id" config and
> without using transaction functions (i.e. initTransactions, beginTxn, etc),
> they can still get idempotency guarantee across multiple sessions of the
> producer identified by the app.id.
>
> 4. We thought about the PID length, note that since we do not expire PIDs,
> we are expecting it to cover all possible producers that we have ever seen,
> and producers without an "app.id" can come and go with different PIDs.
> That
> is why we feel 4 billion may not be sufficient.
>
>
>
> Guozhang
>
>
> On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Jay,
> >
> > Thanks for the questions! Let me take a couple of them.
> >
> > 2. The initTransactions() call is a little annoying. Can we get rid of
> > >    that and call it automatically if you set a transaction.app.id when
> > we
> > >    do the first message send as we do with metadata? Arguably we should
> > > have
> > >    included a general connect() or init() call in the producer, but
> given
> > > that
> > >    we didn't do this it seems weird that the cluster metadata
> initializes
> > >    automatically on demand and the transaction metadata doesn't.
> >
> >
> > The purpose of this call is to fence off any producer with the same AppID
> > and await the completion of any pending transactions. When it returns,
> you
> > know that your producer is safe to resume work. Take the the "consume and
> > produce" use case as an example. We send the offset commits as part of
> the
> > producer's transaction (approximating the idea that it is "just another
> > write to a partition"). When you first initialize the application, you
> have
> > to determine when it's safe for the consumer to read those offsets.
> > Otherwise, you may read stale offsets before a transaction which is
> rolling
> > forward is able to write the marker to __consumer_offsets. So we can't do
> > the initialization in send() because that would assume that we had
> already
> > read data from the consumer, which we can't do until we've initialized
> the
> > producer. Does that make sense?
> >
> > (For what it's worth, we're not married to this name or any of the
> others,
> > so anyone can feel free to suggest alternatives.)
> >
> >
> > 5. One implication of factoring out the message set seems to be you
> > >    can't ever "repack" messages to improve compression beyond what is
> > done
> > > by
> > >    the producer. We'd talked about doing this either by buffering when
> > > writing
> > >    or during log cleaning. This isn't a show stopper but I think one
> > >    implication is that we won't be able to do this. Furthermore with
> log
> > >    cleaning you'd assume that over time ALL messages would collapse
> down
> > > to a
> > >    single wrapper as compaction removes the others.
> >
> >
> > Yeah, that's a fair point. You may still be able to do some merging if
> > adjacent message sets have the same PID, but the potential savings might
> > not be worth the cost of implementation. My gut feeling is that merging
> > message sets from different producers may not be a great idea anyway
> (you'd
> > have to accept the fact that you always need "deep iteration" to find the
> > PIDs contained within the message set), but I haven't thought a ton about
> > it. Ultimately we'll have to decide if the potential for savings in the
> > future is worth some loss in efficiency now (for what it's worth, I think
> > the work that Ben has been looking at also hopes to bundle some more
> > information into the message set header).
> >
> > On a purely pragmatic development level, after spending a ton of recent
> > time working with that code, I can say that the benefit of having a
> > conceptually simpler message format is huge. It allows you to converge
> the
> > paths for validation of message sets on the broker, for example.
> Currently,
> > we pretty much need two separate paths everywhere we process messages. It
> > can be tricky just to tell if the message you're dealing with is the
> inner
> > or outer message, and whether it matters or not. Also, the fact that the
> > inner and outer messages share common fields makes bugs like KAFKA-4298
> > <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk of
> > these bugs is much smaller when you can clearly separate the fields
> allowed
> > in the message set header and those in the messages.
> >
> >
> > Thanks,
> > Jason
> >
> > On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Looks great!
> > >
> > > A few questions:
> > >
> > >    1. What is the relationship between transaction.app.id and the
> > existing
> > >    config application.id in streams?
> > >    2. The initTransactions() call is a little annoying. Can we get rid
> of
> > >    that and call it automatically if you set a transaction.app.id when
> > we
> > >    do the first message send as we do with metadata? Arguably we should
> > > have
> > >    included a general connect() or init() call in the producer, but
> given
> > > that
> > >    we didn't do this it seems weird that the cluster metadata
> initializes
> > >    automatically on demand and the transaction metadata doesn't.
> > >    3. The equivalent concept of what we call "fetch.mode" in databases
> is
> > >    called "isolation level" and takes values like "serializable", "read
> > >    committed", "read uncommitted". Since we went with transaction as
> the
> > > name
> > >    for the thing in between the begin/commit might make sense to use
> this
> > >    terminology for the concept and levels? I think the behavior we are
> > >    planning is "read committed" and the alternative re-ordering
> behavior
> > is
> > >    equivalent to "serializable"?
> > >    4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
> > >    billion concurrent producers should be enough for anyone, right?
> > >    5. One implication of factoring out the message set seems to be you
> > >    can't ever "repack" messages to improve compression beyond what is
> > done
> > > by
> > >    the producer. We'd talked about doing this either by buffering when
> > > writing
> > >    or during log cleaning. This isn't a show stopper but I think one
> > >    implication is that we won't be able to do this. Furthermore with
> log
> > >    cleaning you'd assume that over time ALL messages would collapse
> down
> > > to a
> > >    single wrapper as compaction removes the others.
> > >
> > > -Jay
> > >
> > > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I have just created KIP-98 to enhance Kafka with exactly once
> delivery
> > > > semantics:
> > > >
> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > > >
> > > > This KIP adds a transactional messaging mechanism along with an
> > > idempotent
> > > > producer implementation to make sure that 1) duplicated messages sent
> > > from
> > > > the same identified producer can be detected on the broker side, and
> > 2) a
> > > > group of messages sent within a transaction will atomically be either
> > > > reflected and fetchable to consumers or not as a whole.
> > > >
> > > > The above wiki page provides a high-level view of the proposed
> changes
> > as
> > > > well as summarized guarantees. Initial draft of the detailed
> > > implementation
> > > > design is described in this Google doc:
> > > >
> > > > https://docs.google.com/document/d/11Jqy_
> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > > 0wSw9ra8
> > > >
> > > >
> > > > We would love to hear your comments and suggestions.
> > > >
> > > > Thanks,
> > > >
> > > > -- Guozhang
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Guozhang Wang <wa...@gmail.com>.
@Jay

1. Stream's applicationId is shared among all instances for the app, and is
used as part of the consumer group id, while "app.id" is per producer
instance. So a Streams app that has a single "applicationID" config will
likely contain multiple producers each with a different appID based on
their corresponding taskIDs.

2. Another motivation besides the one pointed out by Jason for making sure
transaction-involved offsets have been committed before resuming, is that
we also want to separate the "app.id" config with the transactional
mechanism. More concretely, if a user does specify the "app.id" config and
without using transaction functions (i.e. initTransactions, beginTxn, etc),
they can still get idempotency guarantee across multiple sessions of the
producer identified by the app.id.

4. We thought about the PID length, note that since we do not expire PIDs,
we are expecting it to cover all possible producers that we have ever seen,
and producers without an "app.id" can come and go with different PIDs. That
is why we feel 4 billion may not be sufficient.



Guozhang


On Thu, Dec 1, 2016 at 11:10 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Jay,
>
> Thanks for the questions! Let me take a couple of them.
>
> 2. The initTransactions() call is a little annoying. Can we get rid of
> >    that and call it automatically if you set a transaction.app.id when
> we
> >    do the first message send as we do with metadata? Arguably we should
> > have
> >    included a general connect() or init() call in the producer, but given
> > that
> >    we didn't do this it seems weird that the cluster metadata initializes
> >    automatically on demand and the transaction metadata doesn't.
>
>
> The purpose of this call is to fence off any producer with the same AppID
> and await the completion of any pending transactions. When it returns, you
> know that your producer is safe to resume work. Take the the "consume and
> produce" use case as an example. We send the offset commits as part of the
> producer's transaction (approximating the idea that it is "just another
> write to a partition"). When you first initialize the application, you have
> to determine when it's safe for the consumer to read those offsets.
> Otherwise, you may read stale offsets before a transaction which is rolling
> forward is able to write the marker to __consumer_offsets. So we can't do
> the initialization in send() because that would assume that we had already
> read data from the consumer, which we can't do until we've initialized the
> producer. Does that make sense?
>
> (For what it's worth, we're not married to this name or any of the others,
> so anyone can feel free to suggest alternatives.)
>
>
> 5. One implication of factoring out the message set seems to be you
> >    can't ever "repack" messages to improve compression beyond what is
> done
> > by
> >    the producer. We'd talked about doing this either by buffering when
> > writing
> >    or during log cleaning. This isn't a show stopper but I think one
> >    implication is that we won't be able to do this. Furthermore with log
> >    cleaning you'd assume that over time ALL messages would collapse down
> > to a
> >    single wrapper as compaction removes the others.
>
>
> Yeah, that's a fair point. You may still be able to do some merging if
> adjacent message sets have the same PID, but the potential savings might
> not be worth the cost of implementation. My gut feeling is that merging
> message sets from different producers may not be a great idea anyway (you'd
> have to accept the fact that you always need "deep iteration" to find the
> PIDs contained within the message set), but I haven't thought a ton about
> it. Ultimately we'll have to decide if the potential for savings in the
> future is worth some loss in efficiency now (for what it's worth, I think
> the work that Ben has been looking at also hopes to bundle some more
> information into the message set header).
>
> On a purely pragmatic development level, after spending a ton of recent
> time working with that code, I can say that the benefit of having a
> conceptually simpler message format is huge. It allows you to converge the
> paths for validation of message sets on the broker, for example. Currently,
> we pretty much need two separate paths everywhere we process messages. It
> can be tricky just to tell if the message you're dealing with is the inner
> or outer message, and whether it matters or not. Also, the fact that the
> inner and outer messages share common fields makes bugs like KAFKA-4298
> <https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk of
> these bugs is much smaller when you can clearly separate the fields allowed
> in the message set header and those in the messages.
>
>
> Thanks,
> Jason
>
> On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Looks great!
> >
> > A few questions:
> >
> >    1. What is the relationship between transaction.app.id and the
> existing
> >    config application.id in streams?
> >    2. The initTransactions() call is a little annoying. Can we get rid of
> >    that and call it automatically if you set a transaction.app.id when
> we
> >    do the first message send as we do with metadata? Arguably we should
> > have
> >    included a general connect() or init() call in the producer, but given
> > that
> >    we didn't do this it seems weird that the cluster metadata initializes
> >    automatically on demand and the transaction metadata doesn't.
> >    3. The equivalent concept of what we call "fetch.mode" in databases is
> >    called "isolation level" and takes values like "serializable", "read
> >    committed", "read uncommitted". Since we went with transaction as the
> > name
> >    for the thing in between the begin/commit might make sense to use this
> >    terminology for the concept and levels? I think the behavior we are
> >    planning is "read committed" and the alternative re-ordering behavior
> is
> >    equivalent to "serializable"?
> >    4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
> >    billion concurrent producers should be enough for anyone, right?
> >    5. One implication of factoring out the message set seems to be you
> >    can't ever "repack" messages to improve compression beyond what is
> done
> > by
> >    the producer. We'd talked about doing this either by buffering when
> > writing
> >    or during log cleaning. This isn't a show stopper but I think one
> >    implication is that we won't be able to do this. Furthermore with log
> >    cleaning you'd assume that over time ALL messages would collapse down
> > to a
> >    single wrapper as compaction removes the others.
> >
> > -Jay
> >
> > On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Hi all,
> > >
> > > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > > semantics:
> > >
> > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > >
> > > This KIP adds a transactional messaging mechanism along with an
> > idempotent
> > > producer implementation to make sure that 1) duplicated messages sent
> > from
> > > the same identified producer can be detected on the broker side, and
> 2) a
> > > group of messages sent within a transaction will atomically be either
> > > reflected and fetchable to consumers or not as a whole.
> > >
> > > The above wiki page provides a high-level view of the proposed changes
> as
> > > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > > design is described in this Google doc:
> > >
> > > https://docs.google.com/document/d/11Jqy_
> GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > 0wSw9ra8
> > >
> > >
> > > We would love to hear your comments and suggestions.
> > >
> > > Thanks,
> > >
> > > -- Guozhang
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Jay,

Thanks for the questions! Let me take a couple of them.

2. The initTransactions() call is a little annoying. Can we get rid of
>    that and call it automatically if you set a transaction.app.id when we
>    do the first message send as we do with metadata? Arguably we should
> have
>    included a general connect() or init() call in the producer, but given
> that
>    we didn't do this it seems weird that the cluster metadata initializes
>    automatically on demand and the transaction metadata doesn't.


The purpose of this call is to fence off any producer with the same AppID
and await the completion of any pending transactions. When it returns, you
know that your producer is safe to resume work. Take the the "consume and
produce" use case as an example. We send the offset commits as part of the
producer's transaction (approximating the idea that it is "just another
write to a partition"). When you first initialize the application, you have
to determine when it's safe for the consumer to read those offsets.
Otherwise, you may read stale offsets before a transaction which is rolling
forward is able to write the marker to __consumer_offsets. So we can't do
the initialization in send() because that would assume that we had already
read data from the consumer, which we can't do until we've initialized the
producer. Does that make sense?

(For what it's worth, we're not married to this name or any of the others,
so anyone can feel free to suggest alternatives.)


5. One implication of factoring out the message set seems to be you
>    can't ever "repack" messages to improve compression beyond what is done
> by
>    the producer. We'd talked about doing this either by buffering when
> writing
>    or during log cleaning. This isn't a show stopper but I think one
>    implication is that we won't be able to do this. Furthermore with log
>    cleaning you'd assume that over time ALL messages would collapse down
> to a
>    single wrapper as compaction removes the others.


Yeah, that's a fair point. You may still be able to do some merging if
adjacent message sets have the same PID, but the potential savings might
not be worth the cost of implementation. My gut feeling is that merging
message sets from different producers may not be a great idea anyway (you'd
have to accept the fact that you always need "deep iteration" to find the
PIDs contained within the message set), but I haven't thought a ton about
it. Ultimately we'll have to decide if the potential for savings in the
future is worth some loss in efficiency now (for what it's worth, I think
the work that Ben has been looking at also hopes to bundle some more
information into the message set header).

On a purely pragmatic development level, after spending a ton of recent
time working with that code, I can say that the benefit of having a
conceptually simpler message format is huge. It allows you to converge the
paths for validation of message sets on the broker, for example. Currently,
we pretty much need two separate paths everywhere we process messages. It
can be tricky just to tell if the message you're dealing with is the inner
or outer message, and whether it matters or not. Also, the fact that the
inner and outer messages share common fields makes bugs like KAFKA-4298
<https://issues.apache.org/jira/browse/KAFKA-4298> possible. The risk of
these bugs is much smaller when you can clearly separate the fields allowed
in the message set header and those in the messages.


Thanks,
Jason

On Thu, Dec 1, 2016 at 8:19 PM, Jay Kreps <ja...@confluent.io> wrote:

> Looks great!
>
> A few questions:
>
>    1. What is the relationship between transaction.app.id and the existing
>    config application.id in streams?
>    2. The initTransactions() call is a little annoying. Can we get rid of
>    that and call it automatically if you set a transaction.app.id when we
>    do the first message send as we do with metadata? Arguably we should
> have
>    included a general connect() or init() call in the producer, but given
> that
>    we didn't do this it seems weird that the cluster metadata initializes
>    automatically on demand and the transaction metadata doesn't.
>    3. The equivalent concept of what we call "fetch.mode" in databases is
>    called "isolation level" and takes values like "serializable", "read
>    committed", "read uncommitted". Since we went with transaction as the
> name
>    for the thing in between the begin/commit might make sense to use this
>    terminology for the concept and levels? I think the behavior we are
>    planning is "read committed" and the alternative re-ordering behavior is
>    equivalent to "serializable"?
>    4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
>    billion concurrent producers should be enough for anyone, right?
>    5. One implication of factoring out the message set seems to be you
>    can't ever "repack" messages to improve compression beyond what is done
> by
>    the producer. We'd talked about doing this either by buffering when
> writing
>    or during log cleaning. This isn't a show stopper but I think one
>    implication is that we won't be able to do this. Furthermore with log
>    cleaning you'd assume that over time ALL messages would collapse down
> to a
>    single wrapper as compaction removes the others.
>
> -Jay
>
> On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Jay Kreps <ja...@confluent.io>.
Looks great!

A few questions:

   1. What is the relationship between transaction.app.id and the existing
   config application.id in streams?
   2. The initTransactions() call is a little annoying. Can we get rid of
   that and call it automatically if you set a transaction.app.id when we
   do the first message send as we do with metadata? Arguably we should have
   included a general connect() or init() call in the producer, but given that
   we didn't do this it seems weird that the cluster metadata initializes
   automatically on demand and the transaction metadata doesn't.
   3. The equivalent concept of what we call "fetch.mode" in databases is
   called "isolation level" and takes values like "serializable", "read
   committed", "read uncommitted". Since we went with transaction as the name
   for the thing in between the begin/commit might make sense to use this
   terminology for the concept and levels? I think the behavior we are
   planning is "read committed" and the alternative re-ordering behavior is
   equivalent to "serializable"?
   4. Can the PID be made 4 bytes if we handle roll-over gracefully? 2
   billion concurrent producers should be enough for anyone, right?
   5. One implication of factoring out the message set seems to be you
   can't ever "repack" messages to improve compression beyond what is done by
   the producer. We'd talked about doing this either by buffering when writing
   or during log cleaning. This isn't a show stopper but I think one
   implication is that we won't be able to do this. Furthermore with log
   cleaning you'd assume that over time ALL messages would collapse down to a
   single wrapper as compaction removes the others.

-Jay

On Wed, Nov 30, 2016 at 2:19 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Apurva Mehta <ap...@confluent.io>.
Hi Ismael,

That is a good suggestion. We did not plan to move the design to a wiki,
but I think it is valuable to move at least the message format and RPC
changes to the wiki. We shall do so once the design is close to final so
that we do not have to edit multiple places as we iterate.

Thanks,
Apurva

On Thu, Dec 1, 2016 at 3:28 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Thanks for submitting this KIP as it includes important improvements to
> Kafka's semantics. I will send a follow-up with more detailed feedback, but
> I have a process question in the meantime: is there a plan to move part or
> all of the Google Doc content to the wiki? At least protocol and message
> format changes should be in the wiki, in my opinion (Google Docs are not as
> discoverable, it's harder to track changes, not hosted on Apache Infra,
> etc.).
>
> Thanks,
> Ismael
>
> On Wed, Nov 30, 2016 at 10:19 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks for submitting this KIP as it includes important improvements to
Kafka's semantics. I will send a follow-up with more detailed feedback, but
I have a process question in the meantime: is there a plan to move part or
all of the Google Doc content to the wiki? At least protocol and message
format changes should be in the wiki, in my opinion (Google Docs are not as
discoverable, it's harder to track changes, not hosted on Apache Infra,
etc.).

Thanks,
Ismael

On Wed, Nov 30, 2016 at 10:19 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi all,
>
> I have just created KIP-98 to enhance Kafka with exactly once delivery
> semantics:
>
> *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
>
> This KIP adds a transactional messaging mechanism along with an idempotent
> producer implementation to make sure that 1) duplicated messages sent from
> the same identified producer can be detected on the broker side, and 2) a
> group of messages sent within a transaction will atomically be either
> reflected and fetchable to consumers or not as a whole.
>
> The above wiki page provides a high-level view of the proposed changes as
> well as summarized guarantees. Initial draft of the detailed implementation
> design is described in this Google doc:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8
>
>
> We would love to hear your comments and suggestions.
>
> Thanks,
>
> -- Guozhang
>