Understood, thank you!
On Fri, Feb 10, 2017 at 2:54 PM, Apurva Mehta wrote:
> The point about management of transactionalId was raised before, and Jason
> tackled it here (note it was called 'AppId' back then):
> https://www.mail-archive.com/dev@kafka.apache.org/msg61200.html
>
> 1. Essentially,
The 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
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 uniqu
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 event
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, J
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 n
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
bloc
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 an
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
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 produce
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
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… peda
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
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
Hi Jason,
Thank you for the responses. Agree that authorizing transactional.id in the
producer requests will be good enough for version 1. And making it tighter
in future based on delegation tokens sounds good too.
Regards,
Rajini
On Fri, Feb 3, 2017 at 8:04 PM, Jason Gustafson wrote:
> Hey
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 permiss
Comments inline.
On Thu, Feb 2, 2017 at 6:28 PM, Jason Gustafson wrote:
> Took me a while to remember why we didn't do this. The timestamp that is
> included at the message set level is the max timestamp of all messages in
> the message set as is the case in the current message format (I will up
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 (b
Yes, I'd also prefer the option where we only have a checksum at the
message set level. I didn't suggest it due to the mentioned auditing use
cases, but if they can be satisfied in some other way, then that would be
great.
Ismael
On 2 Feb 2017 7:03 pm, "Jason Gustafson" wrote:
One more:
1. I d
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
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
Good point Tom. We will update the KIP with the ACLs section and also the
message format changes.
> On Feb 2, 2017, at 06:45, Tom Crayford wrote:
>
> I said this in the voting thread, but can the authors include a section
> about new ACLs if there are going to be ACLs for TransactionalId. It's
I said this in the voting thread, but can the authors include a section
about new ACLs if there are going to be ACLs for TransactionalId. It's
mentioned in the google doc, but I think new ACLs should be in a KIP
directly.
On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma wrote:
> Thanks for the respon
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
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
Ismael, thanks for your feedbacks. Replied inline.
On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma wrote:
> Hi all,
>
> A few comments follow:
>
> 1. The document states "inter-broker communications will be increased by M
> * N * P round trips per sec. We need to conduct some system performance
> te
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 b
> 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 syst
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 Gustaf
+1 for transactional.id.
-Jason
On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang wrote:
> If I have to choose between app.id and transactional.instance.id, I'd
> choose the latter.
>
> Renaming transactional.instance.id to transactional.id sounds even better.
>
>
> Guozhang
>
>
> On Mon, Jan 30,
If I have to choose between app.id and transactional.instance.id, I'd
choose the latter.
Renaming transactional.instance.id to transactional.id sounds even better.
Guozhang
On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta wrote:
> > Bumping one suggestion from Apurva above. The name "AppID" ha
> 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 -> tr
>
> 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
>>
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 fa
> 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 forev
Thanks Apurva - replies inline.
On 2017年01月27日 15:19, Apurva Mehta wrote:
Eugen, moving your email to the main thread so that it doesn't get split.
The `transaction.app.id` is a prerequisite for using transactional APIs.
And only messages wrapped inside transactions will enjoy idempotent
guaran
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 s
Hi, Jason,
Thanks for the reply. They sound good to me.
Jun
On Fri, Jan 27, 2017 at 4:42 PM, Jason Gustafson wrote:
> A few more responses:
>
>
> > 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> to
> > the new version, but the broker message format is still the old
Hi, Apurva,
Thanks for the update. My replies are inlined below.
On Wed, Jan 25, 2017 at 5:15 PM, Apurva Mehta wrote:
> Hi Jun,
>
> Thanks again for the comments. More responses follow:
>
>
> > 101. Compatibility during upgrade: Suppose that the brokers are upgraded
> to
> > the new version, bu
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 t
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 mar
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-pr
On Tue, Jan 17, 2017 at 6:17 PM, Apurva Mehta wrote:
> Hi Jun,
>
> Some answers in line.
>
>
> 109. Could you describe when Producer.send() will receive an Unrecognized
>
> MessageException?
>
>
> This exception will be thrown if the producer sends a sequence number
> which is greater than the se
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
>
Quick update: I have merged the abort index proposal linked above into the
main design document. We are now working on tying up the loose ends raised
by Jun and others.
Thanks,
Jason
On Tue, Jan 17, 2017 at 8:53 PM, Apurva Mehta wrote:
> >
> > 114.The section on Discussion on Pro-active Transac
>
> 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
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 pr
Hello,
Nacho and others had questions about the rationale behind the AppId and PID
design. In particular, why do we propose to have a broker-based map of
AppId to PID? Why can't we use the AppId directly? And if we need to use an
internal PID, why don't we do a static hashing of AppId to PID on th
Hi All,
We've been putting some thought into the need to buffer fetched data in the
consumer in the READ_COMMITTED isolation mode and have a proposal to
address the concern. The basic idea is to introduce an index to keep track
of the aborted transactions. We use this index to return in each fetch
Hi, Jason,
100. Yes, AppId level security is mainly for protecting the shared
transaction log. We could also include AppId in produce request (not in
message format) so that we could protect writes at the AppId level. I agree
that we need to support prefix matching on AppId for applications like
s
Hello all,
I updated the KIP document with the changes discussed here. In particular:
1. The consumer config was renamed from 'fetch.mode' to
'isolation.level'. The values were changed appropriately as well.
2. Clarified the transactional guarantees for the consumer.
3. Added a subse
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 ap
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
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 TransactionR
@jun - good proposal. i was willing to concede that read-uncommitted was
impossible under my proposal but if LSO/NSO is introduced is becomes
possible.
On Tue, Jan 3, 2017 at 7:50 AM, Jun Rao wrote:
> Just to follow up on Radai's idea of pushing the buffering logic to the
> broker. It may be po
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
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 hav
>
>
> @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
when the leader decides to commit a TX (of X msgs, known at this point), it
writes an "intent to append X msgs" msg (control?) followed by the X msgs
(at this point it is the leader and therefor point of sync, so this can be
done with no "foreign" msgs in between).
if there's a crash/change of lead
@Radai, regarding the replication for inflight transactional messages.
I think Jay and Joel have addressed the need for transactional messages to
be persisted durably at the moment we enter the pre-commit phase. If we
don't have durable persistence of these messages, we can't have idempotent
and a
I don't think the simple approach of writing to a local store (in memory or
on disk) and then copying out to the destination topics would work but
there could well be more sophisticated things that would. As you say, it is
fine for the data to be un-replicated while you are accumulating the
transac
obviously anything committed would need to be replicated to all followers -
just like current msgs.
what im trying to say is that in-flight data (written as part of an ongoing
TX and not committed yet) does not necessarily need to be replicated, or
even written out to disk. taken to the extreme it
Cool. It sounds like you guys will sync up and come up with a specific
proposal. I think point (3) does require full replication of the pre-commit
transaction, but I'm not sure, and I would be very happy to learn
otherwise. That was actually the blocker on that alternate proposal. From
my point of
Just got some time to go through most of this thread and KIP - great to see
this materialize and discussed!!
I will add more comments in the coming days on some of the other "tracks"
in this thread; but since Radai brought up the double-journaling approach
that we had discussed I thought I would mo
1. on write amplification: i dont see x6 the writes, at worst i see x2 the
writes - once to the "tx log", then read and again to the destination
partition. if you have some != 1 replication factor than both the 1st and
the 2nd writes get replicated, but it is still a relative factor of x2.
what am
One more thing about the double journal proposal: when discussing about
this method back at LinkedIn, another raised issue besides double writing
was that it will void the offset ordering and enforce people to accept
"transaction ordering", that is, consumer will not see messages from the
same part
small correction in my third point -
3. IO and memory constraints - We would want a solution that *does not take*
2x the number of writes.
On Mon, Dec 19, 2016 at 12:37 PM, Sriram Subramanian
wrote:
> Radai,
>
> I think it is important to understand the key requirements that we don’t
> want to
Radai,
I think it is important to understand the key requirements that we don’t
want to compromise. We can then understand the tradeoffs of the different
approaches. We did in fact start with the double journal approach couple of
years back. I will highlight the must have requirements first and th
Hey Radai,
I'm not sure if I fully understand what you are proposing, but I
interpreted it to be similar to a proposal we worked through back at
LinkedIn. The proposal was to commit to a central txlog topic, and then
recopy to the destination topic upon transaction commit. The observation on
that
regarding efficiency:
I'd like to distinguish between server efficiency (resource utilization of
the broker machine alone) and overall network efficiency (resource
utilization on brokers, producers and consumers, including network traffic).
my proposal is not as resource-efficient on the broker (a
@Jason
Yes, second thought on the number of messages included, the offset delta
will probably be sufficient. The use case I encounter before for number of
messages in a message set is an embedded mirror maker on the destination
broker side which fetches message directly from the source cluster. Id
Yeah good point. I relent!
-jay
On Fri, Dec 16, 2016 at 1:46 PM Jason Gustafson wrote:
> Jay/Ismael,
>
>
>
> I agree that lazy initialization of metadata seems unavoidable. Ideally, we
>
> could follow the same pattern for transactions, but remember that in the
>
> consumer+producer use case, t
Jay/Ismael,
I agree that lazy initialization of metadata seems unavoidable. Ideally, we
could follow the same pattern for transactions, but remember that in the
consumer+producer use case, the initialization needs to be completed prior
to setting the consumer's position. Otherwise we risk reading
a few more points:
1. "we can make that efficient by storing only the 'start offsets' of
transactions and then replaying the log once we hit the corresponding
commit/abort markers"
this is efficient in terms of memory consumption on the consumer. it is
definitely not efficient when i consider wha
Hi Apurva,
here's an outline of what I had in mind:
1. TXs in progress are written "sideways" on the partition leader. this can
be (in order of increasing reliability and decreasing performance):
1.1 an im-mem buffer on the partition leader (TX lost on broker crash,
memory limits need to be
Hi Radai,
Thanks for your email. You raise some good points.
Your larger point centers on your --correct!-- observation that persisting
messages belonging to aborted transactions causes complications for
consumers.
You are right in observing that the current proposal does require consumers
to bu
some clarifications on my alternative proposal:
TX msgs are written "sideways" to a transaction (ad-hoc) partition. this
partition can be replicated to followers, or can be an in-mem buffer -
depends on the resilience guarantees you want to provide for TXs in case of
broker crash.
on "commit" the
I can see several issues with the current proposal.
messages, even if sent under a TX, are delivered directly to their
destination partitions, downstream consumers need to be TX-aware. they can
either:
1. be oblivious to TXs. that means they will deliver "garbage" - msgs
sent during eventually-
Hi Apurva,
Thank you, makes sense.
Rajini
On Wed, Dec 14, 2016 at 7:36 PM, Apurva Mehta wrote:
> Hi Rajini,
>
> I think my original response to your point 15 was not accurate. The regular
> definition of durability is that data once committed would never be lost.
> So it is not enough for only
Hi Rajini,
I think my original response to your point 15 was not accurate. The regular
definition of durability is that data once committed would never be lost.
So it is not enough for only the control messages to be flushed before
being acknowledged -- all the messages (and offset commits) which
Hi Ben,
You are are right on both counts:
Writing apps to do consume-process-produce batching will be tricky to
program using this API directly. The expectation is that 99% of the users
would use the streams API to leverage this functionality, and that API will
take care of the details. This seem
Hi Apurva,
Thank you for the answers. Just one follow-on.
15. Let me rephrase my original question. If all control messages (messages
to transaction logs and markers on user logs) were acknowledged only after
flushing the log segment, will transactions become durable in the
traditional sense (i.e
Hi Apurva,
Thanks for the detailed answers... and sorry for the late reply!
It does sound like, if the input-partitions-to-app-id mapping never
changes, the existing fencing mechanisms should prevent duplicates. Great!
I'm a bit concerned the proposed API will be delicate to program against
succe
ver flushed to disks, then your data is lost today, and it
> > > will
> > > >> be still the case after KIP-98.
> > > >>
> > > >> As for atomicity, KIP-98 does provide all-or-nothing guarantee for
> > > writes
> > > >> to mul
Kafka: if all your replicas are gone at the same time before
> > the
> > > >> data was ever flushed to disks, then your data is lost today, and it
> > > will
> > > >> be still the case after KIP-98.
> > > >>
> > > >> As for
;s
> messages
> > >> could be lost if the above scenarios happen while others can be
> > >> successfully appended. My take is that, if you have concerns that
> > Kafka's
> > >> replication mechanism i not good enough for your durability
> requirements
> > >
gt; >> as well.
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Mon, Dec 12, 2016 at 1:49 AM, Andrew Schofield <
> >> andrew_schofi...@live.com
> >> > wrote:
> >>
> >> > Guozhang,
> >> > Exactly. This is
ut of date wrt to the run-time state and a failure of all
>> > replicas might
>> > take the data slightly back in time.
>> >
>> > Given this, do you think that KIP-98 gives an all-or-nothing,
>> > no-matter-what guarantee
>> > for Kafka transactions? I t
lly in all cases.
> > Asynchronous but
> > atomic would be good.
> >
> > Andrew Schofield
> > IBM Watson and Cloud Platform
> >
> >
> > >
> > > From: Guozhang Wang
> > > Sent: 09 December 2016 22:59
> > > To: dev@kafka.
gt; asynchronously
> flushed is guaranteed to be recovered atomically in all cases.
> Asynchronous but
> atomic would be good.
>
> Andrew Schofield
> IBM Watson and Cloud Platform
>
>
> >
> > From: Guozhang Wang
> > Sent: 09 December 2016 22:59
> > To:
t;> > onurkaraman.apa...@gmail.com>
> >> > wrote:
> >> >
> >> > > In other words, we can see inconsistency when the transaction log
> >> reports
> >> > > the transaction as COMMITTED while the markers and data
> corresponding
ber 2016 22:59
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Onur,
>
> I understand your question now. So it is indeed possible that after
> commitTxn() returned the messages could still be lost permanent
BORT
>> > > > markers (and a subset of the user's data produced in the transaction
>> > for
>> > > > that matter) getting lost from the delta between the write and
>> flush.
>> > > >
>> > > > KIP-98 relie
t; > corner-case durability scenario where the single replica
> participating
> > in
> > > > the transaction experiences a hard failure after the write but before
> > the
> > > > flush, causing the transaction to have partial data loss.
> > > >
&g
gt;> enough of a case. As stated for us I think for these systems we will
> > keep
> > >> our JMS solutions there. I think maybe in the docs when this feature
> is
> > >> written up, one should redirect users to alternative options such as
> jms
> > &g
n
> > optional "written and flushed to disk".
> >
> > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >>
> >&g
> On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce
> > wrote:
> >
> >> Apologies on the spelling.
> >>
> >> *Hi Jay,
> >>
> >> From: Michael Pearce
> >> Sent: Friday, December 9, 2016 7:52:25
ologies on the spelling.
>>
>> *Hi Jay,
>> ____________________
>> From: Michael Pearce
>> Sent: Friday, December 9, 2016 7:52:25 PM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
o developers expecting
> > similar behaviour and will cause issues in prod when they find it doesn't
> > work as they're used to.
> >
> > As my other response earlier, is there a better name to describe this
> > feature, if we're not implementing transact
Apologies on the spelling.
*Hi Jay,
From: Michael Pearce
Sent: Friday, December 9, 2016 7:52:25 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging
Hi Jey
1) I agree, these should be used to add
1 - 100 of 149 matches
Mail list logo