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

2017-02-10 Thread Gwen Shapira
Understood, thank you!

On Fri, Feb 10, 2017 at 2:54 PM, Apurva Mehta  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  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 
>> 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 

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

2017-02-10 Thread Apurva Mehta
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  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 
> 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 

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

2017-02-10 Thread Gwen Shapira
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  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 

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

2017-02-10 Thread Jason Gustafson
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  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 
> wrote:
> > A couple follow-ups from Ismael's previous comments.
> >
> > 1. We have removed the individual message CRC 

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

2017-02-09 Thread Gwen Shapira
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  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  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 
>> 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 
>> 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 
>> > 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
>> 

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

2017-02-09 Thread Gwen Shapira
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.

On Tue, Jan 17, 2017 at 4:54 PM, Apurva Mehta  wrote:
> Hello,
>
> Nacho and others had questions about the rationale behind the AppId and PID
> design. In particular, why do we propose to have a broker-based map of
> AppId to PID? Why can't we use the AppId directly? And if we need to use an
> internal PID, why don't we do a static hashing of AppId to PID on the
> client.
>
> I updated the Rejected Alternatives section of the doc with a discussion of
> the reasoning behind the current proposal. Have a look and let me know if
> there are any further questions:
>
> https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.6uypa62m3vgw
>
> Thanks,
> Apurva
>
>
> On Tue, Jan 10, 2017 at 9:30 AM, Jason Gustafson  wrote:
>
>> Hi All,
>>
>> We've been putting some thought into the need to buffer fetched data in the
>> consumer in the READ_COMMITTED isolation mode and have a proposal to
>> address the concern. The basic idea is to introduce an index to keep track
>> of the aborted transactions. We use this index to return in each fetch a
>> list of the aborted transactions from the fetch range so that the consumer
>> can tell without any buffering whether a record set should be returned to
>> the user. Take a look and let us know what you think:
>> https://docs.google.com/document/d/1Rlqizmk7QCDe8qAnVW5e5X8rGvn6m
>> 2DCR3JR2yqwVjc/edit?usp=sharing
>> .
>>
>> Thanks,
>> Jason
>>
>> On Sun, Jan 8, 2017 at 9:32 PM, Jun Rao  wrote:
>>
>> > Hi, Jason,
>> >
>> > 100. Yes, AppId level security is mainly for protecting the shared
>> > transaction log. We could also include AppId in produce request (not in
>> > message format) so that we could protect writes at the AppId level. I
>> agree
>> > that we need to support prefix matching on AppId for applications like
>> > stream to use this conveniently.
>> >
>> > A couple of other comments.
>> >
>> > 122. Earlier, Becket asked for the use case of knowing the number of
>> > messages in a message set. One potential use case is KAFKA-4293.
>> 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.
>> >
>> > 123. I am wondering if the coordinator needs to add a "BEGIN transaction
>> > message" on a BeginTxnRequest
>> > > > 0wSw9ra8/edit#heading=h.lbrw4crdnl5>.
>> > Could we just wait until an AddPartitionsToTxnRequest
>> > > > 0wSw9ra8/edit#heading=h.r6klddrx9ibz>
>> > ?
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> >
>> > On Thu, Jan 5, 2017 at 11:05 AM, Jason Gustafson 
>> > wrote:
>> >
>> > > 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 

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

2017-02-09 Thread Gwen Shapira
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?

If I missed more edge-cases, feel free to clarify those too :)

Gwen

On Tue, Jan 10, 2017 at 9:30 AM, Jason Gustafson  wrote:
> Hi All,
>
> We've been putting some thought into the need to buffer fetched data in the
> consumer in the READ_COMMITTED isolation mode and have a proposal to
> address the concern. The basic idea is to introduce an index to keep track
> of the aborted transactions. We use this index to return in each fetch a
> list of the aborted transactions from the fetch range so that the consumer
> can tell without any buffering whether a record set should be returned to
> the user. Take a look and let us know what you think:
> https://docs.google.com/document/d/1Rlqizmk7QCDe8qAnVW5e5X8rGvn6m2DCR3JR2yqwVjc/edit?usp=sharing
> .
>
> Thanks,
> Jason
>
> On Sun, Jan 8, 2017 at 9:32 PM, Jun Rao  wrote:
>
>> Hi, Jason,
>>
>> 100. Yes, AppId level security is mainly for protecting the shared
>> transaction log. We could also include AppId in produce request (not in
>> message format) so that we could protect writes at the AppId level. I agree
>> that we need to support prefix matching on AppId for applications like
>> stream to use this conveniently.
>>
>> A couple of other comments.
>>
>> 122. Earlier, Becket asked for the use case of knowing the number of
>> messages in a message set. One potential use case is KAFKA-4293. 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.
>>
>> 123. I am wondering if the coordinator needs to add a "BEGIN transaction
>> message" on a BeginTxnRequest
>> > 0wSw9ra8/edit#heading=h.lbrw4crdnl5>.
>> Could we just wait until an AddPartitionsToTxnRequest
>> > 0wSw9ra8/edit#heading=h.r6klddrx9ibz>
>> ?
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Thu, Jan 5, 2017 at 11:05 AM, Jason Gustafson 
>> wrote:
>>
>> > 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 

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

