You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jun Rao <ju...@confluent.io> on 2017/01/03 15:50:24 UTC

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

Just to follow up on Radai's idea of pushing the buffering logic to the
broker. It may be possible to do this efficiently if we assume aborted
transactions are rare. The following is a draft proposal. For each
partition, the broker maintains the last stable offset (LSO) as described
in the document, and only exposes messages up to this point if the reader
is in the read-committed mode. When a new stable offset (NSO) is
determined, if there is no aborted message in this window, the broker
simply advances the LSO to the NSO. If there is at least one aborted
message, the broker first replaces the current log segment with new log
segments excluding the aborted messages and then advances the LSO. To make
the replacement efficient, we can replace the current log segment with 3
new segments: (1) a new "shadow" log segment that simply references the
portion of the current log segment from the beginning to the LSO, (2) a log
segment created by copying only committed messages between the LSO and the
NSO, (3) a new "shadow" log segment that references the portion of the
current log segment from the NSO (open ended). Note that only (2) involves
real data copying. If aborted transactions are rare, this overhead will be
insignificant. Assuming that applications typically don't abort
transactions, transactions will only be aborted by transaction coordinators
during hard failure of the producers, which should be rare.

This way, the consumer library's logic will be simplified. We can still
expose uncommitted messages to readers in the read-uncommitted mode and
therefore leave the door open for speculative reader in the future.

Thanks,

Jun


On Wed, Dec 21, 2016 at 10:44 AM, Apurva Mehta <ap...@confluent.io> wrote:

> Hi Joel,
>
> The alternatives are embedded in the 'discussion' sections which are spread
> throughout the google doc.
>
> Admittedly, we have not covered high level alternatives like those which
> have been brought up in this thread. In particular, having a separate log
> for transactional mesages and also having multiple producers participate in
> a single transaction.
>
> This is an omission which we will correct.
>
> Thanks,
> Apurva
>
> On Wed, Dec 21, 2016 at 10:34 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > >
> > >
> > > @Joel,
> > >
> > > I read over your wiki, and apart from the introduction of the notion of
> > > journal partitions --whose pros and cons are already being discussed--
> > you
> > > also introduce the notion of a 'producer group' which enables multiple
> > > producers to participate in a single transaction. This is completely
> > > opposite of the model in the KIP where a transaction is defined by a
> > > producer id, and hence there is a 1-1 mapping between producers and
> > > transactions. Further, each producer can have exactly one in-flight
> > > transaction at a time in the KIP.
> > >
> >
> > Hi Apurva - yes I did notice those differences among other things :)
> BTW, I
> > haven't yet gone through the google-doc carefully but on a skim it does
> not
> > seem to contain any rejected alternatives as the wiki states.
> >
>

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

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

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

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


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

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

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

-Jason

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

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

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

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

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

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

-Jay

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

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

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

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

Thanks for the reply. They sound good to me.

Jun

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

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

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

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


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


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

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

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

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


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

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


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

Thanks,
Jason

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

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

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

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

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



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

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

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

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


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

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

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

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



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

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

Thanks,
Apurva

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

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

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

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

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

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

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

Cheers
Eugen

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

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

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


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

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


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

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

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

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

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

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

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

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

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

Cheers,
Eugen

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

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

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

Thanks for the comments!! Some responses below:

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

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


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

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


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

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

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


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

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


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

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


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

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


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

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


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

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


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

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


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

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



Guozhang


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

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



-- 
-- Guozhang

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

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

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

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

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

Hope this clarified things.

Regards,
Apurva


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





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


Cheers

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

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

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

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

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

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

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

Thanks,
Apurva

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

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

Thanks,
Jason

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

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

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

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

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


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

Thanks,
Apurva

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

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

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

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

Some answers in line.


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

MessageException?


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

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

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

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


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


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

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


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


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

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

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

Posted by Rajini Sivaram <ra...@gmail.com>.
I have a few questions on security (sorry, only just catching up on the
updates).

1. Will the transaction coordinator check topic ACLs based on the
requesting client's credentials? Access to transaction logs, topics being
added for transaction etc?
2. If I create a transactional produce request (by hand, not using the
producer API) with a random PID (random, hence unlikely to be in use), will
the broker append a transactional message to the logs, preventing LSO from
moving forward? What validation will broker do for PIDs?
3. Will every broker check that a client sending transactional produce
requests at least has write access to transaction log topic since it is not
validating transactional.id (for every produce request)?
4. I understand that brokers cannot authorize the transactional id for each
produce request since requests contain only the PID. But since there is a
one-to-one mapping between PID and transactional.id, and a connection is
never expected to change its transactional.id, perhaps it is feasible to
add authorization and cache the results in the Session? Perhaps not for
version 1, but feels like it will be good to close the security gap here.
Obviously it would be simpler if transactional.id was in the produce
request if the overhead was acceptable.

Thank you,

Rajini


On Thu, Feb 2, 2017 at 8:37 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Yes, I'd also prefer the option where we only have a checksum at the
> message set level. I didn't suggest it due to the mentioned auditing use
> cases, but if they can be satisfied in some other way, then that would be
> great.
>
> Ismael
>
> On 2 Feb 2017 7:03 pm, "Jason Gustafson" <ja...@confluent.io> wrote:
>
> One more:
>
> 1. I did some benchmarking and CRC32C seems to be a massive win when using
> > the hardware instruction (particularly for messages larger than 65k), so
> > I'm keen on taking advantage of the message format version bump to add
> > support for it. I can write a separate KIP for this as it's not tied to
> > Exactly-once, but it would be good to include the code change in the same
> > PR that bumps the message format version. The benchmark and results can
> be
> > found in the following link:
> > https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>
>
> Yeah, makes sense. We can add this to this KIP or do it separately,
> whichever you prefer. I have also been very interested in removing the
> individual message CRCs. The main reason we haven't done so is because some
> auditing applications depend on them, but there are cases where it's
> already unsafe to depend on the message CRCs not changing on the broker
> (message conversion and the use of log append time can both result in new
> message-level crcs). So I'm wondering a bit about the use cases that
> require the message CRCs and how they handle this. Perhaps if it is not
> dependable anyway, we can remove it and safe some space and computation.
>
> -Jason
>
>
> On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Ismael,
> >
> > 2. The message timestamp field is 8 bytes. Did we consider storing the
> >> first timestamp in the message set and then storing deltas using varints
> >> in
> >> the messages like we do for offsets (the difference would be the usage
> of
> >> signed varints)? It seems like the deltas would be quite a bit smaller
> in
> >> the common case (potentially 0 for log append time, so we could even not
> >> store them at all using attributes like we do for key/value lengths). An
> >> alternative is using MaxTimestamp that is already present in the message
> >> set and computing deltas from that, but that seems more complicated. In
> >> any
> >> case, details aside, was this idea considered and rejected or is it
> worth
> >> exploring further?
> >
> >
> > Took me a while to remember why we didn't do this. The timestamp that is
> > included at the message set level is the max timestamp of all messages in
> > the message set as is the case in the current message format (I will
> update
> > the document to make this explicit). We could make the message timestamps
> > relative to the max timestamp, but that makes serialization a bit awkward
> > since the timestamps are not assumed to be increasing sequentially or
> > monotonically. Once the messages in the message set had been determined,
> we
> > would need to go back and adjust the relative timestamps.
> >
> > Here's one idea. We let the timestamps in the messages be varints, but we
> > make their values be relative to the timestamp of the previous message,
> > with the timestamp of the first message being absolute. For example, if
> we
> > had timestamps 500, 501, 499, then we would write 500 for the first
> > message, 1 for the next, and -2 for the final message. Would that work?
> Let
> > me think a bit about it and see if there are any problems.
> >
> > -Jason
> >
> > On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
> >
> >> Good point Tom. We will update the KIP with the ACLs section and also
> the
> >> message format changes.
> >>
> >> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> >> >
> >> > I said this in the voting thread, but can the authors include a
> section
> >> > about new ACLs if there are going to be ACLs for TransactionalId. It's
> >> > mentioned in the google doc, but I think new ACLs should be in a KIP
> >> > directly.
> >> >
> >> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk>
> wrote:
> >> >>
> >> >> Thanks for the responses and updates to the document, Guozhang and
> >> Jason.
> >> >> They look good. One follow-up and one additional comment:
> >> >>
> >> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
> >> using
> >> >> the hardware instruction (particularly for messages larger than 65k),
> >> so
> >> >> I'm keen on taking advantage of the message format version bump to
> add
> >> >> support for it. I can write a separate KIP for this as it's not tied
> to
> >> >> Exactly-once, but it would be good to include the code change in the
> >> same
> >> >> PR that bumps the message format version. The benchmark and results
> >> can be
> >> >> found in the following link:
> >> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
> >> >>
> >> >> 2. The message timestamp field is 8 bytes. Did we consider storing
> the
> >> >> first timestamp in the message set and then storing deltas using
> >> varints in
> >> >> the messages like we do for offsets (the difference would be the
> usage
> >> of
> >> >> signed varints)? It seems like the deltas would be quite a bit
> smaller
> >> in
> >> >> the common case (potentially 0 for log append time, so we could even
> >> not
> >> >> store them at all using attributes like we do for key/value lengths).
> >> An
> >> >> alternative is using MaxTimestamp that is already present in the
> >> message
> >> >> set and computing deltas from that, but that seems more complicated.
> >> In any
> >> >> case, details aside, was this idea considered and rejected or is it
> >> worth
> >> >> exploring further?
> >> >>
> >> >> Ismael
> >> >>
> >> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> >> >> wrote:
> >> >>
> >> >>> Ismael,
> >> >>>
> >> >>> Thanks for the comments. A few responses below:
> >> >>>
> >> >>>
> >> >>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >> introduces
> >> >>> a
> >> >>>> compatibility issue with regards to existing third-party
> authorizers.
> >> >> It
> >> >>>> would be good to highlight this in the migration/compatibility
> >> section.
> >> >>>
> >> >>>
> >> >>> Ack. I added a note in the migration section.
> >> >>>
> >> >>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>> considered
> >> >>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>> 0.10.2.0)?
> >> >>>
> >> >>>
> >> >>> Thanks, I added a few notes about client compatibility to the
> >> migration
> >> >>> section. I covered the main issues that come to mind, but let me
> know
> >> if
> >> >>> you think of others.
> >> >>>
> >> >>> 7. It seems like there is a bit of inconsistency when it comes to
> >> naming
> >> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >> conventions.
> >> >>>> There are a few other examples like that and it would be good to
> >> clean
> >> >>> them
> >> >>>> up.
> >> >>>
> >> >>>
> >> >>> Let's go with InitPidRequest for consistency.  Haha,
> "InitPIdRequest"
> >> >> seems
> >> >>> like a compromise which satisfies no one.
> >> >>>
> >> >>>
> >> >>> -Jason
> >> >>>
> >> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> >> >> wrote:
> >> >>>
> >> >>>> Ismael, thanks for your feedbacks. Replied inline.
> >> >>>>
> >> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
> >> wrote:
> >> >>>>>
> >> >>>>> Hi all,
> >> >>>>>
> >> >>>>> A few comments follow:
> >> >>>>>
> >> >>>>> 1. The document states "inter-broker communications will be
> >> increased
> >> >>> by
> >> >>>> M
> >> >>>>> * N * P round trips per sec. We need to conduct some system
> >> >> performance
> >> >>>>> test to make sure this additional inter-broker traffic would not
> >> >>> largely
> >> >>>>> impact the broker cluster". Has this testing been done? And if
> not,
> >> >> are
> >> >>>> we
> >> >>>>> planning to do it soon? It seems important to validate this sooner
> >> >>> rather
> >> >>>>> than later. This applies more generally too, it would be great to
> >> >>>>> understand how the new message format affects the producer with
> >> small
> >> >>>>> messages, for example.
> >> >>>>>
> >> >>>>>
> >> >>>> Yes we are conducting the perf tests with the message format
> changes
> >> in
> >> >>> the
> >> >>>> first stage; then the inter-broker communication with minimal
> >> >> transaction
> >> >>>> coordinator implementations in the second stage.
> >> >>>>
> >> >>>>
> >> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >>> introduces
> >> >>>> a
> >> >>>>> compatibility issue with regards to existing third-party
> >> authorizers.
> >> >>> It
> >> >>>>> would be good to highlight this in the migration/compatibility
> >> >> section.
> >> >>>>>
> >> >>>>> 3. I was happy to see that default values for the new configs have
> >> >> been
> >> >>>>> added to the document since I last checked it. It would be good to
> >> >>>> explain
> >> >>>>> the motivation for the choices.
> >> >>>>>
> >> >>>>>
> >> >>>> Updated doc.
> >> >>>>
> >> >>>>
> >> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>>> considered
> >> >>>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>>> 0.10.2.0)?
> >> >>>>>
> >> >>>>> 5. transactional.id sounds good
> >> >>>>>
> >> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have
> we
> >> >>>>> considered mitigating the performance cost by using the more
> >> >> performant
> >> >>>>> CRC32c in the new message format version?
> >> >>>>>
> >> >>>>>
> >> >>>> We have not discussed about this before. But I think it should be
> >> >> doable
> >> >>> as
> >> >>>> long as we can include the additional conversion logic in the
> >> migration
> >> >>>> plan.
> >> >>>>
> >> >>>>
> >> >>>>> Nits:
> >> >>>>>
> >> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
> >> >>> naming
> >> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >>> conventions.
> >> >>>>> There are a few other examples like that and it would be good to
> >> >> clean
> >> >>>> them
> >> >>>>> up.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree with the inconsistency issue. About the name itself though,
> >> >>> should
> >> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
> >> since
> >> >>> we
> >> >>>> need to capitalize "I" right?
> >> >>>>
> >> >>>>
> >> >>>>> 8. The document states "The first four fields of a message set in
> >> >> this
> >> >>>>> format must to be the same as the existing format because any
> fields
> >> >>>> before
> >> >>>>> the magic byte cannot be changed in order to provide a path for
> >> >>> upgrades
> >> >>>>> following a similar approach as was used in KIP-32". This makes
> >> >> things
> >> >>>>> easier, but it seems to me that the only strict requirement is
> that
> >> >> the
> >> >>>>> magic byte remains in the same offset and with the same size.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree theoretically it is not required, but I think in practice
> it
> >> is
> >> >>>> actually better to make it more restrict: the three fields before
> >> magic
> >> >>>> byte are offset, length, and crc. Among them, crc needs to be
> before
> >> >>> magic
> >> >>>> byte if it wants to cover the magic byte fields; length would
> better
> >> be
> >> >>>> before the magic byte as well for pre-allocate memory to
> >> >> deser/decompress
> >> >>>> the message set, and the only field that does not matter too much
> to
> >> be
> >> >>>> after magic byte is offset, but in KIP-98 we will use it as the
> base
> >> >>> offset
> >> >>>> for message set and some validation checks can be optimized to not
> >> scan
> >> >>>> through the whole message with this field in front of the format.
> >> >>>>
> >> >>>>
> >> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> >>>>> wrote:
> >> >>>>>
> >> >>>>>> Hello Folks,
> >> >>>>>>
> >> >>>>>> We have addressed all the comments collected so far, and would
> like
> >> >>> to
> >> >>>>>> propose a voting thread this Wednesday. If you have any further
> >> >>>> comments
> >> >>>>> on
> >> >>>>>> this KIP, please feel free to continue sending them on this
> thread
> >> >>>> before
> >> >>>>>> that.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> Guozhang
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> >> >> jason@confluent.io
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>> +1 for transactional.id.
> >> >>>>>>>
> >> >>>>>>> -Jason
> >> >>>>>>>
> >> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> >> >> wangguoz@gmail.com
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>>> If I have to choose between app.id and
> >> >> transactional.instance.id
> >> >>> ,
> >> >>>>> I'd
> >> >>>>>>>> choose the latter.
> >> >>>>>>>>
> >> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
> >> >>> even
> >> >>>>>>> better.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> Guozhang
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> >> >>>> apurva@confluent.io>
> >> >>>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
> >> >>> has
> >> >>>>>> caused
> >> >>>>>>>>> some
> >> >>>>>>>>>> confusion. We're considering the following renaming:
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> >> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
> >> >>>>>>>> transactional.id
> >> >>>>>>>>> )
> >> >>>>>>>>>
> >> >>>>>>>>> This makes it clear that this id just needs to be set when
> >> >> the
> >> >>>>>>>> application
> >> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
> >> >>> in
> >> >>>>> the
> >> >>>>>>>>> context of how this id is used, viz. to maintain transactions
> >> >>>>> across
> >> >>>>>>>>> producer sessions.
> >> >>>>>>>>>
> >> >>>>>>>>> Thanks,
> >> >>>>>>>>> Apurva
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> --
> >> >>>>>>>> -- Guozhang
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> --
> >> >>>>>> -- Guozhang
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> --
> >> >>>> -- Guozhang
> >> >>>>
> >> >>>
> >> >>
> >>
> >
> >
>

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

Posted by Ismael Juma <is...@juma.me.uk>.
Yes, I'd also prefer the option where we only have a checksum at the
message set level. I didn't suggest it due to the mentioned auditing use
cases, but if they can be satisfied in some other way, then that would be
great.

Ismael

On 2 Feb 2017 7:03 pm, "Jason Gustafson" <ja...@confluent.io> wrote:

One more:

1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.


Yeah, makes sense. We can add this to this KIP or do it separately,
whichever you prefer. I have also been very interested in removing the
individual message CRCs. The main reason we haven't done so is because some
auditing applications depend on them, but there are cases where it's
already unsafe to depend on the message CRCs not changing on the broker
(message conversion and the use of log append time can both result in new
message-level crcs). So I'm wondering a bit about the use cases that
require the message CRCs and how they handle this. Perhaps if it is not
dependable anyway, we can remove it and safe some space and computation.

-Jason


On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Ismael,
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints
>> in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In
>> any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>
>
> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will
update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined,
we
> would need to go back and adjust the relative timestamps.
>
> Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work?
Let
> me think a bit about it and see if there are any problems.
>
> -Jason
>
> On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Good point Tom. We will update the KIP with the ACLs section and also the
>> message format changes.
>>
>> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
>> >
>> > I said this in the voting thread, but can the authors include a section
>> > about new ACLs if there are going to be ACLs for TransactionalId. It's
>> > mentioned in the google doc, but I think new ACLs should be in a KIP
>> > directly.
>> >
>> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> >>
>> >> Thanks for the responses and updates to the document, Guozhang and
>> Jason.
>> >> They look good. One follow-up and one additional comment:
>> >>
>> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
>> using
>> >> the hardware instruction (particularly for messages larger than 65k),
>> so
>> >> I'm keen on taking advantage of the message format version bump to add
>> >> support for it. I can write a separate KIP for this as it's not tied
to
>> >> Exactly-once, but it would be good to include the code change in the
>> same
>> >> PR that bumps the message format version. The benchmark and results
>> can be
>> >> found in the following link:
>> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> >>
>> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> >> first timestamp in the message set and then storing deltas using
>> varints in
>> >> the messages like we do for offsets (the difference would be the usage
>> of
>> >> signed varints)? It seems like the deltas would be quite a bit smaller
>> in
>> >> the common case (potentially 0 for log append time, so we could even
>> not
>> >> store them at all using attributes like we do for key/value lengths).
>> An
>> >> alternative is using MaxTimestamp that is already present in the
>> message
>> >> set and computing deltas from that, but that seems more complicated.
>> In any
>> >> case, details aside, was this idea considered and rejected or is it
>> worth
>> >> exploring further?
>> >>
>> >> Ismael
>> >>
>> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >>
>> >>> Ismael,
>> >>>
>> >>> Thanks for the comments. A few responses below:
>> >>>
>> >>>
>> >>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >> introduces
>> >>> a
>> >>>> compatibility issue with regards to existing third-party
authorizers.
>> >> It
>> >>>> would be good to highlight this in the migration/compatibility
>> section.
>> >>>
>> >>>
>> >>> Ack. I added a note in the migration section.
>> >>>
>> >>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>> considered
>> >>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>> 0.10.2.0)?
>> >>>
>> >>>
>> >>> Thanks, I added a few notes about client compatibility to the
>> migration
>> >>> section. I covered the main issues that come to mind, but let me know
>> if
>> >>> you think of others.
>> >>>
>> >>> 7. It seems like there is a bit of inconsistency when it comes to
>> naming
>> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >> conventions.
>> >>>> There are a few other examples like that and it would be good to
>> clean
>> >>> them
>> >>>> up.
>> >>>
>> >>>
>> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> >> seems
>> >>> like a compromise which satisfies no one.
>> >>>
>> >>>
>> >>> -Jason
>> >>>
>> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>
>> >>>> Ismael, thanks for your feedbacks. Replied inline.
>> >>>>
>> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
>> wrote:
>> >>>>>
>> >>>>> Hi all,
>> >>>>>
>> >>>>> A few comments follow:
>> >>>>>
>> >>>>> 1. The document states "inter-broker communications will be
>> increased
>> >>> by
>> >>>> M
>> >>>>> * N * P round trips per sec. We need to conduct some system
>> >> performance
>> >>>>> test to make sure this additional inter-broker traffic would not
>> >>> largely
>> >>>>> impact the broker cluster". Has this testing been done? And if not,
>> >> are
>> >>>> we
>> >>>>> planning to do it soon? It seems important to validate this sooner
>> >>> rather
>> >>>>> than later. This applies more generally too, it would be great to
>> >>>>> understand how the new message format affects the producer with
>> small
>> >>>>> messages, for example.
>> >>>>>
>> >>>>>
>> >>>> Yes we are conducting the perf tests with the message format changes
>> in
>> >>> the
>> >>>> first stage; then the inter-broker communication with minimal
>> >> transaction
>> >>>> coordinator implementations in the second stage.
>> >>>>
>> >>>>
>> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >>> introduces
>> >>>> a
>> >>>>> compatibility issue with regards to existing third-party
>> authorizers.
>> >>> It
>> >>>>> would be good to highlight this in the migration/compatibility
>> >> section.
>> >>>>>
>> >>>>> 3. I was happy to see that default values for the new configs have
>> >> been
>> >>>>> added to the document since I last checked it. It would be good to
>> >>>> explain
>> >>>>> the motivation for the choices.
>> >>>>>
>> >>>>>
>> >>>> Updated doc.
>> >>>>
>> >>>>
>> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>>> considered
>> >>>>> if there's any additional work required due to KIP-97 (introduced
in
>> >>>>> 0.10.2.0)?
>> >>>>>
>> >>>>> 5. transactional.id sounds good
>> >>>>>
>> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>> >>>>> considered mitigating the performance cost by using the more
>> >> performant
>> >>>>> CRC32c in the new message format version?
>> >>>>>
>> >>>>>
>> >>>> We have not discussed about this before. But I think it should be
>> >> doable
>> >>> as
>> >>>> long as we can include the additional conversion logic in the
>> migration
>> >>>> plan.
>> >>>>
>> >>>>
>> >>>>> Nits:
>> >>>>>
>> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
>> >>> naming
>> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >>> conventions.
>> >>>>> There are a few other examples like that and it would be good to
>> >> clean
>> >>>> them
>> >>>>> up.
>> >>>>>
>> >>>>>
>> >>>> I agree with the inconsistency issue. About the name itself though,
>> >>> should
>> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
>> since
>> >>> we
>> >>>> need to capitalize "I" right?
>> >>>>
>> >>>>
>> >>>>> 8. The document states "The first four fields of a message set in
>> >> this
>> >>>>> format must to be the same as the existing format because any
fields
>> >>>> before
>> >>>>> the magic byte cannot be changed in order to provide a path for
>> >>> upgrades
>> >>>>> following a similar approach as was used in KIP-32". This makes
>> >> things
>> >>>>> easier, but it seems to me that the only strict requirement is that
>> >> the
>> >>>>> magic byte remains in the same offset and with the same size.
>> >>>>>
>> >>>>>
>> >>>> I agree theoretically it is not required, but I think in practice it
>> is
>> >>>> actually better to make it more restrict: the three fields before
>> magic
>> >>>> byte are offset, length, and crc. Among them, crc needs to be before
>> >>> magic
>> >>>> byte if it wants to cover the magic byte fields; length would better
>> be
>> >>>> before the magic byte as well for pre-allocate memory to
>> >> deser/decompress
>> >>>> the message set, and the only field that does not matter too much to
>> be
>> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
>> >>> offset
>> >>>> for message set and some validation checks can be optimized to not
>> scan
>> >>>> through the whole message with this field in front of the format.
>> >>>>
>> >>>>
>> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Hello Folks,
>> >>>>>>
>> >>>>>> We have addressed all the comments collected so far, and would
like
>> >>> to
>> >>>>>> propose a voting thread this Wednesday. If you have any further
>> >>>> comments
>> >>>>> on
>> >>>>>> this KIP, please feel free to continue sending them on this thread
>> >>>> before
>> >>>>>> that.
>> >>>>>>
>> >>>>>>
>> >>>>>> Guozhang
>> >>>>>>
>> >>>>>>
>> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> >> jason@confluent.io
>> >>>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>> +1 for transactional.id.
>> >>>>>>>
>> >>>>>>> -Jason
>> >>>>>>>
>> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> >> wangguoz@gmail.com
>> >>>>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> If I have to choose between app.id and
>> >> transactional.instance.id
>> >>> ,
>> >>>>> I'd
>> >>>>>>>> choose the latter.
>> >>>>>>>>
>> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>> >>> even
>> >>>>>>> better.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>> >>>> apurva@confluent.io>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>> >>> has
>> >>>>>> caused
>> >>>>>>>>> some
>> >>>>>>>>>> confusion. We're considering the following renaming:
>> >>>>>>>>>>
>> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>> >>>>>>>> transactional.id
>> >>>>>>>>> )
>> >>>>>>>>>
>> >>>>>>>>> This makes it clear that this id just needs to be set when
>> >> the
>> >>>>>>>> application
>> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
>> >>> in
>> >>>>> the
>> >>>>>>>>> context of how this id is used, viz. to maintain transactions
>> >>>>> across
>> >>>>>>>>> producer sessions.
>> >>>>>>>>>
>> >>>>>>>>> Thanks,
>> >>>>>>>>> Apurva
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> --
>> >>>>>>>> -- Guozhang
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>>> -- Guozhang
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> --
>> >>>> -- Guozhang
>> >>>>
>> >>>
>> >>
>>
>
>

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