2017-02-07 Thread Jason Gustafson
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  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 
> 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 
> 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 
> > 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 
> > > > 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.
> > > > >
> > > > > 

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

2017-02-06 Thread Apurva Mehta
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  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  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 
> 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 
> > > 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 
> > > > 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 <
> > > rajinisiva...@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.
> > 

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

2017-02-06 Thread Jason Gustafson
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  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  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 
> > 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 
> > > 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 <
> > rajinisiva...@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  >
> > > >> 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 

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

2017-02-06 Thread Apurva Mehta
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  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 
> 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 
> > 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 <
> rajinisiva...@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 
> > >> 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.
> > >> >

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

2017-02-06 Thread Tom Crayford
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  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 
> 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 
> > 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 
> >> 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 

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

2017-02-03 Thread Jason Gustafson
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 
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 
> 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  wrote:
> >
> > > Comments inline.
> > >
> > > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson 
> > > 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
> > > 

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

2017-02-03 Thread Rajini Sivaram
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  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  wrote:
>
> > Comments inline.
> >
> > On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson 
> > 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 

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

2017-02-03 Thread Jason Gustafson
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  wrote:

> Comments inline.
>
> On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson 
> 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

2017-02-03 Thread Ismael Juma
Comments inline.

On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson  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

2017-02-03 Thread Rajini Sivaram
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  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"  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 
> 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 
> wrote:
> >
> >> Good 

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

2017-02-02 Thread Ismael Juma
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"  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  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  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  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  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 

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

2017-02-02 Thread Jason Gustafson
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  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  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  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  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
>> >> 

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

2017-02-02 Thread Jason Gustafson
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  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  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  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 
> >> 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 

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

2017-02-02 Thread Apurva Mehta
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  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  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 
>> 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 
>> wrote:
>>> 
 Ismael, thanks for your feedbacks. Replied inline.
 
> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma  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 

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

2017-02-02 Thread Tom Crayford
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  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 
> 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 
> wrote:
> >
> > > Ismael, thanks for your feedbacks. Replied inline.
> > >
> > > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma  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 

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

2017-02-02 Thread Ismael Juma
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  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  wrote:
>
> > Ismael, thanks for your feedbacks. Replied inline.
> >
> > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma  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 

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

2017-02-01 Thread Jason Gustafson
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  wrote:

> Ismael, thanks for your feedbacks. Replied inline.
>
> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma  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 
> > wrote:
> >
> > > Hello Folks,
> > >
> > > We have addressed all the comments collected so far, and 

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

2017-02-01 Thread Guozhang Wang
Ismael, thanks for your feedbacks. Replied inline.

On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma  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 
> 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 
> > wrote:
> >
> > > +1 for transactional.id.
> > >
> > > -Jason
> > >
> > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang 
> > 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 
> > > > 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
> > > > > 

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

2017-02-01 Thread Ismael Juma
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  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 
> wrote:
>
> > +1 for transactional.id.
> >
> > -Jason
> >
> > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang 
> 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 
> > > 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

2017-01-30 Thread Eugen Dueck

> 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年01月31日 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

2017-01-30 Thread Guozhang Wang
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  wrote:

> +1 for transactional.id.
>
> -Jason
>
> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang  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 
> > 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

2017-01-30 Thread Jason Gustafson
+1 for transactional.id.

-Jason

On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang  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 
> 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

2017-01-30 Thread Guozhang Wang
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  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