Posted by Jason Gustafson <ja...@confluent.io>.
One more:

1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.


Yeah, makes sense. We can add this to this KIP or do it separately,
whichever you prefer. I have also been very interested in removing the
individual message CRCs. The main reason we haven't done so is because some
auditing applications depend on them, but there are cases where it's
already unsafe to depend on the message CRCs not changing on the broker
(message conversion and the use of log append time can both result in new
message-level crcs). So I'm wondering a bit about the use cases that
require the message CRCs and how they handle this. Perhaps if it is not
dependable anyway, we can remove it and safe some space and computation.

-Jason


On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Hey Ismael,
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints
>> in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In
>> any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>
>
> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined, we
> would need to go back and adjust the relative timestamps.
>
> Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work? Let
> me think a bit about it and see if there are any problems.
>
> -Jason
>
> On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:
>
>> Good point Tom. We will update the KIP with the ACLs section and also the
>> message format changes.
>>
>> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
>> >
>> > I said this in the voting thread, but can the authors include a section
>> > about new ACLs if there are going to be ACLs for TransactionalId. It's
>> > mentioned in the google doc, but I think new ACLs should be in a KIP
>> > directly.
>> >
>> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> >>
>> >> Thanks for the responses and updates to the document, Guozhang and
>> Jason.
>> >> They look good. One follow-up and one additional comment:
>> >>
>> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
>> using
>> >> the hardware instruction (particularly for messages larger than 65k),
>> so
>> >> I'm keen on taking advantage of the message format version bump to add
>> >> support for it. I can write a separate KIP for this as it's not tied to
>> >> Exactly-once, but it would be good to include the code change in the
>> same
>> >> PR that bumps the message format version. The benchmark and results
>> can be
>> >> found in the following link:
>> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> >>
>> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> >> first timestamp in the message set and then storing deltas using
>> varints in
>> >> the messages like we do for offsets (the difference would be the usage
>> of
>> >> signed varints)? It seems like the deltas would be quite a bit smaller
>> in
>> >> the common case (potentially 0 for log append time, so we could even
>> not
>> >> store them at all using attributes like we do for key/value lengths).
>> An
>> >> alternative is using MaxTimestamp that is already present in the
>> message
>> >> set and computing deltas from that, but that seems more complicated.
>> In any
>> >> case, details aside, was this idea considered and rejected or is it
>> worth
>> >> exploring further?
>> >>
>> >> Ismael
>> >>
>> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> >> wrote:
>> >>
>> >>> Ismael,
>> >>>
>> >>> Thanks for the comments. A few responses below:
>> >>>
>> >>>
>> >>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >> introduces
>> >>> a
>> >>>> compatibility issue with regards to existing third-party authorizers.
>> >> It
>> >>>> would be good to highlight this in the migration/compatibility
>> section.
>> >>>
>> >>>
>> >>> Ack. I added a note in the migration section.
>> >>>
>> >>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>> considered
>> >>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>> 0.10.2.0)?
>> >>>
>> >>>
>> >>> Thanks, I added a few notes about client compatibility to the
>> migration
>> >>> section. I covered the main issues that come to mind, but let me know
>> if
>> >>> you think of others.
>> >>>
>> >>> 7. It seems like there is a bit of inconsistency when it comes to
>> naming
>> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >> conventions.
>> >>>> There are a few other examples like that and it would be good to
>> clean
>> >>> them
>> >>>> up.
>> >>>
>> >>>
>> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> >> seems
>> >>> like a compromise which satisfies no one.
>> >>>
>> >>>
>> >>> -Jason
>> >>>
>> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>
>> >>>> Ismael, thanks for your feedbacks. Replied inline.
>> >>>>
>> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
>> wrote:
>> >>>>>
>> >>>>> Hi all,
>> >>>>>
>> >>>>> A few comments follow:
>> >>>>>
>> >>>>> 1. The document states "inter-broker communications will be
>> increased
>> >>> by
>> >>>> M
>> >>>>> * N * P round trips per sec. We need to conduct some system
>> >> performance
>> >>>>> test to make sure this additional inter-broker traffic would not
>> >>> largely
>> >>>>> impact the broker cluster". Has this testing been done? And if not,
>> >> are
>> >>>> we
>> >>>>> planning to do it soon? It seems important to validate this sooner
>> >>> rather
>> >>>>> than later. This applies more generally too, it would be great to
>> >>>>> understand how the new message format affects the producer with
>> small
>> >>>>> messages, for example.
>> >>>>>
>> >>>>>
>> >>>> Yes we are conducting the perf tests with the message format changes
>> in
>> >>> the
>> >>>> first stage; then the inter-broker communication with minimal
>> >> transaction
>> >>>> coordinator implementations in the second stage.
>> >>>>
>> >>>>
>> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> >>> introduces
>> >>>> a
>> >>>>> compatibility issue with regards to existing third-party
>> authorizers.
>> >>> It
>> >>>>> would be good to highlight this in the migration/compatibility
>> >> section.
>> >>>>>
>> >>>>> 3. I was happy to see that default values for the new configs have
>> >> been
>> >>>>> added to the document since I last checked it. It would be good to
>> >>>> explain
>> >>>>> the motivation for the choices.
>> >>>>>
>> >>>>>
>> >>>> Updated doc.
>> >>>>
>> >>>>
>> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
>> >>>> considered
>> >>>>> if there's any additional work required due to KIP-97 (introduced in
>> >>>>> 0.10.2.0)?
>> >>>>>
>> >>>>> 5. transactional.id sounds good
>> >>>>>
>> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>> >>>>> considered mitigating the performance cost by using the more
>> >> performant
>> >>>>> CRC32c in the new message format version?
>> >>>>>
>> >>>>>
>> >>>> We have not discussed about this before. But I think it should be
>> >> doable
>> >>> as
>> >>>> long as we can include the additional conversion logic in the
>> migration
>> >>>> plan.
>> >>>>
>> >>>>
>> >>>>> Nits:
>> >>>>>
>> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
>> >>> naming
>> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> >>> conventions.
>> >>>>> There are a few other examples like that and it would be good to
>> >> clean
>> >>>> them
>> >>>>> up.
>> >>>>>
>> >>>>>
>> >>>> I agree with the inconsistency issue. About the name itself though,
>> >>> should
>> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
>> since
>> >>> we
>> >>>> need to capitalize "I" right?
>> >>>>
>> >>>>
>> >>>>> 8. The document states "The first four fields of a message set in
>> >> this
>> >>>>> format must to be the same as the existing format because any fields
>> >>>> before
>> >>>>> the magic byte cannot be changed in order to provide a path for
>> >>> upgrades
>> >>>>> following a similar approach as was used in KIP-32". This makes
>> >> things
>> >>>>> easier, but it seems to me that the only strict requirement is that
>> >> the
>> >>>>> magic byte remains in the same offset and with the same size.
>> >>>>>
>> >>>>>
>> >>>> I agree theoretically it is not required, but I think in practice it
>> is
>> >>>> actually better to make it more restrict: the three fields before
>> magic
>> >>>> byte are offset, length, and crc. Among them, crc needs to be before
>> >>> magic
>> >>>> byte if it wants to cover the magic byte fields; length would better
>> be
>> >>>> before the magic byte as well for pre-allocate memory to
>> >> deser/decompress
>> >>>> the message set, and the only field that does not matter too much to
>> be
>> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
>> >>> offset
>> >>>> for message set and some validation checks can be optimized to not
>> scan
>> >>>> through the whole message with this field in front of the format.
>> >>>>
>> >>>>
>> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Hello Folks,
>> >>>>>>
>> >>>>>> We have addressed all the comments collected so far, and would like
>> >>> to
>> >>>>>> propose a voting thread this Wednesday. If you have any further
>> >>>> comments
>> >>>>> on
>> >>>>>> this KIP, please feel free to continue sending them on this thread
>> >>>> before
>> >>>>>> that.
>> >>>>>>
>> >>>>>>
>> >>>>>> Guozhang
>> >>>>>>
>> >>>>>>
>> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> >> jason@confluent.io
>> >>>>
>> >>>>>> wrote:
>> >>>>>>
>> >>>>>>> +1 for transactional.id.
>> >>>>>>>
>> >>>>>>> -Jason
>> >>>>>>>
>> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> >> wangguoz@gmail.com
>> >>>>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> If I have to choose between app.id and
>> >> transactional.instance.id
>> >>> ,
>> >>>>> I'd
>> >>>>>>>> choose the latter.
>> >>>>>>>>
>> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>> >>> even
>> >>>>>>> better.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Guozhang
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>> >>>> apurva@confluent.io>
>> >>>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>> >>> has
>> >>>>>> caused
>> >>>>>>>>> some
>> >>>>>>>>>> confusion. We're considering the following renaming:
>> >>>>>>>>>>
>> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>> >>>>>>>> transactional.id
>> >>>>>>>>> )
>> >>>>>>>>>
>> >>>>>>>>> This makes it clear that this id just needs to be set when
>> >> the
>> >>>>>>>> application
>> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
>> >>> in
>> >>>>> the
>> >>>>>>>>> context of how this id is used, viz. to maintain transactions
>> >>>>> across
>> >>>>>>>>> producer sessions.
>> >>>>>>>>>
>> >>>>>>>>> Thanks,
>> >>>>>>>>> Apurva
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> --
>> >>>>>>>> -- Guozhang
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> --
>> >>>>>> -- Guozhang
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> --
>> >>>> -- Guozhang
>> >>>>
>> >>>
>> >>
>>
>
>

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

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

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



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

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

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

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

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

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

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

Thanks,
Apurva



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

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

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

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

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

Gwen




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



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

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

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

Thanks for the questions and comments. Responses below:

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

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


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

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

0, 1, 2
N, T, N

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

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

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

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

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

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

0, 1, 2
T1, T2, T1

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

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

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

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

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

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

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

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

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


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

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


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

Thanks,
Jason

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

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

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

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

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

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



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

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

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

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

Thanks,
Jason

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

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

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

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

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

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

Thanks,
Apurva

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

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

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

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

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

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

-Jason

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

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

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

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

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

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

Thanks for pointing that out.

Regards,
Apurva





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

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

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

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

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

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

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

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

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

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

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

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

-Jason

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

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

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

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

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


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

Thanks,
Jason

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

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

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

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

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

Regards,

Rajini


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

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

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

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

Thanks for the questions. Responses below:


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


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

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


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

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


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

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


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

Thanks,
Jason


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

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

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

Posted by Ismael Juma <is...@juma.me.uk>.
Comments inline.

On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson <ja...@confluent.io> wrote:

> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will update
> the document to make this explicit). We could make the message timestamps
> relative to the max timestamp, but that makes serialization a bit awkward
> since the timestamps are not assumed to be increasing sequentially or
> monotonically. Once the messages in the message set had been determined, we
> would need to go back and adjust the relative timestamps.
>

Yes, I thought this would be a bit tricky and hence why I mentioned the
option of adding a new field at the message set level for the first
timestamp even though that's not ideal either.

Here's one idea. We let the timestamps in the messages be varints, but we
> make their values be relative to the timestamp of the previous message,
> with the timestamp of the first message being absolute. For example, if we
> had timestamps 500, 501, 499, then we would write 500 for the first
> message, 1 for the next, and -2 for the final message. Would that work? Let
> me think a bit about it and see if there are any problems.
>

It's an interesting idea. Comparing to the option of having the first
timestamp in the message set, It's a little more space efficient as we
don't have both a full timestamp in the message set _and_ a varint in the
first message (which would always be 0, so we avoid the extra byte) and
also the deltas could be a little smaller in the common case. The main
downside is that it introduces a semantics inconsistency between the first
message and the rest. Not ideal, but maybe we can live with that.

Ismael

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

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

2. The message timestamp field is 8 bytes. Did we consider storing the
> first timestamp in the message set and then storing deltas using varints in
> the messages like we do for offsets (the difference would be the usage of
> signed varints)? It seems like the deltas would be quite a bit smaller in
> the common case (potentially 0 for log append time, so we could even not
> store them at all using attributes like we do for key/value lengths). An
> alternative is using MaxTimestamp that is already present in the message
> set and computing deltas from that, but that seems more complicated. In any
> case, details aside, was this idea considered and rejected or is it worth
> exploring further?


Took me a while to remember why we didn't do this. The timestamp that is
included at the message set level is the max timestamp of all messages in
the message set as is the case in the current message format (I will update
the document to make this explicit). We could make the message timestamps
relative to the max timestamp, but that makes serialization a bit awkward
since the timestamps are not assumed to be increasing sequentially or
monotonically. Once the messages in the message set had been determined, we
would need to go back and adjust the relative timestamps.

Here's one idea. We let the timestamps in the messages be varints, but we
make their values be relative to the timestamp of the previous message,
with the timestamp of the first message being absolute. For example, if we
had timestamps 500, 501, 499, then we would write 500 for the first
message, 1 for the next, and -2 for the final message. Would that work? Let
me think a bit about it and see if there are any problems.

-Jason

On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <ap...@confluent.io> wrote:

> Good point Tom. We will update the KIP with the ACLs section and also the
> message format changes.
>
> > On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> >
> > I said this in the voting thread, but can the authors include a section
> > about new ACLs if there are going to be ACLs for TransactionalId. It's
> > mentioned in the google doc, but I think new ACLs should be in a KIP
> > directly.
> >
> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
> >>
> >> Thanks for the responses and updates to the document, Guozhang and
> Jason.
> >> They look good. One follow-up and one additional comment:
> >>
> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
> using
> >> the hardware instruction (particularly for messages larger than 65k), so
> >> I'm keen on taking advantage of the message format version bump to add
> >> support for it. I can write a separate KIP for this as it's not tied to
> >> Exactly-once, but it would be good to include the code change in the
> same
> >> PR that bumps the message format version. The benchmark and results can
> be
> >> found in the following link:
> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
> >>
> >> 2. The message timestamp field is 8 bytes. Did we consider storing the
> >> first timestamp in the message set and then storing deltas using
> varints in
> >> the messages like we do for offsets (the difference would be the usage
> of
> >> signed varints)? It seems like the deltas would be quite a bit smaller
> in
> >> the common case (potentially 0 for log append time, so we could even not
> >> store them at all using attributes like we do for key/value lengths). An
> >> alternative is using MaxTimestamp that is already present in the message
> >> set and computing deltas from that, but that seems more complicated. In
> any
> >> case, details aside, was this idea considered and rejected or is it
> worth
> >> exploring further?
> >>
> >> Ismael
> >>
> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> >> wrote:
> >>
> >>> Ismael,
> >>>
> >>> Thanks for the comments. A few responses below:
> >>>
> >>>
> >>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> introduces
> >>> a
> >>>> compatibility issue with regards to existing third-party authorizers.
> >> It
> >>>> would be good to highlight this in the migration/compatibility
> section.
> >>>
> >>>
> >>> Ack. I added a note in the migration section.
> >>>
> >>> 4. The Migration plan is relatively brief at the moment. Have we
> >>> considered
> >>>> if there's any additional work required due to KIP-97 (introduced in
> >>>> 0.10.2.0)?
> >>>
> >>>
> >>> Thanks, I added a few notes about client compatibility to the migration
> >>> section. I covered the main issues that come to mind, but let me know
> if
> >>> you think of others.
> >>>
> >>> 7. It seems like there is a bit of inconsistency when it comes to
> naming
> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> conventions.
> >>>> There are a few other examples like that and it would be good to clean
> >>> them
> >>>> up.
> >>>
> >>>
> >>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
> >> seems
> >>> like a compromise which satisfies no one.
> >>>
> >>>
> >>> -Jason
> >>>
> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> Ismael, thanks for your feedbacks. Replied inline.
> >>>>
> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk>
> wrote:
> >>>>>
> >>>>> Hi all,
> >>>>>
> >>>>> A few comments follow:
> >>>>>
> >>>>> 1. The document states "inter-broker communications will be increased
> >>> by
> >>>> M
> >>>>> * N * P round trips per sec. We need to conduct some system
> >> performance
> >>>>> test to make sure this additional inter-broker traffic would not
> >>> largely
> >>>>> impact the broker cluster". Has this testing been done? And if not,
> >> are
> >>>> we
> >>>>> planning to do it soon? It seems important to validate this sooner
> >>> rather
> >>>>> than later. This applies more generally too, it would be great to
> >>>>> understand how the new message format affects the producer with small
> >>>>> messages, for example.
> >>>>>
> >>>>>
> >>>> Yes we are conducting the perf tests with the message format changes
> in
> >>> the
> >>>> first stage; then the inter-broker communication with minimal
> >> transaction
> >>>> coordinator implementations in the second stage.
> >>>>
> >>>>
> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
> >>> introduces
> >>>> a
> >>>>> compatibility issue with regards to existing third-party authorizers.
> >>> It
> >>>>> would be good to highlight this in the migration/compatibility
> >> section.
> >>>>>
> >>>>> 3. I was happy to see that default values for the new configs have
> >> been
> >>>>> added to the document since I last checked it. It would be good to
> >>>> explain
> >>>>> the motivation for the choices.
> >>>>>
> >>>>>
> >>>> Updated doc.
> >>>>
> >>>>
> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
> >>>> considered
> >>>>> if there's any additional work required due to KIP-97 (introduced in
> >>>>> 0.10.2.0)?
> >>>>>
> >>>>> 5. transactional.id sounds good
> >>>>>
> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
> >>>>> considered mitigating the performance cost by using the more
> >> performant
> >>>>> CRC32c in the new message format version?
> >>>>>
> >>>>>
> >>>> We have not discussed about this before. But I think it should be
> >> doable
> >>> as
> >>>> long as we can include the additional conversion logic in the
> migration
> >>>> plan.
> >>>>
> >>>>
> >>>>> Nits:
> >>>>>
> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
> >>> naming
> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >>> conventions.
> >>>>> There are a few other examples like that and it would be good to
> >> clean
> >>>> them
> >>>>> up.
> >>>>>
> >>>>>
> >>>> I agree with the inconsistency issue. About the name itself though,
> >>> should
> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
> since
> >>> we
> >>>> need to capitalize "I" right?
> >>>>
> >>>>
> >>>>> 8. The document states "The first four fields of a message set in
> >> this
> >>>>> format must to be the same as the existing format because any fields
> >>>> before
> >>>>> the magic byte cannot be changed in order to provide a path for
> >>> upgrades
> >>>>> following a similar approach as was used in KIP-32". This makes
> >> things
> >>>>> easier, but it seems to me that the only strict requirement is that
> >> the
> >>>>> magic byte remains in the same offset and with the same size.
> >>>>>
> >>>>>
> >>>> I agree theoretically it is not required, but I think in practice it
> is
> >>>> actually better to make it more restrict: the three fields before
> magic
> >>>> byte are offset, length, and crc. Among them, crc needs to be before
> >>> magic
> >>>> byte if it wants to cover the magic byte fields; length would better
> be
> >>>> before the magic byte as well for pre-allocate memory to
> >> deser/decompress
> >>>> the message set, and the only field that does not matter too much to
> be
> >>>> after magic byte is offset, but in KIP-98 we will use it as the base
> >>> offset
> >>>> for message set and some validation checks can be optimized to not
> scan
> >>>> through the whole message with this field in front of the format.
> >>>>
> >>>>
> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hello Folks,
> >>>>>>
> >>>>>> We have addressed all the comments collected so far, and would like
> >>> to
> >>>>>> propose a voting thread this Wednesday. If you have any further
> >>>> comments
> >>>>> on
> >>>>>> this KIP, please feel free to continue sending them on this thread
> >>>> before
> >>>>>> that.
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>>
> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> >> jason@confluent.io
> >>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> +1 for transactional.id.
> >>>>>>>
> >>>>>>> -Jason
> >>>>>>>
> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> >> wangguoz@gmail.com
> >>>>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> If I have to choose between app.id and
> >> transactional.instance.id
> >>> ,
> >>>>> I'd
> >>>>>>>> choose the latter.
> >>>>>>>>
> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
> >>> even
> >>>>>>> better.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Guozhang
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> >>>> apurva@confluent.io>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
> >>> has
> >>>>>> caused
> >>>>>>>>> some
> >>>>>>>>>> confusion. We're considering the following renaming:
> >>>>>>>>>>
> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
> >>>>>>>> transactional.id
> >>>>>>>>> )
> >>>>>>>>>
> >>>>>>>>> This makes it clear that this id just needs to be set when
> >> the
> >>>>>>>> application
> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
> >>> in
> >>>>> the
> >>>>>>>>> context of how this id is used, viz. to maintain transactions
> >>>>> across
> >>>>>>>>> producer sessions.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Apurva
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> --
> >>>>>>>> -- Guozhang
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> -- Guozhang
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>
> >>
>

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

Posted by Apurva Mehta <ap...@confluent.io>.
Good point Tom. We will update the KIP with the ACLs section and also the message format changes. 