2017-01-30 Thread Apurva Mehta
> 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

2017-01-30 Thread Apurva Mehta
>
> 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

2017-01-30 Thread Jason Gustafson
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  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

2017-01-30 Thread Apurva Mehta
>   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

2017-01-29 Thread Eugen Dueck

Thanks Apurva - replies inline.

On 2017年01月27日 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  wrote:




On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta  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 

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

2017-01-29 Thread Guozhang Wang
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
.
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

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
,
here

and here

.


*> 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

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

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  wrote:

> On Tue, Jan 17, 2017 at 

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

2017-01-29 Thread Jun Rao
Hi, Jason,

Thanks for the reply. They sound good to me.

Jun

On Fri, Jan 27, 2017 at 4:42 PM, Jason Gustafson  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  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 

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

2017-01-29 Thread Jun Rao
Hi, Apurva,

Thanks for the update. My replies are inlined below.

On Wed, Jan 25, 2017 at 5:15 PM, Apurva Mehta  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 

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

2017-01-27 Thread Jason Gustafson
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  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 

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

2017-01-27 Thread Apurva Mehta
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

2017-01-26 Thread Apurva Mehta
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  wrote:

>
>
> On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta  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

2017-01-25 Thread Apurva Mehta
On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta  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

2017-01-25 Thread Apurva Mehta
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 

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

2017-01-19 Thread Jason Gustafson
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  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  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  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 

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

2017-01-17 Thread Apurva Mehta
>
> 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  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  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?
>>

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

2017-01-17 Thread Apurva Mehta
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  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 

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

2017-01-17 Thread Apurva Mehta
Hello,

Nacho and others had questions about the rationale behind the AppId and PID
design. In particular, why do we propose to have a broker-based map of
AppId to PID? Why can't we use the AppId directly? And if we need to use an
internal PID, why don't we do a static hashing of AppId to PID on the
client.

I updated the Rejected Alternatives section of the doc with a discussion of
the reasoning behind the current proposal. Have a look and let me know if
there are any further questions:

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

Thanks,
Apurva


On Tue, Jan 10, 2017 at 9:30 AM, Jason Gustafson  wrote:

> Hi All,
>
> We've been putting some thought into the need to buffer fetched data in the
> consumer in the READ_COMMITTED isolation mode and have a proposal to
> address the concern. The basic idea is to introduce an index to keep track
> of the aborted transactions. We use this index to return in each fetch a
> list of the aborted transactions from the fetch range so that the consumer
> can tell without any buffering whether a record set should be returned to
> the user. Take a look and let us know what you think:
> https://docs.google.com/document/d/1Rlqizmk7QCDe8qAnVW5e5X8rGvn6m
> 2DCR3JR2yqwVjc/edit?usp=sharing
> .
>
> Thanks,
> Jason
>
> On Sun, Jan 8, 2017 at 9:32 PM, Jun Rao  wrote:
>
> > Hi, Jason,
> >
> > 100. Yes, AppId level security is mainly for protecting the shared
> > transaction log. We could also include AppId in produce request (not in
> > message format) so that we could protect writes at the AppId level. I
> agree
> > that we need to support prefix matching on AppId for applications like
> > stream to use this conveniently.
> >
> > A couple of other comments.
> >
> > 122. Earlier, Becket asked for the use case of knowing the number of
> > messages in a message set. One potential use case is KAFKA-4293.
> 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.
> >
> > 123. I am wondering if the coordinator needs to add a "BEGIN transaction
> > message" on a BeginTxnRequest
> >  > 0wSw9ra8/edit#heading=h.lbrw4crdnl5>.
> > Could we just wait until an AddPartitionsToTxnRequest
> >  > 0wSw9ra8/edit#heading=h.r6klddrx9ibz>
> > ?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Jan 5, 2017 at 11:05 AM, Jason Gustafson 
> > wrote:
> >
> > > 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 

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

2017-01-10 Thread Jason Gustafson
Hi All,

We've been putting some thought into the need to buffer fetched data in the
consumer in the READ_COMMITTED isolation mode and have a proposal to
address the concern. The basic idea is to introduce an index to keep track
of the aborted transactions. We use this index to return in each fetch a
list of the aborted transactions from the fetch range so that the consumer
can tell without any buffering whether a record set should be returned to
the user. Take a look and let us know what you think:
https://docs.google.com/document/d/1Rlqizmk7QCDe8qAnVW5e5X8rGvn6m2DCR3JR2yqwVjc/edit?usp=sharing
.

Thanks,
Jason

On Sun, Jan 8, 2017 at 9:32 PM, Jun Rao  wrote:

> Hi, Jason,
>
> 100. Yes, AppId level security is mainly for protecting the shared
> transaction log. We could also include AppId in produce request (not in
> message format) so that we could protect writes at the AppId level. I agree
> that we need to support prefix matching on AppId for applications like
> stream to use this conveniently.
>
> A couple of other comments.
>
> 122. Earlier, Becket asked for the use case of knowing the number of
> messages in a message set. One potential use case is KAFKA-4293. 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.
>
> 123. I am wondering if the coordinator needs to add a "BEGIN transaction
> message" on a BeginTxnRequest
>  0wSw9ra8/edit#heading=h.lbrw4crdnl5>.
> Could we just wait until an AddPartitionsToTxnRequest
>  0wSw9ra8/edit#heading=h.r6klddrx9ibz>
> ?
>
> Thanks,
>
> Jun
>
>
> On Thu, Jan 5, 2017 at 11:05 AM, Jason Gustafson 
> wrote:
>
> > 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  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
> > > 

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

2017-01-08 Thread Jun Rao
Hi, Jason,

100. Yes, AppId level security is mainly for protecting the shared
transaction log. We could also include AppId in produce request (not in
message format) so that we could protect writes at the AppId level. I agree
that we need to support prefix matching on AppId for applications like
stream to use this conveniently.

A couple of other comments.

122. Earlier, Becket asked for the use case of knowing the number of
messages in a message set. One potential use case is KAFKA-4293. 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.

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

?

Thanks,

Jun


On Thu, Jan 5, 2017 at 11:05 AM, Jason Gustafson  wrote:

> 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  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
> > 

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

2017-01-05 Thread Apurva Mehta
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  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
>  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

2017-01-05 Thread Jason Gustafson
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  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  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 

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

2017-01-04 Thread Jay Kreps
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  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 
> 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 
> 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

2017-01-04 Thread Jun Rao
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 

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

2017-01-03 Thread radai
@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  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 
> 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 
> 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

2017-01-03 Thread Jun Rao
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  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  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

2016-12-21 Thread Apurva Mehta
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  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

2016-12-21 Thread Joel Koshy
>
>
> @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

2016-12-20 Thread radai
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  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

2016-12-20 Thread Apurva Mehta
@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  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
>

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

2016-12-20 Thread Jay Kreps
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  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  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 

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

2016-12-20 Thread radai
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  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  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
> >  > 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 

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

2016-12-20 Thread Jay Kreps
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  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
>  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 
> 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 

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

2016-12-20 Thread Joel Koshy
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

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  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  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 

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

2016-12-19 Thread radai
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  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  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
> 

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

2016-12-19 Thread Guozhang Wang
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  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  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 
> 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 

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

2016-12-19 Thread Sriram Subramanian
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 
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  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 

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

2016-12-19 Thread Sriram Subramanian
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  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 

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

2016-12-19 Thread Jay Kreps
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  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  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 

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

2016-12-19 Thread radai
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  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 

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

2016-12-18 Thread Becket Qin
@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  wrote:

> Yeah good point. I relent!
>
> -jay
>
> On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson 
> 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 

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

2016-12-16 Thread Jay Kreps
Yeah good point. I relent!

-jay

On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson  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 
> 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  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 
>
> > 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  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 

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

2016-12-16 Thread Jason Gustafson
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  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  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 
> 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  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 

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

2016-12-16 Thread radai
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  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 
> 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 

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

2016-12-16 Thread radai
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  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
>  0wSw9ra8/edit#bookmark=id.3s6zgsaq9cep>.
> In particular, the recommended sequence is to bump all 

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

2016-12-16 Thread Apurva Mehta
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
.
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  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  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 
> > wrote:
> >
> >> Hi Apurva,
> >>
> >> Thank you, makes sense.
> >>
> >> Rajini
> >>
> >> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta 
> >> wrote:
> >>
> >> > Hi Rajini,
> >> >
> >> > I think my original 

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

2016-12-15 Thread radai
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  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 
> wrote:
>
>> Hi Apurva,
>>
>> Thank you, makes sense.
>>
>> Rajini
>>
>> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta 
>> 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 
>> > 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 

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