> On Feb 2, 2017, at 06:45, Tom Crayford <tc...@heroku.com> wrote:
> 
> I said this in the voting thread, but can the authors include a section
> about new ACLs if there are going to be ACLs for TransactionalId. It's
> mentioned in the google doc, but I think new ACLs should be in a KIP
> directly.
> 
>> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:
>> 
>> Thanks for the responses and updates to the document, Guozhang and Jason.
>> They look good. One follow-up and one additional comment:
>> 
>> 1. I did some benchmarking and CRC32C seems to be a massive win when using
>> the hardware instruction (particularly for messages larger than 65k), so
>> I'm keen on taking advantage of the message format version bump to add
>> support for it. I can write a separate KIP for this as it's not tied to
>> Exactly-once, but it would be good to include the code change in the same
>> PR that bumps the message format version. The benchmark and results can be
>> found in the following link:
>> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>> 
>> 2. The message timestamp field is 8 bytes. Did we consider storing the
>> first timestamp in the message set and then storing deltas using varints in
>> the messages like we do for offsets (the difference would be the usage of
>> signed varints)? It seems like the deltas would be quite a bit smaller in
>> the common case (potentially 0 for log append time, so we could even not
>> store them at all using attributes like we do for key/value lengths). An
>> alternative is using MaxTimestamp that is already present in the message
>> set and computing deltas from that, but that seems more complicated. In any
>> case, details aside, was this idea considered and rejected or is it worth
>> exploring further?
>> 
>> Ismael
>> 
>> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
>> wrote:
>> 
>>> Ismael,
>>> 
>>> Thanks for the comments. A few responses below:
>>> 
>>> 
>>>> 2. `ProducerAppId` is a new authorization resource type. This
>> introduces
>>> a
>>>> compatibility issue with regards to existing third-party authorizers.
>> It
>>>> would be good to highlight this in the migration/compatibility section.
>>> 
>>> 
>>> Ack. I added a note in the migration section.
>>> 
>>> 4. The Migration plan is relatively brief at the moment. Have we
>>> considered
>>>> if there's any additional work required due to KIP-97 (introduced in
>>>> 0.10.2.0)?
>>> 
>>> 
>>> Thanks, I added a few notes about client compatibility to the migration
>>> section. I covered the main issues that come to mind, but let me know if
>>> you think of others.
>>> 
>>> 7. It seems like there is a bit of inconsistency when it comes to naming
>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>> conventions.
>>>> There are a few other examples like that and it would be good to clean
>>> them
>>>> up.
>>> 
>>> 
>>> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
>> seems
>>> like a compromise which satisfies no one.
>>> 
>>> 
>>> -Jason
>>> 
>>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>> 
>>>> Ismael, thanks for your feedbacks. Replied inline.
>>>> 
>>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
>>>>> 
>>>>> Hi all,
>>>>> 
>>>>> A few comments follow:
>>>>> 
>>>>> 1. The document states "inter-broker communications will be increased
>>> by
>>>> M
>>>>> * N * P round trips per sec. We need to conduct some system
>> performance
>>>>> test to make sure this additional inter-broker traffic would not
>>> largely
>>>>> impact the broker cluster". Has this testing been done? And if not,
>> are
>>>> we
>>>>> planning to do it soon? It seems important to validate this sooner
>>> rather
>>>>> than later. This applies more generally too, it would be great to
>>>>> understand how the new message format affects the producer with small
>>>>> messages, for example.
>>>>> 
>>>>> 
>>>> Yes we are conducting the perf tests with the message format changes in
>>> the
>>>> first stage; then the inter-broker communication with minimal
>> transaction
>>>> coordinator implementations in the second stage.
>>>> 
>>>> 
>>>>> 2. `ProducerAppId` is a new authorization resource type. This
>>> introduces
>>>> a
>>>>> compatibility issue with regards to existing third-party authorizers.
>>> It
>>>>> would be good to highlight this in the migration/compatibility
>> section.
>>>>> 
>>>>> 3. I was happy to see that default values for the new configs have
>> been
>>>>> added to the document since I last checked it. It would be good to
>>>> explain
>>>>> the motivation for the choices.
>>>>> 
>>>>> 
>>>> Updated doc.
>>>> 
>>>> 
>>>>> 4. The Migration plan is relatively brief at the moment. Have we
>>>> considered
>>>>> if there's any additional work required due to KIP-97 (introduced in
>>>>> 0.10.2.0)?
>>>>> 
>>>>> 5. transactional.id sounds good
>>>>> 
>>>>> 6. Since we are keeping per message CRCs for auditing apps, have we
>>>>> considered mitigating the performance cost by using the more
>> performant
>>>>> CRC32c in the new message format version?
>>>>> 
>>>>> 
>>>> We have not discussed about this before. But I think it should be
>> doable
>>> as
>>>> long as we can include the additional conversion logic in the migration
>>>> plan.
>>>> 
>>>> 
>>>>> Nits:
>>>>> 
>>>>> 7. It seems like there is a bit of inconsistency when it comes to
>>> naming
>>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
>>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
>>> conventions.
>>>>> There are a few other examples like that and it would be good to
>> clean
>>>> them
>>>>> up.
>>>>> 
>>>>> 
>>>> I agree with the inconsistency issue. About the name itself though,
>>> should
>>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
>>> we
>>>> need to capitalize "I" right?
>>>> 
>>>> 
>>>>> 8. The document states "The first four fields of a message set in
>> this
>>>>> format must to be the same as the existing format because any fields
>>>> before
>>>>> the magic byte cannot be changed in order to provide a path for
>>> upgrades
>>>>> following a similar approach as was used in KIP-32". This makes
>> things
>>>>> easier, but it seems to me that the only strict requirement is that
>> the
>>>>> magic byte remains in the same offset and with the same size.
>>>>> 
>>>>> 
>>>> I agree theoretically it is not required, but I think in practice it is
>>>> actually better to make it more restrict: the three fields before magic
>>>> byte are offset, length, and crc. Among them, crc needs to be before
>>> magic
>>>> byte if it wants to cover the magic byte fields; length would better be
>>>> before the magic byte as well for pre-allocate memory to
>> deser/decompress
>>>> the message set, and the only field that does not matter too much to be
>>>> after magic byte is offset, but in KIP-98 we will use it as the base
>>> offset
>>>> for message set and some validation checks can be optimized to not scan
>>>> through the whole message with this field in front of the format.
>>>> 
>>>> 
>>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>> 
>>>>>> Hello Folks,
>>>>>> 
>>>>>> We have addressed all the comments collected so far, and would like
>>> to
>>>>>> propose a voting thread this Wednesday. If you have any further
>>>> comments
>>>>> on
>>>>>> this KIP, please feel free to continue sending them on this thread
>>>> before
>>>>>> that.
>>>>>> 
>>>>>> 
>>>>>> Guozhang
>>>>>> 
>>>>>> 
>>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
>> jason@confluent.io
>>>> 
>>>>>> wrote:
>>>>>> 
>>>>>>> +1 for transactional.id.
>>>>>>> 
>>>>>>> -Jason
>>>>>>> 
>>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
>> wangguoz@gmail.com
>>>> 
>>>>>> wrote:
>>>>>>> 
>>>>>>>> If I have to choose between app.id and
>> transactional.instance.id
>>> ,
>>>>> I'd
>>>>>>>> choose the latter.
>>>>>>>> 
>>>>>>>> Renaming transactional.instance.id to transactional.id sounds
>>> even
>>>>>>> better.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Guozhang
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
>>>> apurva@confluent.io>
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
>>> has
>>>>>> caused
>>>>>>>>> some
>>>>>>>>>> confusion. We're considering the following renaming:
>>>>>>>>>> 
>>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
>>>>>>>>>> 2. PID -> IPID (internal producer ID)
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
>>>>>>>> transactional.id
>>>>>>>>> )
>>>>>>>>> 
>>>>>>>>> This makes it clear that this id just needs to be set when
>> the
>>>>>>>> application
>>>>>>>>> wishes to use transactions. I also think it is more intuitive
>>> in
>>>>> the
>>>>>>>>> context of how this id is used, viz. to maintain transactions
>>>>> across
>>>>>>>>> producer sessions.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> Apurva
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> --
>>>>>>>> -- Guozhang
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> -- Guozhang
>>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> --
>>>> -- Guozhang
>>>> 
>>> 
>> 

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

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

On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Thanks for the responses and updates to the document, Guozhang and Jason.
> They look good. One follow-up and one additional comment:
>
> 1. I did some benchmarking and CRC32C seems to be a massive win when using
> the hardware instruction (particularly for messages larger than 65k), so
> I'm keen on taking advantage of the message format version bump to add
> support for it. I can write a separate KIP for this as it's not tied to
> Exactly-once, but it would be good to include the code change in the same
> PR that bumps the message format version. The benchmark and results can be
> found in the following link:
> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>
> 2. The message timestamp field is 8 bytes. Did we consider storing the
> first timestamp in the message set and then storing deltas using varints in
> the messages like we do for offsets (the difference would be the usage of
> signed varints)? It seems like the deltas would be quite a bit smaller in
> the common case (potentially 0 for log append time, so we could even not
> store them at all using attributes like we do for key/value lengths). An
> alternative is using MaxTimestamp that is already present in the message
> set and computing deltas from that, but that seems more complicated. In any
> case, details aside, was this idea considered and rejected or is it worth
> exploring further?
>
> Ismael
>
> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Ismael,
> >
> > Thanks for the comments. A few responses below:
> >
> >
> > > 2. `ProducerAppId` is a new authorization resource type. This
> introduces
> > a
> > > compatibility issue with regards to existing third-party authorizers.
> It
> > > would be good to highlight this in the migration/compatibility section.
> >
> >
> > Ack. I added a note in the migration section.
> >
> >  4. The Migration plan is relatively brief at the moment. Have we
> > considered
> > > if there's any additional work required due to KIP-97 (introduced in
> > > 0.10.2.0)?
> >
> >
> > Thanks, I added a few notes about client compatibility to the migration
> > section. I covered the main issues that come to mind, but let me know if
> > you think of others.
> >
> > 7. It seems like there is a bit of inconsistency when it comes to naming
> > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > and `InvalidPidMapping`. The latter two match Kafka's naming
> conventions.
> > > There are a few other examples like that and it would be good to clean
> > them
> > > up.
> >
> >
> > Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest"
> seems
> > like a compromise which satisfies no one.
> >
> >
> > -Jason
> >
> > On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Ismael, thanks for your feedbacks. Replied inline.
> > >
> > > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
> > >
> > > > Hi all,
> > > >
> > > > A few comments follow:
> > > >
> > > > 1. The document states "inter-broker communications will be increased
> > by
> > > M
> > > > * N * P round trips per sec. We need to conduct some system
> performance
> > > > test to make sure this additional inter-broker traffic would not
> > largely
> > > > impact the broker cluster". Has this testing been done? And if not,
> are
> > > we
> > > > planning to do it soon? It seems important to validate this sooner
> > rather
> > > > than later. This applies more generally too, it would be great to
> > > > understand how the new message format affects the producer with small
> > > > messages, for example.
> > > >
> > > >
> > > Yes we are conducting the perf tests with the message format changes in
> > the
> > > first stage; then the inter-broker communication with minimal
> transaction
> > > coordinator implementations in the second stage.
> > >
> > >
> > > > 2. `ProducerAppId` is a new authorization resource type. This
> > introduces
> > > a
> > > > compatibility issue with regards to existing third-party authorizers.
> > It
> > > > would be good to highlight this in the migration/compatibility
> section.
> > > >
> > > > 3. I was happy to see that default values for the new configs have
> been
> > > > added to the document since I last checked it. It would be good to
> > > explain
> > > > the motivation for the choices.
> > > >
> > > >
> > > Updated doc.
> > >
> > >
> > > > 4. The Migration plan is relatively brief at the moment. Have we
> > > considered
> > > > if there's any additional work required due to KIP-97 (introduced in
> > > > 0.10.2.0)?
> > > >
> > > > 5. transactional.id sounds good
> > > >
> > > > 6. Since we are keeping per message CRCs for auditing apps, have we
> > > > considered mitigating the performance cost by using the more
> performant
> > > > CRC32c in the new message format version?
> > > >
> > > >
> > > We have not discussed about this before. But I think it should be
> doable
> > as
> > > long as we can include the additional conversion logic in the migration
> > > plan.
> > >
> > >
> > > > Nits:
> > > >
> > > > 7. It seems like there is a bit of inconsistency when it comes to
> > naming
> > > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > > and `InvalidPidMapping`. The latter two match Kafka's naming
> > conventions.
> > > > There are a few other examples like that and it would be good to
> clean
> > > them
> > > > up.
> > > >
> > > >
> > > I agree with the inconsistency issue. About the name itself though,
> > should
> > > it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
> > we
> > > need to capitalize "I" right?
> > >
> > >
> > > > 8. The document states "The first four fields of a message set in
> this
> > > > format must to be the same as the existing format because any fields
> > > before
> > > > the magic byte cannot be changed in order to provide a path for
> > upgrades
> > > > following a similar approach as was used in KIP-32". This makes
> things
> > > > easier, but it seems to me that the only strict requirement is that
> the
> > > > magic byte remains in the same offset and with the same size.
> > > >
> > > >
> > > I agree theoretically it is not required, but I think in practice it is
> > > actually better to make it more restrict: the three fields before magic
> > > byte are offset, length, and crc. Among them, crc needs to be before
> > magic
> > > byte if it wants to cover the magic byte fields; length would better be
> > > before the magic byte as well for pre-allocate memory to
> deser/decompress
> > > the message set, and the only field that does not matter too much to be
> > > after magic byte is offset, but in KIP-98 we will use it as the base
> > offset
> > > for message set and some validation checks can be optimized to not scan
> > > through the whole message with this field in front of the format.
> > >
> > >
> > > > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Folks,
> > > > >
> > > > > We have addressed all the comments collected so far, and would like
> > to
> > > > > propose a voting thread this Wednesday. If you have any further
> > > comments
> > > > on
> > > > > this KIP, please feel free to continue sending them on this thread
> > > before
> > > > > that.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> jason@confluent.io
> > >
> > > > > wrote:
> > > > >
> > > > > > +1 for transactional.id.
> > > > > >
> > > > > > -Jason
> > > > > >
> > > > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > If I have to choose between app.id and
> transactional.instance.id
> > ,
> > > > I'd
> > > > > > > choose the latter.
> > > > > > >
> > > > > > > Renaming transactional.instance.id to transactional.id sounds
> > even
> > > > > > better.
> > > > > > >
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> > > apurva@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > > Bumping one suggestion from Apurva above. The name "AppID"
> > has
> > > > > caused
> > > > > > > > some
> > > > > > > > > confusion. We're considering the following renaming:
> > > > > > > > >
> > > > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > > > transactional.id
> > > > > > > > )
> > > > > > > >
> > > > > > > > This makes it clear that this id just needs to be set when
> the
> > > > > > > application
> > > > > > > > wishes to use transactions. I also think it is more intuitive
> > in
> > > > the
> > > > > > > > context of how this id is used, viz. to maintain transactions
> > > > across
> > > > > > > > producer sessions.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Apurva
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

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

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks for the responses and updates to the document, Guozhang and Jason.
They look good. One follow-up and one additional comment:

1. I did some benchmarking and CRC32C seems to be a massive win when using
the hardware instruction (particularly for messages larger than 65k), so
I'm keen on taking advantage of the message format version bump to add
support for it. I can write a separate KIP for this as it's not tied to
Exactly-once, but it would be good to include the code change in the same
PR that bumps the message format version. The benchmark and results can be
found in the following link:
https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.

2. The message timestamp field is 8 bytes. Did we consider storing the
first timestamp in the message set and then storing deltas using varints in
the messages like we do for offsets (the difference would be the usage of
signed varints)? It seems like the deltas would be quite a bit smaller in
the common case (potentially 0 for log append time, so we could even not
store them at all using attributes like we do for key/value lengths). An
alternative is using MaxTimestamp that is already present in the message
set and computing deltas from that, but that seems more complicated. In any
case, details aside, was this idea considered and rejected or is it worth
exploring further?

Ismael

On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io> wrote:

> Ismael,
>
> Thanks for the comments. A few responses below:
>
>
> > 2. `ProducerAppId` is a new authorization resource type. This introduces
> a
> > compatibility issue with regards to existing third-party authorizers. It
> > would be good to highlight this in the migration/compatibility section.
>
>
> Ack. I added a note in the migration section.
>
>  4. The Migration plan is relatively brief at the moment. Have we
> considered
> > if there's any additional work required due to KIP-97 (introduced in
> > 0.10.2.0)?
>
>
> Thanks, I added a few notes about client compatibility to the migration
> section. I covered the main issues that come to mind, but let me know if
> you think of others.
>
> 7. It seems like there is a bit of inconsistency when it comes to naming
> > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> > There are a few other examples like that and it would be good to clean
> them
> > up.
>
>
> Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest" seems
> like a compromise which satisfies no one.
>
>
> -Jason
>
> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Ismael, thanks for your feedbacks. Replied inline.
> >
> > On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
> >
> > > Hi all,
> > >
> > > A few comments follow:
> > >
> > > 1. The document states "inter-broker communications will be increased
> by
> > M
> > > * N * P round trips per sec. We need to conduct some system performance
> > > test to make sure this additional inter-broker traffic would not
> largely
> > > impact the broker cluster". Has this testing been done? And if not, are
> > we
> > > planning to do it soon? It seems important to validate this sooner
> rather
> > > than later. This applies more generally too, it would be great to
> > > understand how the new message format affects the producer with small
> > > messages, for example.
> > >
> > >
> > Yes we are conducting the perf tests with the message format changes in
> the
> > first stage; then the inter-broker communication with minimal transaction
> > coordinator implementations in the second stage.
> >
> >
> > > 2. `ProducerAppId` is a new authorization resource type. This
> introduces
> > a
> > > compatibility issue with regards to existing third-party authorizers.
> It
> > > would be good to highlight this in the migration/compatibility section.
> > >
> > > 3. I was happy to see that default values for the new configs have been
> > > added to the document since I last checked it. It would be good to
> > explain
> > > the motivation for the choices.
> > >
> > >
> > Updated doc.
> >
> >
> > > 4. The Migration plan is relatively brief at the moment. Have we
> > considered
> > > if there's any additional work required due to KIP-97 (introduced in
> > > 0.10.2.0)?
> > >
> > > 5. transactional.id sounds good
> > >
> > > 6. Since we are keeping per message CRCs for auditing apps, have we
> > > considered mitigating the performance cost by using the more performant
> > > CRC32c in the new message format version?
> > >
> > >
> > We have not discussed about this before. But I think it should be doable
> as
> > long as we can include the additional conversion logic in the migration
> > plan.
> >
> >
> > > Nits:
> > >
> > > 7. It seems like there is a bit of inconsistency when it comes to
> naming
> > > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > > and `InvalidPidMapping`. The latter two match Kafka's naming
> conventions.
> > > There are a few other examples like that and it would be good to clean
> > them
> > > up.
> > >
> > >
> > I agree with the inconsistency issue. About the name itself though,
> should
> > it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since
> we
> > need to capitalize "I" right?
> >
> >
> > > 8. The document states "The first four fields of a message set in this
> > > format must to be the same as the existing format because any fields
> > before
> > > the magic byte cannot be changed in order to provide a path for
> upgrades
> > > following a similar approach as was used in KIP-32". This makes things
> > > easier, but it seems to me that the only strict requirement is that the
> > > magic byte remains in the same offset and with the same size.
> > >
> > >
> > I agree theoretically it is not required, but I think in practice it is
> > actually better to make it more restrict: the three fields before magic
> > byte are offset, length, and crc. Among them, crc needs to be before
> magic
> > byte if it wants to cover the magic byte fields; length would better be
> > before the magic byte as well for pre-allocate memory to deser/decompress
> > the message set, and the only field that does not matter too much to be
> > after magic byte is offset, but in KIP-98 we will use it as the base
> offset
> > for message set and some validation checks can be optimized to not scan
> > through the whole message with this field in front of the format.
> >
> >
> > > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > >
> > > > Hello Folks,
> > > >
> > > > We have addressed all the comments collected so far, and would like
> to
> > > > propose a voting thread this Wednesday. If you have any further
> > comments
> > > on
> > > > this KIP, please feel free to continue sending them on this thread
> > before
> > > > that.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <jason@confluent.io
> >
> > > > wrote:
> > > >
> > > > > +1 for transactional.id.
> > > > >
> > > > > -Jason
> > > > >
> > > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > If I have to choose between app.id and transactional.instance.id
> ,
> > > I'd
> > > > > > choose the latter.
> > > > > >
> > > > > > Renaming transactional.instance.id to transactional.id sounds
> even
> > > > > better.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> > apurva@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > > Bumping one suggestion from Apurva above. The name "AppID"
> has
> > > > caused
> > > > > > > some
> > > > > > > > confusion. We're considering the following renaming:
> > > > > > > >
> > > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > > transactional.id
> > > > > > > )
> > > > > > >
> > > > > > > This makes it clear that this id just needs to be set when the
> > > > > > application
> > > > > > > wishes to use transactions. I also think it is more intuitive
> in
> > > the
> > > > > > > context of how this id is used, viz. to maintain transactions
> > > across
> > > > > > > producer sessions.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Apurva
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

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

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

Thanks for the comments. A few responses below:


> 2. `ProducerAppId` is a new authorization resource type. This introduces a
> compatibility issue with regards to existing third-party authorizers. It
> would be good to highlight this in the migration/compatibility section.


Ack. I added a note in the migration section.

 4. The Migration plan is relatively brief at the moment. Have we considered
> if there's any additional work required due to KIP-97 (introduced in
> 0.10.2.0)?


Thanks, I added a few notes about client compatibility to the migration
section. I covered the main issues that come to mind, but let me know if
you think of others.

7. It seems like there is a bit of inconsistency when it comes to naming
> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> There are a few other examples like that and it would be good to clean them
> up.


Let's go with InitPidRequest for consistency.  Haha, "InitPIdRequest" seems
like a compromise which satisfies no one.


-Jason

On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Ismael, thanks for your feedbacks. Replied inline.
>
> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hi all,
> >
> > A few comments follow:
> >
> > 1. The document states "inter-broker communications will be increased by
> M
> > * N * P round trips per sec. We need to conduct some system performance
> > test to make sure this additional inter-broker traffic would not largely
> > impact the broker cluster". Has this testing been done? And if not, are
> we
> > planning to do it soon? It seems important to validate this sooner rather
> > than later. This applies more generally too, it would be great to
> > understand how the new message format affects the producer with small
> > messages, for example.
> >
> >
> Yes we are conducting the perf tests with the message format changes in the
> first stage; then the inter-broker communication with minimal transaction
> coordinator implementations in the second stage.
>
>
> > 2. `ProducerAppId` is a new authorization resource type. This introduces
> a
> > compatibility issue with regards to existing third-party authorizers. It
> > would be good to highlight this in the migration/compatibility section.
> >
> > 3. I was happy to see that default values for the new configs have been
> > added to the document since I last checked it. It would be good to
> explain
> > the motivation for the choices.
> >
> >
> Updated doc.
>
>
> > 4. The Migration plan is relatively brief at the moment. Have we
> considered
> > if there's any additional work required due to KIP-97 (introduced in
> > 0.10.2.0)?
> >
> > 5. transactional.id sounds good
> >
> > 6. Since we are keeping per message CRCs for auditing apps, have we
> > considered mitigating the performance cost by using the more performant
> > CRC32c in the new message format version?
> >
> >
> We have not discussed about this before. But I think it should be doable as
> long as we can include the additional conversion logic in the migration
> plan.
>
>
> > Nits:
> >
> > 7. It seems like there is a bit of inconsistency when it comes to naming
> > convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> > and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> > There are a few other examples like that and it would be good to clean
> them
> > up.
> >
> >
> I agree with the inconsistency issue. About the name itself though, should
> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since we
> need to capitalize "I" right?
>
>
> > 8. The document states "The first four fields of a message set in this
> > format must to be the same as the existing format because any fields
> before
> > the magic byte cannot be changed in order to provide a path for upgrades
> > following a similar approach as was used in KIP-32". This makes things
> > easier, but it seems to me that the only strict requirement is that the
> > magic byte remains in the same offset and with the same size.
> >
> >
> I agree theoretically it is not required, but I think in practice it is
> actually better to make it more restrict: the three fields before magic
> byte are offset, length, and crc. Among them, crc needs to be before magic
> byte if it wants to cover the magic byte fields; length would better be
> before the magic byte as well for pre-allocate memory to deser/decompress
> the message set, and the only field that does not matter too much to be
> after magic byte is offset, but in KIP-98 we will use it as the base offset
> for message set and some validation checks can be optimized to not scan
> through the whole message with this field in front of the format.
>
>
> > On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> >
> > > Hello Folks,
> > >
> > > We have addressed all the comments collected so far, and would like to
> > > propose a voting thread this Wednesday. If you have any further
> comments
> > on
> > > this KIP, please feel free to continue sending them on this thread
> before
> > > that.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> > > wrote:
> > >
> > > > +1 for transactional.id.
> > > >
> > > > -Jason
> > > >
> > > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > If I have to choose between app.id and transactional.instance.id,
> > I'd
> > > > > choose the latter.
> > > > >
> > > > > Renaming transactional.instance.id to transactional.id sounds even
> > > > better.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> apurva@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> > > caused
> > > > > > some
> > > > > > > confusion. We're considering the following renaming:
> > > > > > >
> > > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > > 2. PID -> IPID (internal producer ID)
> > > > > > >
> > > > > >
> > > > > >
> > > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > > transactional.id
> > > > > > )
> > > > > >
> > > > > > This makes it clear that this id just needs to be set when the
> > > > > application
> > > > > > wishes to use transactions. I also think it is more intuitive in
> > the
> > > > > > context of how this id is used, viz. to maintain transactions
> > across
> > > > > > producer sessions.
> > > > > >
> > > > > > Thanks,
> > > > > > Apurva
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
Ismael, thanks for your feedbacks. Replied inline.

On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi all,
>
> A few comments follow:
>
> 1. The document states "inter-broker communications will be increased by M
> * N * P round trips per sec. We need to conduct some system performance
> test to make sure this additional inter-broker traffic would not largely
> impact the broker cluster". Has this testing been done? And if not, are we
> planning to do it soon? It seems important to validate this sooner rather
> than later. This applies more generally too, it would be great to
> understand how the new message format affects the producer with small
> messages, for example.
>
>
Yes we are conducting the perf tests with the message format changes in the
first stage; then the inter-broker communication with minimal transaction
coordinator implementations in the second stage.


> 2. `ProducerAppId` is a new authorization resource type. This introduces a
> compatibility issue with regards to existing third-party authorizers. It
> would be good to highlight this in the migration/compatibility section.
>
> 3. I was happy to see that default values for the new configs have been
> added to the document since I last checked it. It would be good to explain
> the motivation for the choices.
>
>
Updated doc.


> 4. The Migration plan is relatively brief at the moment. Have we considered
> if there's any additional work required due to KIP-97 (introduced in
> 0.10.2.0)?
>
> 5. transactional.id sounds good
>
> 6. Since we are keeping per message CRCs for auditing apps, have we
> considered mitigating the performance cost by using the more performant
> CRC32c in the new message format version?
>
>
We have not discussed about this before. But I think it should be doable as
long as we can include the additional conversion logic in the migration
plan.


> Nits:
>
> 7. It seems like there is a bit of inconsistency when it comes to naming
> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
> There are a few other examples like that and it would be good to clean them
> up.
>
>
I agree with the inconsistency issue. About the name itself though, should
it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though, since we
need to capitalize "I" right?


> 8. The document states "The first four fields of a message set in this
> format must to be the same as the existing format because any fields before
> the magic byte cannot be changed in order to provide a path for upgrades
> following a similar approach as was used in KIP-32". This makes things
> easier, but it seems to me that the only strict requirement is that the
> magic byte remains in the same offset and with the same size.
>
>
I agree theoretically it is not required, but I think in practice it is
actually better to make it more restrict: the three fields before magic
byte are offset, length, and crc. Among them, crc needs to be before magic
byte if it wants to cover the magic byte fields; length would better be
before the magic byte as well for pre-allocate memory to deser/decompress
the message set, and the only field that does not matter too much to be
after magic byte is offset, but in KIP-98 we will use it as the base offset
for message set and some validation checks can be optimized to not scan
through the whole message with this field in front of the format.


> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > Hello Folks,
> >
> > We have addressed all the comments collected so far, and would like to
> > propose a voting thread this Wednesday. If you have any further comments
> on
> > this KIP, please feel free to continue sending them on this thread before
> > that.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> > wrote:
> >
> > > +1 for transactional.id.
> > >
> > > -Jason
> > >
> > > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > If I have to choose between app.id and transactional.instance.id,
> I'd
> > > > choose the latter.
> > > >
> > > > Renaming transactional.instance.id to transactional.id sounds even
> > > better.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > > > wrote:
> > > >
> > > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> > caused
> > > > > some
> > > > > > confusion. We're considering the following renaming:
> > > > > >
> > > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > > 2. PID -> IPID (internal producer ID)
> > > > > >
> > > > >
> > > > >
> > > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > > transactional.id
> > > > > )
> > > > >
> > > > > This makes it clear that this id just needs to be set when the
> > > > application
> > > > > wishes to use transactions. I also think it is more intuitive in
> the
> > > > > context of how this id is used, viz. to maintain transactions
> across
> > > > > producer sessions.
> > > > >
> > > > > Thanks,
> > > > > Apurva
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

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

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