2016-12-15 Thread radai
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  wrote:

> Hi Apurva,
>
> Thank you, makes sense.
>
> Rajini
>
> On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta  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 
> > 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  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 

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

2016-12-15 Thread Rajini Sivaram
Hi Apurva,

Thank you, makes sense.

Rajini

On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta  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 
> 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  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 
> > 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  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 

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

2016-12-14 Thread Apurva Mehta
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  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  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 
> 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  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

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

2016-12-14 Thread Apurva Mehta
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  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  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  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 

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

2016-12-14 Thread Rajini Sivaram
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  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  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  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 
> > 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
> > > >  > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging>*
> > 

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

2016-12-13 Thread Ben Kirwin
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  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  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 
> 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
> > >  > > 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

2016-12-13 Thread Jay Kreps
 <wangg...@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_schofi...@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 <wangg...@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

2016-12-13 Thread Sriram Subramanian
gt; > > > 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 <wangg...@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_schofi...@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 <wangg...@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

2016-12-12 Thread Ismael Juma
lient.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 <wangg...@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
> >

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

2016-12-12 Thread Jay Kreps
nd 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 <wangg...@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_schofi...@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 <wangg...@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

2016-12-12 Thread Jason Gustafson
ime 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_schofi...@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 <wangg...@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

2016-12-12 Thread Jason Gustafson
@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 <wangg...@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_schofi...@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 <wangg...@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

2016-12-12 Thread Guozhang Wang
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_schofi...@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 <wangg...@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

2016-12-12 Thread Apurva Mehta
gt;
> >> > 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.apa...@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.pea...@ig.com
> >> > >
> >> > > > wrote:
> >> > > >
> >> > > >> Apologies on the spelling.
> >> > > >>
> >> > > >> *Hi Jay,
> >> > > >> 
> >> > > >> From: Michael Pearce <michael.pea...@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 <j...@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
> >> > > >>
> >> &

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

2016-12-12 Thread Andrew Schofield
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 <wangg...@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

2016-12-12 Thread Rajini Sivaram
t; 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.apa...@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.apa...@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.pea...@ig.com
>> > >
>> > > > wrote:
>> > > >
>> > > >> Apologies on the spelling.
>> > > >>
>> > > >> *Hi Jay,
>> > > >> 
>> > > >> From: Michael Pearce <michael.pea...@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 <j...@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,

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

2016-12-12 Thread Rajini Sivaram
lications 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 <wangg...@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.apa...@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.apa...@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

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

2016-12-11 Thread Becket Qin
; lost
> > after-the-fact because of kafka's durability guarantees.
> >
> > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> > onurkaraman.apa...@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.pea...@ig.com
> >
> > > wrote:
> > >
> > >> Apologies on the spelling.
> > >>
> > >> *Hi Jay,
> > >> 
> > >> From: Michael Pearce <michael.pea...@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 <j...@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 

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

2016-12-11 Thread Neha Narkhede
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 <wangg...@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.apa...@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.apa...@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.pea...@ig.com>
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >> 
> >> From: Michael Pearce <michael.pea...@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

2016-12-09 Thread Guozhang Wang
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.apa...@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.apa...@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.pea...@ig.com>
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >> ____
> >> From: Michael Pearce <michael.pea...@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 <j...@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
> >> tran

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

2016-12-09 Thread Onur Karaman
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.apa...@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.pea...@ig.com>
> wrote:
>
>> Apologies on the spelling.
>>
>> *Hi Jay,
>> 
>> From: Michael Pearce <michael.pea...@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 <j...@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.pea...@ig.com>
>> wrote:
>>
>> > Hi Jay,
>> >
>> > 

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

2016-12-09 Thread Onur Karaman
@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.pea...@ig.com>
wrote:

> Apologies on the spelling.
>
> *Hi Jay,
> 
> From: Michael Pearce <michael.pea...@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 <j...@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.pea...@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 s

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

2016-12-09 Thread Michael Pearce
Apologies on the spelling.

*Hi Jay,

From: Michael Pearce <michael.pea...@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 <j...@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.pea...@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 <j...@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 a

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

2016-12-09 Thread Michael Pearce
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 <j...@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.pea...@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 <j...@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.pea...@ig.com>
> wrote:
>
> > Usecase in IG:
> >
> > Fund trans

  1   2   >