A few comments follow:

1. The document states "inter-broker communications will be increased by M
* N * P round trips per sec. We need to conduct some system performance
test to make sure this additional inter-broker traffic would not largely
impact the broker cluster". Has this testing been done? And if not, are we
planning to do it soon? It seems important to validate this sooner rather
than later. This applies more generally too, it would be great to
understand how the new message format affects the producer with small
messages, for example.

2. `ProducerAppId` is a new authorization resource type. This introduces a
compatibility issue with regards to existing third-party authorizers. It
would be good to highlight this in the migration/compatibility section.

3. I was happy to see that default values for the new configs have been
added to the document since I last checked it. It would be good to explain
the motivation for the choices.

4. The Migration plan is relatively brief at the moment. Have we considered
if there's any additional work required due to KIP-97 (introduced in
0.10.2.0)?

5. transactional.id sounds good

6. Since we are keeping per message CRCs for auditing apps, have we
considered mitigating the performance cost by using the more performant
CRC32c in the new message format version?

Nits:

7. It seems like there is a bit of inconsistency when it comes to naming
convention. For example, we have `InitPIDRequest`, `PidSnapshot`
and `InvalidPidMapping`. The latter two match Kafka's naming conventions.
There are a few other examples like that and it would be good to clean them
up.

8. The document states "The first four fields of a message set in this
format must to be the same as the existing format because any fields before
the magic byte cannot be changed in order to provide a path for upgrades
following a similar approach as was used in KIP-32". This makes things
easier, but it seems to me that the only strict requirement is that the
magic byte remains in the same offset and with the same size.


On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Hello Folks,
>
> We have addressed all the comments collected so far, and would like to
> propose a voting thread this Wednesday. If you have any further comments on
> this KIP, please feel free to continue sending them on this thread before
> that.
>
>
> Guozhang
>
>
> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > +1 for transactional.id.
> >
> > -Jason
> >
> > On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > If I have to choose between app.id and transactional.instance.id, I'd
> > > choose the latter.
> > >
> > > Renaming transactional.instance.id to transactional.id sounds even
> > better.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > > wrote:
> > >
> > > > > Bumping one suggestion from Apurva above. The name "AppID" has
> caused
> > > > some
> > > > > confusion. We're considering the following renaming:
> > > > >
> > > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > > 2. PID -> IPID (internal producer ID)
> > > > >
> > > >
> > > >
> > > > How about AppId -> TransactionalId (transaction.app.id ->
> > > transactional.id
> > > > )
> > > >
> > > > This makes it clear that this id just needs to be set when the
> > > application
> > > > wishes to use transactions. I also think it is more intuitive in the
> > > > context of how this id is used, viz. to maintain transactions across
> > > > producer sessions.
> > > >
> > > > Thanks,
> > > > Apurva
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
>
> --
> -- Guozhang
>

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

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

We have addressed all the comments collected so far, and would like to
propose a voting thread this Wednesday. If you have any further comments on
this KIP, please feel free to continue sending them on this thread before
that.


Guozhang


On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io> wrote:

> +1 for transactional.id.
>
> -Jason
>
> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > If I have to choose between app.id and transactional.instance.id, I'd
> > choose the latter.
> >
> > Renaming transactional.instance.id to transactional.id sounds even
> better.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > > some
> > > > confusion. We're considering the following renaming:
> > > >
> > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > 2. PID -> IPID (internal producer ID)
> > > >
> > >
> > >
> > > How about AppId -> TransactionalId (transaction.app.id ->
> > transactional.id
> > > )
> > >
> > > This makes it clear that this id just needs to be set when the
> > application
> > > wishes to use transactions. I also think it is more intuitive in the
> > > context of how this id is used, viz. to maintain transactions across
> > > producer sessions.
> > >
> > > Thanks,
> > > Apurva
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

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

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

We have addressed all the comments collected so far, and would like to
propose a voting thread this Wednesday. If you have any further comments on
this KIP, please feel free to continue sending them on this thread before
that.


Guozhang


On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <ja...@confluent.io> wrote:

> +1 for transactional.id.
>
> -Jason
>
> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > If I have to choose between app.id and transactional.instance.id, I'd
> > choose the latter.
> >
> > Renaming transactional.instance.id to transactional.id sounds even
> better.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> > wrote:
> >
> > > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > > some
> > > > confusion. We're considering the following renaming:
> > > >
> > > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > > 2. PID -> IPID (internal producer ID)
> > > >
> > >
> > >
> > > How about AppId -> TransactionalId (transaction.app.id ->
> > transactional.id
> > > )
> > >
> > > This makes it clear that this id just needs to be set when the
> > application
> > > wishes to use transactions. I also think it is more intuitive in the
> > > context of how this id is used, viz. to maintain transactions across
> > > producer sessions.
> > >
> > > Thanks,
> > > Apurva
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

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

Posted by Jason Gustafson <ja...@confluent.io>.
+1 for transactional.id.

-Jason

On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <wa...@gmail.com> wrote:

> If I have to choose between app.id and transactional.instance.id, I'd
> choose the latter.
>
> Renaming transactional.instance.id to transactional.id sounds even better.
>
>
> Guozhang
>
>
> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io>
> wrote:
>
> > > Bumping one suggestion from Apurva above. The name "AppID" has caused
> > some
> > > confusion. We're considering the following renaming:
> > >
> > > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > > 2. PID -> IPID (internal producer ID)
> > >
> >
> >
> > How about AppId -> TransactionalId (transaction.app.id ->
> transactional.id
> > )
> >
> > This makes it clear that this id just needs to be set when the
> application
> > wishes to use transactions. I also think it is more intuitive in the
> > context of how this id is used, viz. to maintain transactions across
> > producer sessions.
> >
> > Thanks,
> > Apurva
> >
>
>
>
> --
> -- Guozhang
>

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

Posted by Guozhang Wang <wa...@gmail.com>.
If I have to choose between app.id and transactional.instance.id, I'd
choose the latter.

Renaming transactional.instance.id to transactional.id sounds even better.


Guozhang


On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <ap...@confluent.io> wrote:

> > Bumping one suggestion from Apurva above. The name "AppID" has caused
> some
> > confusion. We're considering the following renaming:
> >
> > 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> > 2. PID -> IPID (internal producer ID)
> >
>
>
> How about AppId -> TransactionalId (transaction.app.id -> transactional.id
> )
>
> This makes it clear that this id just needs to be set when the application
> wishes to use transactions. I also think it is more intuitive in the
> context of how this id is used, viz. to maintain transactions across
> producer sessions.
>
> Thanks,
> Apurva
>



-- 
-- Guozhang

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

Posted by Apurva Mehta <ap...@confluent.io>.
> Bumping one suggestion from Apurva above. The name "AppID" has caused some
> confusion. We're considering the following renaming:
>
> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> 2. PID -> IPID (internal producer ID)
>


How about AppId -> TransactionalId (transaction.app.id -> transactional.id)

This makes it clear that this id just needs to be set when the application
wishes to use transactions. I also think it is more intuitive in the
context of how this id is used, viz. to maintain transactions across
producer sessions.

Thanks,
Apurva

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

Posted by Jason Gustafson <ja...@confluent.io>.
Bumping one suggestion from Apurva above. The name "AppID" has caused some
confusion. We're considering the following renaming:

1. AppID -> ProducerId (transaction.app.id -> producer.id)
2. PID -> IPID (internal producer ID)

The main drawback of using ProducerId is that it fails to express the fact
that it is tied to the use of transactions (it's actually slightly
preferable to not set it if you're not using transactions). It's a bit more
verbose, but I was also considering something like "
transactional.instance.id"

Feedback appreciated.

-Jason

On Mon, Jan 30, 2017 at 11:17 AM, Apurva Mehta <ap...@confluent.io> wrote:

> >   102. When there is a correlated hard failure (e.g., power outage), it's
> >>   possible that an existing commit/abort marker is lost in all replicas.
> >>  This  may not be fixed by the transaction coordinator automatically and
> >> the
> >>   consumer may get stuck on that incomplete transaction forever. Not
> sure
> >>   what's the best way to address this. Perhaps, one way is to run a tool
> >> to
> >>   add an abort maker for all pids in all affected partitions.
> >
> >
>
> There can be two types of tools, one for diagnosing the issue and another
> > for fixing the issue. I think having at least a diagnostic tool in the
> > first version could be helpful. For example, the tool can report things
> > like which producer id is preventing the LSO from being advanced. That
> way,
> > at least the users can try to fix this themselves.
> >
>
>
> That sounds reasonable. Will add a work item to track this so that such a
> tool is available in the first version.
>

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

Posted by Apurva Mehta <ap...@confluent.io>.
>   102. When there is a correlated hard failure (e.g., power outage), it's
>>   possible that an existing commit/abort marker is lost in all replicas.
>>  This  may not be fixed by the transaction coordinator automatically and
>> the
>>   consumer may get stuck on that incomplete transaction forever. Not sure
>>   what's the best way to address this. Perhaps, one way is to run a tool
>> to
>>   add an abort maker for all pids in all affected partitions.
>
>

There can be two types of tools, one for diagnosing the issue and another
> for fixing the issue. I think having at least a diagnostic tool in the
> first version could be helpful. For example, the tool can report things
> like which producer id is preventing the LSO from being advanced. That way,
> at least the users can try to fix this themselves.
>


That sounds reasonable. Will add a work item to track this so that such a
tool is available in the first version.

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

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

Thanks for the update. My replies are inlined below.

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

> Hi Jun,
>
> Thanks again for the comments. More responses follow:
>
>
> > 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> to
> > the new version, but the broker message format is still the old one. If a
> > new producer uses the transaction feature, should the producer get an
> error
> > in this case? A tricky case can be that the leader broker is on the new
> > message format, but the follower broker is still on the old message
> format.
> > In this case, the transactional info will be lost in the follower due to
> > down conversion. Should we failed the transactional requests when the
> > followers are still on the old message format?
> >
>
> This will only be an issue if applications are written to use transactions
> and are deployed with the new client before all the brokers are upgraded to
> the new message format.
>
> There are a variety of engineering solutions to this problem, one of which
> is for each broker to register itself as 'transaction ready' in zookeeper
> when it is on the right version of the message format. Once the controller
> detects that all brokers in the cluster are transaction ready, it will
> signal to each broker via the UpdateMetadataRequest that the cluster is
> ready for transactions. Any transactional requests received by brokers
> before this point will be rejected.
>
> A simpler way to solve this problem is through organizational policy: a
> cluster should not be advertised to application developers as 'transaction
> ready' until all brokers are on the new message format.
>
> I think the non-engineering solution is reasonable, and as such would
> prefer to not include engineering solutions in V1. It could be future work
> if necessary.
>
> We can make the problems that arise out of premature use of transactions
> clear in the release notes so that operators can take the necessary
> precautions. Is that reasonable?
>
>
> Yes, in the first version, we can just document the impact in the upgrade
doc.



>
>
> > 102. When there is a correlated hard failure (e.g., power outage), it's
> > possible that an existing commit/abort marker is lost in all replicas.
> This
> > may not be fixed by the transaction coordinator automatically and the
> > consumer may get stuck on that incomplete transaction forever. Not sure
> > what's the best way to address this. Perhaps, one way is to run a tool to
> > add an abort maker for all pids in all affected partitions.
> >
> >
> This is a good point. With the abort index proposal, if a correlated hard
> failure causes us to lose the markers everywhere, the LSO on the broker
> would not advance and consumers would block (but not buffer). This would be
> a noticeable situation.
>
> A simple tool may make use of internal functions to effectively do a
> 'initPID', 'beginTransaction', 'AddTopicPartitiontoTransaction',
> 'commitTransaction'. This would ensure that the markers are rewritten to
> all partitions by the transaction coordinator, but would also fence the
> existing producer with the same AppId.
>
> To make this workable, we need to make sure that the transaction
> coordinator adds a sufficient logging so that we know the AppID -> PID
> mapping as well as the partitions participating in each transaction. The
> broker should also probably log information so that we know which
> unfinished transaction (ie. which PID) is preventing the LSO from moving
> forward. Both these things will make it fairly easy to configure the tool.
>
> Of course, it is possible for the producer to continue onto another
> transaction before the tool is run, in which case the data will be corrupt
> since the second transaction will include messages from the first. But this
> is no worse than Kafka's existing durability semantics which this proposal
> relies on.
>
> I think such a tool can be a follow up work, and I have added it to the
> 'future work' section of the document.
>
>
> There can be two types of tools, one for diagnosing the issue and another
for fixing the issue. I think having at least a diagnostic tool in the
first version could be helpful. For example, the tool can report things
like which producer id is preventing the LSO from being advanced. That way,
at least the users can try to fix this themselves.


>
> > 103. Currently, there is no check for producer liveness. This means that
> if
> > a producer has not been sending transactional requests for a long time,
> its
> > appId will be expired by the coordinator. Have we considered having
> > producers sending heartbeatRequest just like the consumer to keep it
> alive?
> >
> >
> In the current proposal, a producer whose AppId has expired is a Zombie
> which will get a Fatal 'ProducerFencedException' when it tries to make any
> new transactional requests. A bounce of the producer will reinitialize it,
> at which point it can continue.
>
> As such, while the proposed behavior is not ideal, I think that a heartbeat
> thread would be a nice to have that may not be worth putting into V1. I
> have made a note to add this in the 'Future Work' section of the document.
>
>
>
Yes, as long as we have a path to add heartbeat in the future, this is fine.


>
> > 104. The logic for handling follower truncation can be a bit tricker now.
> > The truncation may rewind the sequence number for some pids. The question
> > is how to quickly recover the last sequence number of those pids. Do we
> > plan to reload from a PID snapshot and scan forward?
> >
> >
> Yes, this is exactly what we intend to do.
>
>
> > 115. Message format:
> > 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> > since it's possible for a producer to be restarted 10s thousands of
> times?
> > 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> > producer to publish more than 2 billion messages in a session.
> >
>
> Yes, both the epoch and the sequence number will have a circular space and
> wrap around.
>
> In the pure idempotent producer case, ie. where there is no AppId, each
> producer session is guaranteed to have a unique PID and the leader for the
> partition validates the incoming sequence number per PID before committing
> to the log. So wrapping is safe.
>
> In the transactional case, the transaction coordinator assigns the epoch
> and returns the current epoch as part of the InitPIDRequest. All
> transaction requests (Begin, AddTPToTransaction, Prepare) will be accepted
> only if their epoch exactly matches the epoch recorded by the coordinator.
> Wrap around is safe except in some degenerate cases like very long lived
> zombie producers, described below.
>
> Say we have a producer at epoch 20, who suddenly becomes a zombie. Assume
> other producers with the same id are bounced for 65536 sessions (since
> epoch are 2 bytes). Now after this period, the zombie producer comes back
> online and would have the same PID/epoch as the current producer. In this
> case, it is possible for both to produce transactions and for this
> situation to go undetected.
>
> We can solve this problem by including 'session ids' for producers and
> validating that as well. But since the degenerate case is so rare, we think
> that any such session id can be part of future work if it becomes a real
> problem.
>
> I have updated the doc to explicitly mention that the epoch and sequence
> numbers may wrap.
>
>
>
Sounds good.


>
> > 119. InitPIDRequest
> > <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8/edit#heading=h.z99xar1h2enr>
> > : Should we write the completion of open transactions before append the
> pid
> > with bumped up epoch to the transaction log?
> >
> >
> Done.
>
>
> > 120. transaction.app.id: An app may have multiple concurrent instances.
> > Perhaps we should name it transaction.instance.id or just instance.id?
> >
> >
> This has been the most debated point. How about just `producer.id`?
>
>
The only thing is that this will be set in the producer config so the
"producer" part seems redundant.


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

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

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

Thanks again for the comments. More responses follow:


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

This will only be an issue if applications are written to use transactions
and are deployed with the new client before all the brokers are upgraded to
the new message format.

There are a variety of engineering solutions to this problem, one of which
is for each broker to register itself as 'transaction ready' in zookeeper
when it is on the right version of the message format. Once the controller
detects that all brokers in the cluster are transaction ready, it will
signal to each broker via the UpdateMetadataRequest that the cluster is
ready for transactions. Any transactional requests received by brokers
before this point will be rejected.

A simpler way to solve this problem is through organizational policy: a
cluster should not be advertised to application developers as 'transaction
ready' until all brokers are on the new message format.

I think the non-engineering solution is reasonable, and as such would
prefer to not include engineering solutions in V1. It could be future work
if necessary.

We can make the problems that arise out of premature use of transactions
clear in the release notes so that operators can take the necessary
precautions. Is that reasonable?




> 102. When there is a correlated hard failure (e.g., power outage), it's
> possible that an existing commit/abort marker is lost in all replicas. This
> may not be fixed by the transaction coordinator automatically and the
> consumer may get stuck on that incomplete transaction forever. Not sure
> what's the best way to address this. Perhaps, one way is to run a tool to
> add an abort maker for all pids in all affected partitions.
>
>
This is a good point. With the abort index proposal, if a correlated hard
failure causes us to lose the markers everywhere, the LSO on the broker
would not advance and consumers would block (but not buffer). This would be
a noticeable situation.

A simple tool may make use of internal functions to effectively do a
'initPID', 'beginTransaction', 'AddTopicPartitiontoTransaction',
'commitTransaction'. This would ensure that the markers are rewritten to
all partitions by the transaction coordinator, but would also fence the
existing producer with the same AppId.

To make this workable, we need to make sure that the transaction
coordinator adds a sufficient logging so that we know the AppID -> PID
mapping as well as the partitions participating in each transaction. The
broker should also probably log information so that we know which
unfinished transaction (ie. which PID) is preventing the LSO from moving
forward. Both these things will make it fairly easy to configure the tool.

Of course, it is possible for the producer to continue onto another
transaction before the tool is run, in which case the data will be corrupt
since the second transaction will include messages from the first. But this
is no worse than Kafka's existing durability semantics which this proposal
relies on.

I think such a tool can be a follow up work, and I have added it to the
'future work' section of the document.



> 103. Currently, there is no check for producer liveness. This means that if
> a producer has not been sending transactional requests for a long time, its
> appId will be expired by the coordinator. Have we considered having
> producers sending heartbeatRequest just like the consumer to keep it alive?
>
>
In the current proposal, a producer whose AppId has expired is a Zombie
which will get a Fatal 'ProducerFencedException' when it tries to make any
new transactional requests. A bounce of the producer will reinitialize it,
at which point it can continue.

As such, while the proposed behavior is not ideal, I think that a heartbeat
thread would be a nice to have that may not be worth putting into V1. I
have made a note to add this in the 'Future Work' section of the document.



> 104. The logic for handling follower truncation can be a bit tricker now.
> The truncation may rewind the sequence number for some pids. The question
> is how to quickly recover the last sequence number of those pids. Do we
> plan to reload from a PID snapshot and scan forward?
>
>
Yes, this is exactly what we intend to do.


> 115. Message format:
> 115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
> since it's possible for a producer to be restarted 10s thousands of times?
> 115.2 Sequence number int32. Does it wrap after max? It's possible for a
> producer to publish more than 2 billion messages in a session.
>

Yes, both the epoch and the sequence number will have a circular space and
wrap around.

In the pure idempotent producer case, ie. where there is no AppId, each
producer session is guaranteed to have a unique PID and the leader for the
partition validates the incoming sequence number per PID before committing
to the log. So wrapping is safe.

In the transactional case, the transaction coordinator assigns the epoch
and returns the current epoch as part of the InitPIDRequest. All
transaction requests (Begin, AddTPToTransaction, Prepare) will be accepted
only if their epoch exactly matches the epoch recorded by the coordinator.
Wrap around is safe except in some degenerate cases like very long lived
zombie producers, described below.

Say we have a producer at epoch 20, who suddenly becomes a zombie. Assume
other producers with the same id are bounced for 65536 sessions (since
epoch are 2 bytes). Now after this period, the zombie producer comes back
online and would have the same PID/epoch as the current producer. In this
case, it is possible for both to produce transactions and for this
situation to go undetected.

We can solve this problem by including 'session ids' for producers and
validating that as well. But since the degenerate case is so rare, we think
that any such session id can be part of future work if it becomes a real
problem.

I have updated the doc to explicitly mention that the epoch and sequence
numbers may wrap.



> 119. InitPIDRequest
> <https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> 0wSw9ra8/edit#heading=h.z99xar1h2enr>
> : Should we write the completion of open transactions before append the pid
> with bumped up epoch to the transaction log?
>
>
Done.


> 120. transaction.app.id: An app may have multiple concurrent instances.
> Perhaps we should name it transaction.instance.id or just instance.id?
>
>
This has been the most debated point. How about just `producer.id`?



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

This makes sense. Doc has been updated.


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

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

Posted by Jun Rao <ju...@confluent.io>.
Thanks for the proposal. A few more detailed comments.

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

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

102. When there is a correlated hard failure (e.g., power outage), it's
possible that an existing commit/abort marker is lost in all replicas. This
may not be fixed by the transaction coordinator automatically and the
consumer may get stuck on that incomplete transaction forever. Not sure
what's the best way to address this. Perhaps, one way is to run a tool to
add an abort maker for all pids in all affected partitions.

103. Currently, there is no check for producer liveness. This means that if
a producer has not been sending transactional requests for a long time, its
appId will be expired by the coordinator. Have we considered having
producers sending heartbeatRequest just like the consumer to keep it alive?

104. The logic for handling follower truncation can be a bit tricker now.
The truncation may rewind the sequence number for some pids. The question
is how to quickly recover the last sequence number of those pids. Do we
plan to reload from a PID snapshot and scan forward?

105. When the transaction coordinator changes (due to leadership changes),
it's possible for both the old and the new coordinator sending requests to
a broker at the same time (for a short period of time). I am wondering if
we need to add logic to fence off the old coordinator. One way to do that
is to include the leaderEpoch of the partition associated with the
coordinator in the coordinator to broker request and control messages.

106. Compacted topics.
106.1. When all messages in a transaction are removed, we could remove the
commit/abort marker for that transaction too. However, we have to be a bit
careful. If the marker is removed too quickly, it's possible for a consumer
to see a message in that transaction, but not to see the marker, and
therefore will be stuck in that transaction forever. We have a similar
issue when dealing with tombstones. The solution is to preserve the
tombstone for at least a preconfigured amount of time after the cleaning
has passed the tombstone. Then, as long as a consumer can finish reading to
the cleaning point within the configured amount of time, it's guaranteed
not to miss the tombstone after it has seen a non-tombstone message on the
same key. I am wondering if we should do something similar here.
106.2. "To address this problem, we propose to preserve the last epoch and
sequence number written by each producer for a fixed amount of time as an
empty message set. This is allowed by the new message format we are
proposing in this document. The time to preserve the sequence number will
be governed by the log retention settings. " Could you be a bit more
specific on what retention time will be used since by default, there is no
retention time for compacted (but not delete) topic?
106.3 "As for control messages, if the broker does not have any
corresponding transaction cached with the PID when encountering a control
message, that message can be safely removed."
Do controlled messages have keys? If not, do we need to relax the
constraint that messages in a compacted topic must have keys?

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

108. Could you describe the format of the PID snapshot file?

109. Could you describe when Producer.send() will receive an UnrecognizedM
essageException?

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

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

112. Control message: Will control messages be used for timestamp indexing?
If so, what timestamp will we use if the timestamp type is creation time?

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

114.The section on Discussion on Pro-active Transaction Timeout: "If there
is no other instance with the same PID yet, or it has not started a
transaction, or it has not appended a message to some of the partitions,
then the zombie can continue appending messages to those partitions after
the abort marker whose epoch has not been incremented yet, but its commitTxn
call will fail."
Is that correct, in earlier discussion, it seems that if a transaction is
timed out by the coordinator, the coordinator will bump up epoch and write
the abort marker to those inserted partitions.

115. Message format:
115.1 Epoch is int16. Does it wrap after max? If not, is int16 too small
since it's possible for a producer to be restarted 10s thousands of times?
115.2 Sequence number int32. Does it wrap after max? It's possible for a
producer to publish more than 2 billion messages in a session.
115.3 "Null-value bit is 1: skip the key-length (since it can now be
calculated) and value fields." It seems that it's unnatural for the format
of key to depend on value. It seems it's easier to just skip value in this
case?

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

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

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

119. InitPIDRequest
<https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#heading=h.z99xar1h2enr>
: Should we write the completion of open transactions before append the pid
with bumped up epoch to the transaction log?

120. transaction.app.id: An app may have multiple concurrent instances.
Perhaps we should name it transaction.instance.id or just instance.id?

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

Thanks,

Jun


On Tue, Jan 3, 2017 at 5:38 PM, radai <ra...@gmail.com> wrote:

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

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

Posted by radai <ra...@gmail.com>.
@jun - good proposal. i was willing to concede that read-uncommitted was
impossible under my proposal but if LSO/NSO is introduced is becomes
possible.


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

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