Re: Attempt to prove Kafka transactions work

2019-11-27 Thread Edward Capriolo
On Wed, Nov 20, 2019 at 6:35 PM Edward Capriolo 
wrote:

>
>
> On Wednesday, November 20, 2019, Matthias J. Sax 
> wrote:
>
>> I am not sure what Spring does, but using Kafka Streams writing the
>> output and committing offset would be part of the same transaction.
>>
>> It seems Spring is doing something else and thus, is seems it does not
>> use the EOS API correctly.
>>
>> If you use transactions to copy data from input to output topic,
>> committing offsets must be done on the producer as part to the
>> transaction; the consumer would not commit offsets.
>>
>> To me, it seems that Spring is committing offset using the consumer
>> independent if the transaction was successful or not. This would be an
>> incorrect usage of the API.
>>
>>
>> -Matthias
>>
>> On 11/20/19 6:16 AM, Edward Capriolo wrote:
>> > Ok. I'm at a point where I believe the exactly once is in question.
>> >
>> > Topic input 10 partitions topic output 10 partitions.
>> >
>> > Producer writes messages 1 to 100 to topic input.
>> >
>> > CTP process calls poll. It receives 100 messages 10 in each partiton.
>> >
>> > Process is simple mirroring take from input write to output.
>> >
>> > 10 producers with 10 transactional ids are created. During processing 1
>> of
>> > the 10 producers throws kafka exception. 90 out of 100 writes are
>> committed
>> > tranactionally 10 are not.
>> >
>> > If poll is called again 10 messages do not appear in the next poll. Are
>> > they lost?
>> >
>> >
>> >
>> > On Saturday, November 9, 2019, Edward Capriolo 
>> > wrote:
>> >
>> >>
>> >> On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax > >
>> >> wrote:
>> >>
>> >>> Quite a project to test transactions...
>> >>>
>> >>> The current system test suite is part of the code base:
>> >>> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
>> >>>
>> >>> There is course also some unit/integration test for transactions.
>> >>>
>> >>> There is also a blog post that describes in a high level what testing
>> >>> was done when EOS was introduced:
>> >>> https://www.confluent.io/blog/exactly-once-semantics-are-
>> >>> possible-heres-how-apache-kafka-does-it/
>> >>>
>> >>> And yes, transactions are built on some assumptions and if you
>> configure
>> >>> your system incorrectly of violate those assumptions, it may break. We
>> >>> also fixed some bugs since the first release. And there might be more
>> >>> bugs --- software is always buggy. However, for practical
>> consideration,
>> >>> transactions should work.
>> >>>
>> >>> We would of course love if you could share your test results! If you
>> >>> discover a bug, please report it, so we can fix it.
>> >>>
>> >>>
>> >>> -Matthias
>> >>>
>> >>> On 10/28/19 10:06 AM, Edward Capriolo wrote:
>>  On Sunday, October 27, 2019, Boyang Chen > >
>> >>> wrote:
>> 
>> > Hey Edward,
>> >
>> > just to summarize and make sure I understood your question, you
>> want to
>> > implement some Chaos testing to validate Kafka EOS model, but not
>> sure
>> >>> how
>> > to start or curious about whether there are already works in the
>> >>> community
>> > doing that?
>> >
>> > For the correctness of Kafka EOS, we have tons of unit tests and
>> system
>> > tests to prove its functionality. They could be found inside the
>> repo.
>> >>> You
>> > could check them out and see if we still have gaps (which I believe
>> we
>> > definitely have).
>> >
>> > Boyang
>> >
>> > On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo <
>> edlinuxg...@gmail.com
>> 
>> > wrote:
>> >
>> >> Hello all,
>> >>
>> >> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
>> >> thousand ad impression. If numbers are 5% off you can blame
>> javascript
>> >> click trackers.
>> >>
>> >> Now, I work in a non addtech industry and they are really, really
>> >>> serious
>> >> about exactly once.
>> >>
>> >> So there is this blog:
>> >>
>> >> https://www.confluent.io/blog/transactions-apache-kafka/
>> >>
>> >> Great little snippet of code. I think I can copy it and implement
>> it
>> >> correctly.
>> >> You know, but if you read that section about the zombie fencing,
>> you
>> > learn
>> >> that you either need to manually assign partitions or use the
>> >>> rebalance
>> >> listener and have N producers. Hunting around github is not super
>> > helpful,
>> >> some code snippets are less complete then even the snipped in the
>> >>> blog.
>> >>
>> >> I looked at what spring-kafka does. It does get the zombie fencing
>> > correct
>> >> with respect to fencing id, other bits other bits of the code seem
>> >> plausible.
>> >>
>> >> Notice I said "plausible", because I do not count a few end to end
>> >>> tests
>> >> running single VM as a solid enough evidence that this works in the
>> >>> face
>> > of
>> >> failures.
>> >>
>> >> I have been contemplating 

Re: Attempt to prove Kafka transactions work

2019-11-20 Thread Edward Capriolo
On Wednesday, November 20, 2019, Matthias J. Sax 
wrote:

> I am not sure what Spring does, but using Kafka Streams writing the
> output and committing offset would be part of the same transaction.
>
> It seems Spring is doing something else and thus, is seems it does not
> use the EOS API correctly.
>
> If you use transactions to copy data from input to output topic,
> committing offsets must be done on the producer as part to the
> transaction; the consumer would not commit offsets.
>
> To me, it seems that Spring is committing offset using the consumer
> independent if the transaction was successful or not. This would be an
> incorrect usage of the API.
>
>
> -Matthias
>
> On 11/20/19 6:16 AM, Edward Capriolo wrote:
> > Ok. I'm at a point where I believe the exactly once is in question.
> >
> > Topic input 10 partitions topic output 10 partitions.
> >
> > Producer writes messages 1 to 100 to topic input.
> >
> > CTP process calls poll. It receives 100 messages 10 in each partiton.
> >
> > Process is simple mirroring take from input write to output.
> >
> > 10 producers with 10 transactional ids are created. During processing 1
> of
> > the 10 producers throws kafka exception. 90 out of 100 writes are
> committed
> > tranactionally 10 are not.
> >
> > If poll is called again 10 messages do not appear in the next poll. Are
> > they lost?
> >
> >
> >
> > On Saturday, November 9, 2019, Edward Capriolo 
> > wrote:
> >
> >>
> >> On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax 
> >> wrote:
> >>
> >>> Quite a project to test transactions...
> >>>
> >>> The current system test suite is part of the code base:
> >>> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
> >>>
> >>> There is course also some unit/integration test for transactions.
> >>>
> >>> There is also a blog post that describes in a high level what testing
> >>> was done when EOS was introduced:
> >>> https://www.confluent.io/blog/exactly-once-semantics-are-
> >>> possible-heres-how-apache-kafka-does-it/
> >>>
> >>> And yes, transactions are built on some assumptions and if you
> configure
> >>> your system incorrectly of violate those assumptions, it may break. We
> >>> also fixed some bugs since the first release. And there might be more
> >>> bugs --- software is always buggy. However, for practical
> consideration,
> >>> transactions should work.
> >>>
> >>> We would of course love if you could share your test results! If you
> >>> discover a bug, please report it, so we can fix it.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 10/28/19 10:06 AM, Edward Capriolo wrote:
>  On Sunday, October 27, 2019, Boyang Chen 
> >>> wrote:
> 
> > Hey Edward,
> >
> > just to summarize and make sure I understood your question, you want
> to
> > implement some Chaos testing to validate Kafka EOS model, but not
> sure
> >>> how
> > to start or curious about whether there are already works in the
> >>> community
> > doing that?
> >
> > For the correctness of Kafka EOS, we have tons of unit tests and
> system
> > tests to prove its functionality. They could be found inside the
> repo.
> >>> You
> > could check them out and see if we still have gaps (which I believe
> we
> > definitely have).
> >
> > Boyang
> >
> > On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo <
> edlinuxg...@gmail.com
> 
> > wrote:
> >
> >> Hello all,
> >>
> >> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
> >> thousand ad impression. If numbers are 5% off you can blame
> javascript
> >> click trackers.
> >>
> >> Now, I work in a non addtech industry and they are really, really
> >>> serious
> >> about exactly once.
> >>
> >> So there is this blog:
> >>
> >> https://www.confluent.io/blog/transactions-apache-kafka/
> >>
> >> Great little snippet of code. I think I can copy it and implement it
> >> correctly.
> >> You know, but if you read that section about the zombie fencing, you
> > learn
> >> that you either need to manually assign partitions or use the
> >>> rebalance
> >> listener and have N producers. Hunting around github is not super
> > helpful,
> >> some code snippets are less complete then even the snipped in the
> >>> blog.
> >>
> >> I looked at what spring-kafka does. It does get the zombie fencing
> > correct
> >> with respect to fencing id, other bits other bits of the code seem
> >> plausible.
> >>
> >> Notice I said "plausible", because I do not count a few end to end
> >>> tests
> >> running single VM as a solid enough evidence that this works in the
> >>> face
> > of
> >> failures.
> >>
> >> I have been contemplating how one stress tests this exactly once
> >>> concept,
> >> with something Jepsen like or something brute force that I can run
> >>> for 5
> >> hours in a row
> >>
> >> If I faithfully implemented the code 

Re: Attempt to prove Kafka transactions work

2019-11-20 Thread Edward Capriolo
On Wednesday, November 20, 2019, Eric Azama  wrote:

> Calls to KafkaConsumer#poll() are completely independent of commits. As
> such they will always return the next set of records, even if the previous
> set have not been committed. This is how the consumer acts, regardless of
> the Exactly Once semantics.
>
> In order for the Consumer to reset to the currently committed offsets, you
> need to either initiate a Consumer Group Rebalance, or use a combination of
> the KafkaConsumer#committed() and KafkaConsumer#seek() methods.
>
> On Wed, Nov 20, 2019 at 6:16 AM Edward Capriolo 
> wrote:
>
> > Ok. I'm at a point where I believe the exactly once is in question.
> >
> > Topic input 10 partitions topic output 10 partitions.
> >
> > Producer writes messages 1 to 100 to topic input.
> >
> > CTP process calls poll. It receives 100 messages 10 in each partiton.
> >
> > Process is simple mirroring take from input write to output.
> >
> > 10 producers with 10 transactional ids are created. During processing 1
> of
> > the 10 producers throws kafka exception. 90 out of 100 writes are
> committed
> > tranactionally 10 are not.
> >
> > If poll is called again 10 messages do not appear in the next poll. Are
> > they lost?
> >
> >
> >
> > On Saturday, November 9, 2019, Edward Capriolo 
> > wrote:
> >
> > >
> > > On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax  >
> > > wrote:
> > >
> > >> Quite a project to test transactions...
> > >>
> > >> The current system test suite is part of the code base:
> > >> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
> > >>
> > >> There is course also some unit/integration test for transactions.
> > >>
> > >> There is also a blog post that describes in a high level what testing
> > >> was done when EOS was introduced:
> > >> https://www.confluent.io/blog/exactly-once-semantics-are-
> > >> possible-heres-how-apache-kafka-does-it/
> > >>
> > >> And yes, transactions are built on some assumptions and if you
> configure
> > >> your system incorrectly of violate those assumptions, it may break. We
> > >> also fixed some bugs since the first release. And there might be more
> > >> bugs --- software is always buggy. However, for practical
> consideration,
> > >> transactions should work.
> > >>
> > >> We would of course love if you could share your test results! If you
> > >> discover a bug, please report it, so we can fix it.
> > >>
> > >>
> > >> -Matthias
> > >>
> > >> On 10/28/19 10:06 AM, Edward Capriolo wrote:
> > >> > On Sunday, October 27, 2019, Boyang Chen <
> reluctanthero...@gmail.com>
> > >> wrote:
> > >> >
> > >> >> Hey Edward,
> > >> >>
> > >> >> just to summarize and make sure I understood your question, you
> want
> > to
> > >> >> implement some Chaos testing to validate Kafka EOS model, but not
> > sure
> > >> how
> > >> >> to start or curious about whether there are already works in the
> > >> community
> > >> >> doing that?
> > >> >>
> > >> >> For the correctness of Kafka EOS, we have tons of unit tests and
> > system
> > >> >> tests to prove its functionality. They could be found inside the
> > repo.
> > >> You
> > >> >> could check them out and see if we still have gaps (which I believe
> > we
> > >> >> definitely have).
> > >> >>
> > >> >> Boyang
> > >> >>
> > >> >> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo <
> > edlinuxg...@gmail.com
> > >> >
> > >> >> wrote:
> > >> >>
> > >> >>> Hello all,
> > >> >>>
> > >> >>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5
> per
> > >> >>> thousand ad impression. If numbers are 5% off you can blame
> > javascript
> > >> >>> click trackers.
> > >> >>>
> > >> >>> Now, I work in a non addtech industry and they are really, really
> > >> serious
> > >> >>> about exactly once.
> > >> >>>
> > >> >>> So there is this blog:
> > >> >>>
> > >> >>> https://www.confluent.io/blog/transactions-apache-kafka/
> > >> >>>
> > >> >>> Great little snippet of code. I think I can copy it and implement
> it
> > >> >>> correctly.
> > >> >>> You know, but if you read that section about the zombie fencing,
> you
> > >> >> learn
> > >> >>> that you either need to manually assign partitions or use the
> > >> rebalance
> > >> >>> listener and have N producers. Hunting around github is not super
> > >> >> helpful,
> > >> >>> some code snippets are less complete then even the snipped in the
> > >> blog.
> > >> >>>
> > >> >>> I looked at what spring-kafka does. It does get the zombie fencing
> > >> >> correct
> > >> >>> with respect to fencing id, other bits other bits of the code seem
> > >> >>> plausible.
> > >> >>>
> > >> >>> Notice I said "plausible", because I do not count a few end to end
> > >> tests
> > >> >>> running single VM as a solid enough evidence that this works in
> the
> > >> face
> > >> >> of
> > >> >>> failures.
> > >> >>>
> > >> >>> I have been contemplating how one stress tests this exactly once
> > >> concept,
> > >> >>> with something Jepsen like or something brute force that I can run
> > >> for 5
> > >> 

Re: Attempt to prove Kafka transactions work

2019-11-20 Thread Matthias J. Sax
I am not sure what Spring does, but using Kafka Streams writing the
output and committing offset would be part of the same transaction.

It seems Spring is doing something else and thus, is seems it does not
use the EOS API correctly.

If you use transactions to copy data from input to output topic,
committing offsets must be done on the producer as part to the
transaction; the consumer would not commit offsets.

To me, it seems that Spring is committing offset using the consumer
independent if the transaction was successful or not. This would be an
incorrect usage of the API.


-Matthias

On 11/20/19 6:16 AM, Edward Capriolo wrote:
> Ok. I'm at a point where I believe the exactly once is in question.
> 
> Topic input 10 partitions topic output 10 partitions.
> 
> Producer writes messages 1 to 100 to topic input.
> 
> CTP process calls poll. It receives 100 messages 10 in each partiton.
> 
> Process is simple mirroring take from input write to output.
> 
> 10 producers with 10 transactional ids are created. During processing 1 of
> the 10 producers throws kafka exception. 90 out of 100 writes are committed
> tranactionally 10 are not.
> 
> If poll is called again 10 messages do not appear in the next poll. Are
> they lost?
> 
> 
> 
> On Saturday, November 9, 2019, Edward Capriolo 
> wrote:
> 
>>
>> On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax 
>> wrote:
>>
>>> Quite a project to test transactions...
>>>
>>> The current system test suite is part of the code base:
>>> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
>>>
>>> There is course also some unit/integration test for transactions.
>>>
>>> There is also a blog post that describes in a high level what testing
>>> was done when EOS was introduced:
>>> https://www.confluent.io/blog/exactly-once-semantics-are-
>>> possible-heres-how-apache-kafka-does-it/
>>>
>>> And yes, transactions are built on some assumptions and if you configure
>>> your system incorrectly of violate those assumptions, it may break. We
>>> also fixed some bugs since the first release. And there might be more
>>> bugs --- software is always buggy. However, for practical consideration,
>>> transactions should work.
>>>
>>> We would of course love if you could share your test results! If you
>>> discover a bug, please report it, so we can fix it.
>>>
>>>
>>> -Matthias
>>>
>>> On 10/28/19 10:06 AM, Edward Capriolo wrote:
 On Sunday, October 27, 2019, Boyang Chen 
>>> wrote:

> Hey Edward,
>
> just to summarize and make sure I understood your question, you want to
> implement some Chaos testing to validate Kafka EOS model, but not sure
>>> how
> to start or curious about whether there are already works in the
>>> community
> doing that?
>
> For the correctness of Kafka EOS, we have tons of unit tests and system
> tests to prove its functionality. They could be found inside the repo.
>>> You
> could check them out and see if we still have gaps (which I believe we
> definitely have).
>
> Boyang
>
> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo >>>
> wrote:
>
>> Hello all,
>>
>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
>> thousand ad impression. If numbers are 5% off you can blame javascript
>> click trackers.
>>
>> Now, I work in a non addtech industry and they are really, really
>>> serious
>> about exactly once.
>>
>> So there is this blog:
>>
>> https://www.confluent.io/blog/transactions-apache-kafka/
>>
>> Great little snippet of code. I think I can copy it and implement it
>> correctly.
>> You know, but if you read that section about the zombie fencing, you
> learn
>> that you either need to manually assign partitions or use the
>>> rebalance
>> listener and have N producers. Hunting around github is not super
> helpful,
>> some code snippets are less complete then even the snipped in the
>>> blog.
>>
>> I looked at what spring-kafka does. It does get the zombie fencing
> correct
>> with respect to fencing id, other bits other bits of the code seem
>> plausible.
>>
>> Notice I said "plausible", because I do not count a few end to end
>>> tests
>> running single VM as a solid enough evidence that this works in the
>>> face
> of
>> failures.
>>
>> I have been contemplating how one stress tests this exactly once
>>> concept,
>> with something Jepsen like or something brute force that I can run
>>> for 5
>> hours in a row
>>
>> If I faithfully implemented the code in the transactional read-write
>>> loop
>> and I feed it into my jepsen like black box tester it should:
>>
>> Create a topic with 10 partitions, Start launching read-write
>>> transaction
>> code, start feeding input data, Maybe strings like 1 -1000, now start
>> randomly killing vms with kill -9 kill graceful exits, maybe even
>>> killing

Re: Attempt to prove Kafka transactions work

2019-11-20 Thread Eric Azama
Calls to KafkaConsumer#poll() are completely independent of commits. As
such they will always return the next set of records, even if the previous
set have not been committed. This is how the consumer acts, regardless of
the Exactly Once semantics.

In order for the Consumer to reset to the currently committed offsets, you
need to either initiate a Consumer Group Rebalance, or use a combination of
the KafkaConsumer#committed() and KafkaConsumer#seek() methods.

On Wed, Nov 20, 2019 at 6:16 AM Edward Capriolo 
wrote:

> Ok. I'm at a point where I believe the exactly once is in question.
>
> Topic input 10 partitions topic output 10 partitions.
>
> Producer writes messages 1 to 100 to topic input.
>
> CTP process calls poll. It receives 100 messages 10 in each partiton.
>
> Process is simple mirroring take from input write to output.
>
> 10 producers with 10 transactional ids are created. During processing 1 of
> the 10 producers throws kafka exception. 90 out of 100 writes are committed
> tranactionally 10 are not.
>
> If poll is called again 10 messages do not appear in the next poll. Are
> they lost?
>
>
>
> On Saturday, November 9, 2019, Edward Capriolo 
> wrote:
>
> >
> > On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax 
> > wrote:
> >
> >> Quite a project to test transactions...
> >>
> >> The current system test suite is part of the code base:
> >> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
> >>
> >> There is course also some unit/integration test for transactions.
> >>
> >> There is also a blog post that describes in a high level what testing
> >> was done when EOS was introduced:
> >> https://www.confluent.io/blog/exactly-once-semantics-are-
> >> possible-heres-how-apache-kafka-does-it/
> >>
> >> And yes, transactions are built on some assumptions and if you configure
> >> your system incorrectly of violate those assumptions, it may break. We
> >> also fixed some bugs since the first release. And there might be more
> >> bugs --- software is always buggy. However, for practical consideration,
> >> transactions should work.
> >>
> >> We would of course love if you could share your test results! If you
> >> discover a bug, please report it, so we can fix it.
> >>
> >>
> >> -Matthias
> >>
> >> On 10/28/19 10:06 AM, Edward Capriolo wrote:
> >> > On Sunday, October 27, 2019, Boyang Chen 
> >> wrote:
> >> >
> >> >> Hey Edward,
> >> >>
> >> >> just to summarize and make sure I understood your question, you want
> to
> >> >> implement some Chaos testing to validate Kafka EOS model, but not
> sure
> >> how
> >> >> to start or curious about whether there are already works in the
> >> community
> >> >> doing that?
> >> >>
> >> >> For the correctness of Kafka EOS, we have tons of unit tests and
> system
> >> >> tests to prove its functionality. They could be found inside the
> repo.
> >> You
> >> >> could check them out and see if we still have gaps (which I believe
> we
> >> >> definitely have).
> >> >>
> >> >> Boyang
> >> >>
> >> >> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo <
> edlinuxg...@gmail.com
> >> >
> >> >> wrote:
> >> >>
> >> >>> Hello all,
> >> >>>
> >> >>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
> >> >>> thousand ad impression. If numbers are 5% off you can blame
> javascript
> >> >>> click trackers.
> >> >>>
> >> >>> Now, I work in a non addtech industry and they are really, really
> >> serious
> >> >>> about exactly once.
> >> >>>
> >> >>> So there is this blog:
> >> >>>
> >> >>> https://www.confluent.io/blog/transactions-apache-kafka/
> >> >>>
> >> >>> Great little snippet of code. I think I can copy it and implement it
> >> >>> correctly.
> >> >>> You know, but if you read that section about the zombie fencing, you
> >> >> learn
> >> >>> that you either need to manually assign partitions or use the
> >> rebalance
> >> >>> listener and have N producers. Hunting around github is not super
> >> >> helpful,
> >> >>> some code snippets are less complete then even the snipped in the
> >> blog.
> >> >>>
> >> >>> I looked at what spring-kafka does. It does get the zombie fencing
> >> >> correct
> >> >>> with respect to fencing id, other bits other bits of the code seem
> >> >>> plausible.
> >> >>>
> >> >>> Notice I said "plausible", because I do not count a few end to end
> >> tests
> >> >>> running single VM as a solid enough evidence that this works in the
> >> face
> >> >> of
> >> >>> failures.
> >> >>>
> >> >>> I have been contemplating how one stress tests this exactly once
> >> concept,
> >> >>> with something Jepsen like or something brute force that I can run
> >> for 5
> >> >>> hours in a row
> >> >>>
> >> >>> If I faithfully implemented the code in the transactional read-write
> >> loop
> >> >>> and I feed it into my jepsen like black box tester it should:
> >> >>>
> >> >>> Create a topic with 10 partitions, Start launching read-write
> >> transaction
> >> >>> code, start feeding input data, Maybe strings like 1 -1000, now
> start
> >> >>> 

Re: Attempt to prove Kafka transactions work

2019-11-20 Thread Edward Capriolo
Ok. I'm at a point where I believe the exactly once is in question.

Topic input 10 partitions topic output 10 partitions.

Producer writes messages 1 to 100 to topic input.

CTP process calls poll. It receives 100 messages 10 in each partiton.

Process is simple mirroring take from input write to output.

10 producers with 10 transactional ids are created. During processing 1 of
the 10 producers throws kafka exception. 90 out of 100 writes are committed
tranactionally 10 are not.

If poll is called again 10 messages do not appear in the next poll. Are
they lost?



On Saturday, November 9, 2019, Edward Capriolo 
wrote:

>
> On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax 
> wrote:
>
>> Quite a project to test transactions...
>>
>> The current system test suite is part of the code base:
>> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
>>
>> There is course also some unit/integration test for transactions.
>>
>> There is also a blog post that describes in a high level what testing
>> was done when EOS was introduced:
>> https://www.confluent.io/blog/exactly-once-semantics-are-
>> possible-heres-how-apache-kafka-does-it/
>>
>> And yes, transactions are built on some assumptions and if you configure
>> your system incorrectly of violate those assumptions, it may break. We
>> also fixed some bugs since the first release. And there might be more
>> bugs --- software is always buggy. However, for practical consideration,
>> transactions should work.
>>
>> We would of course love if you could share your test results! If you
>> discover a bug, please report it, so we can fix it.
>>
>>
>> -Matthias
>>
>> On 10/28/19 10:06 AM, Edward Capriolo wrote:
>> > On Sunday, October 27, 2019, Boyang Chen 
>> wrote:
>> >
>> >> Hey Edward,
>> >>
>> >> just to summarize and make sure I understood your question, you want to
>> >> implement some Chaos testing to validate Kafka EOS model, but not sure
>> how
>> >> to start or curious about whether there are already works in the
>> community
>> >> doing that?
>> >>
>> >> For the correctness of Kafka EOS, we have tons of unit tests and system
>> >> tests to prove its functionality. They could be found inside the repo.
>> You
>> >> could check them out and see if we still have gaps (which I believe we
>> >> definitely have).
>> >>
>> >> Boyang
>> >>
>> >> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo > >
>> >> wrote:
>> >>
>> >>> Hello all,
>> >>>
>> >>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
>> >>> thousand ad impression. If numbers are 5% off you can blame javascript
>> >>> click trackers.
>> >>>
>> >>> Now, I work in a non addtech industry and they are really, really
>> serious
>> >>> about exactly once.
>> >>>
>> >>> So there is this blog:
>> >>>
>> >>> https://www.confluent.io/blog/transactions-apache-kafka/
>> >>>
>> >>> Great little snippet of code. I think I can copy it and implement it
>> >>> correctly.
>> >>> You know, but if you read that section about the zombie fencing, you
>> >> learn
>> >>> that you either need to manually assign partitions or use the
>> rebalance
>> >>> listener and have N producers. Hunting around github is not super
>> >> helpful,
>> >>> some code snippets are less complete then even the snipped in the
>> blog.
>> >>>
>> >>> I looked at what spring-kafka does. It does get the zombie fencing
>> >> correct
>> >>> with respect to fencing id, other bits other bits of the code seem
>> >>> plausible.
>> >>>
>> >>> Notice I said "plausible", because I do not count a few end to end
>> tests
>> >>> running single VM as a solid enough evidence that this works in the
>> face
>> >> of
>> >>> failures.
>> >>>
>> >>> I have been contemplating how one stress tests this exactly once
>> concept,
>> >>> with something Jepsen like or something brute force that I can run
>> for 5
>> >>> hours in a row
>> >>>
>> >>> If I faithfully implemented the code in the transactional read-write
>> loop
>> >>> and I feed it into my jepsen like black box tester it should:
>> >>>
>> >>> Create a topic with 10 partitions, Start launching read-write
>> transaction
>> >>> code, start feeding input data, Maybe strings like 1 -1000, now start
>> >>> randomly killing vms with kill -9 kill graceful exits, maybe even
>> killing
>> >>> kafka, and make sure 1-1000, pop out on the other end.
>> >>>
>> >>> I thought of some other "crazy" ideas. One such idea:
>> >>>
>> >>> If I make a transactional "echo", read x; write x back to the same
>> topic.
>> >>> RunN instances of that and kill them randomly. If I am loosing
>> messages
>> >>> (and not duplicating messages) then the topic would eventually have no
>> >>> data..
>> >>>
>> >>> Or should I make a program with some math formula like receive x write
>> >> xx.
>> >>> If duplication is happening I would start seeing multiple xx's
>> >>>
>> >>> Or send 1,000,000,000 messages through and consumer logs them to a
>> file.
>> >>> Then use an etl tool to validate messages come out on the other side.

Re: Attempt to prove Kafka transactions work

2019-11-09 Thread Edward Capriolo
On Thu, Oct 31, 2019 at 6:11 AM Matthias J. Sax 
wrote:

> Quite a project to test transactions...
>
> The current system test suite is part of the code base:
> https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests
>
> There is course also some unit/integration test for transactions.
>
> There is also a blog post that describes in a high level what testing
> was done when EOS was introduced:
>
> https://www.confluent.io/blog/exactly-once-semantics-are-possible-heres-how-apache-kafka-does-it/
>
> And yes, transactions are built on some assumptions and if you configure
> your system incorrectly of violate those assumptions, it may break. We
> also fixed some bugs since the first release. And there might be more
> bugs --- software is always buggy. However, for practical consideration,
> transactions should work.
>
> We would of course love if you could share your test results! If you
> discover a bug, please report it, so we can fix it.
>
>
> -Matthias
>
> On 10/28/19 10:06 AM, Edward Capriolo wrote:
> > On Sunday, October 27, 2019, Boyang Chen 
> wrote:
> >
> >> Hey Edward,
> >>
> >> just to summarize and make sure I understood your question, you want to
> >> implement some Chaos testing to validate Kafka EOS model, but not sure
> how
> >> to start or curious about whether there are already works in the
> community
> >> doing that?
> >>
> >> For the correctness of Kafka EOS, we have tons of unit tests and system
> >> tests to prove its functionality. They could be found inside the repo.
> You
> >> could check them out and see if we still have gaps (which I believe we
> >> definitely have).
> >>
> >> Boyang
> >>
> >> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo 
> >> wrote:
> >>
> >>> Hello all,
> >>>
> >>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
> >>> thousand ad impression. If numbers are 5% off you can blame javascript
> >>> click trackers.
> >>>
> >>> Now, I work in a non addtech industry and they are really, really
> serious
> >>> about exactly once.
> >>>
> >>> So there is this blog:
> >>>
> >>> https://www.confluent.io/blog/transactions-apache-kafka/
> >>>
> >>> Great little snippet of code. I think I can copy it and implement it
> >>> correctly.
> >>> You know, but if you read that section about the zombie fencing, you
> >> learn
> >>> that you either need to manually assign partitions or use the rebalance
> >>> listener and have N producers. Hunting around github is not super
> >> helpful,
> >>> some code snippets are less complete then even the snipped in the blog.
> >>>
> >>> I looked at what spring-kafka does. It does get the zombie fencing
> >> correct
> >>> with respect to fencing id, other bits other bits of the code seem
> >>> plausible.
> >>>
> >>> Notice I said "plausible", because I do not count a few end to end
> tests
> >>> running single VM as a solid enough evidence that this works in the
> face
> >> of
> >>> failures.
> >>>
> >>> I have been contemplating how one stress tests this exactly once
> concept,
> >>> with something Jepsen like or something brute force that I can run for
> 5
> >>> hours in a row
> >>>
> >>> If I faithfully implemented the code in the transactional read-write
> loop
> >>> and I feed it into my jepsen like black box tester it should:
> >>>
> >>> Create a topic with 10 partitions, Start launching read-write
> transaction
> >>> code, start feeding input data, Maybe strings like 1 -1000, now start
> >>> randomly killing vms with kill -9 kill graceful exits, maybe even
> killing
> >>> kafka, and make sure 1-1000, pop out on the other end.
> >>>
> >>> I thought of some other "crazy" ideas. One such idea:
> >>>
> >>> If I make a transactional "echo", read x; write x back to the same
> topic.
> >>> RunN instances of that and kill them randomly. If I am loosing messages
> >>> (and not duplicating messages) then the topic would eventually have no
> >>> data..
> >>>
> >>> Or should I make a program with some math formula like receive x write
> >> xx.
> >>> If duplication is happening I would start seeing multiple xx's
> >>>
> >>> Or send 1,000,000,000 messages through and consumer logs them to a
> file.
> >>> Then use an etl tool to validate messages come out on the other side.
> >>>
> >>> Or should I use a nosql with increments and count up and ensure no key
> >> has
> >>> been incremented twice.
> >>>
> >>> note: I realize I can just use kafka streams or storm, which has its
> own
> >>> systems to guarantee "at most once" but Iooking for a way to prove what
> >> can
> >>> be done with pure kafka. (and not just prove it adtech work (good
> enough
> >> 5%
> >>> here or there) )
> >>>
> >>> I imagine someone somewhere must be doing this. How where tips? Is it
> >> part
> >>> of some kafka release stress test? I'm down to write it if it does not
> >>> exist.
> >>>
> >>> Thanks,
> >>> Edward,
> >>>
> >>> Thanks,
> >>> Edward
> >>>
> >>
> >
> > Boyang,
> >
> > I just to summarize and make sure I understood your question, you want to

Re: Attempt to prove Kafka transactions work

2019-10-31 Thread Matthias J. Sax
Quite a project to test transactions...

The current system test suite is part of the code base:
https://github.com/apache/kafka/tree/trunk/tests/kafkatest/tests

There is course also some unit/integration test for transactions.

There is also a blog post that describes in a high level what testing
was done when EOS was introduced:
https://www.confluent.io/blog/exactly-once-semantics-are-possible-heres-how-apache-kafka-does-it/

And yes, transactions are built on some assumptions and if you configure
your system incorrectly of violate those assumptions, it may break. We
also fixed some bugs since the first release. And there might be more
bugs --- software is always buggy. However, for practical consideration,
transactions should work.

We would of course love if you could share your test results! If you
discover a bug, please report it, so we can fix it.


-Matthias

On 10/28/19 10:06 AM, Edward Capriolo wrote:
> On Sunday, October 27, 2019, Boyang Chen  wrote:
> 
>> Hey Edward,
>>
>> just to summarize and make sure I understood your question, you want to
>> implement some Chaos testing to validate Kafka EOS model, but not sure how
>> to start or curious about whether there are already works in the community
>> doing that?
>>
>> For the correctness of Kafka EOS, we have tons of unit tests and system
>> tests to prove its functionality. They could be found inside the repo. You
>> could check them out and see if we still have gaps (which I believe we
>> definitely have).
>>
>> Boyang
>>
>> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo 
>> wrote:
>>
>>> Hello all,
>>>
>>> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
>>> thousand ad impression. If numbers are 5% off you can blame javascript
>>> click trackers.
>>>
>>> Now, I work in a non addtech industry and they are really, really serious
>>> about exactly once.
>>>
>>> So there is this blog:
>>>
>>> https://www.confluent.io/blog/transactions-apache-kafka/
>>>
>>> Great little snippet of code. I think I can copy it and implement it
>>> correctly.
>>> You know, but if you read that section about the zombie fencing, you
>> learn
>>> that you either need to manually assign partitions or use the rebalance
>>> listener and have N producers. Hunting around github is not super
>> helpful,
>>> some code snippets are less complete then even the snipped in the blog.
>>>
>>> I looked at what spring-kafka does. It does get the zombie fencing
>> correct
>>> with respect to fencing id, other bits other bits of the code seem
>>> plausible.
>>>
>>> Notice I said "plausible", because I do not count a few end to end tests
>>> running single VM as a solid enough evidence that this works in the face
>> of
>>> failures.
>>>
>>> I have been contemplating how one stress tests this exactly once concept,
>>> with something Jepsen like or something brute force that I can run for 5
>>> hours in a row
>>>
>>> If I faithfully implemented the code in the transactional read-write loop
>>> and I feed it into my jepsen like black box tester it should:
>>>
>>> Create a topic with 10 partitions, Start launching read-write transaction
>>> code, start feeding input data, Maybe strings like 1 -1000, now start
>>> randomly killing vms with kill -9 kill graceful exits, maybe even killing
>>> kafka, and make sure 1-1000, pop out on the other end.
>>>
>>> I thought of some other "crazy" ideas. One such idea:
>>>
>>> If I make a transactional "echo", read x; write x back to the same topic.
>>> RunN instances of that and kill them randomly. If I am loosing messages
>>> (and not duplicating messages) then the topic would eventually have no
>>> data..
>>>
>>> Or should I make a program with some math formula like receive x write
>> xx.
>>> If duplication is happening I would start seeing multiple xx's
>>>
>>> Or send 1,000,000,000 messages through and consumer logs them to a file.
>>> Then use an etl tool to validate messages come out on the other side.
>>>
>>> Or should I use a nosql with increments and count up and ensure no key
>> has
>>> been incremented twice.
>>>
>>> note: I realize I can just use kafka streams or storm, which has its own
>>> systems to guarantee "at most once" but Iooking for a way to prove what
>> can
>>> be done with pure kafka. (and not just prove it adtech work (good enough
>> 5%
>>> here or there) )
>>>
>>> I imagine someone somewhere must be doing this. How where tips? Is it
>> part
>>> of some kafka release stress test? I'm down to write it if it does not
>>> exist.
>>>
>>> Thanks,
>>> Edward,
>>>
>>> Thanks,
>>> Edward
>>>
>>
> 
> Boyang,
> 
> I just to summarize and make sure I understood your question, you want to
> implement some Chaos testing to validate Kafka EOS model, but not sure how
> to start or curious about whether there are already works in the community
> doing that?
> 
> Yes.
> 
> I am not an expert in this field, but I know what distributed systems can
> mask failures. For example if you have atomic increment you might unit test

Re: Attempt to prove Kafka transactions work

2019-10-28 Thread Edward Capriolo
On Sunday, October 27, 2019, Boyang Chen  wrote:

> Hey Edward,
>
> just to summarize and make sure I understood your question, you want to
> implement some Chaos testing to validate Kafka EOS model, but not sure how
> to start or curious about whether there are already works in the community
> doing that?
>
> For the correctness of Kafka EOS, we have tons of unit tests and system
> tests to prove its functionality. They could be found inside the repo. You
> could check them out and see if we still have gaps (which I believe we
> definitely have).
>
> Boyang
>
> On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo 
> wrote:
>
> > Hello all,
> >
> > I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
> > thousand ad impression. If numbers are 5% off you can blame javascript
> > click trackers.
> >
> > Now, I work in a non addtech industry and they are really, really serious
> > about exactly once.
> >
> > So there is this blog:
> >
> > https://www.confluent.io/blog/transactions-apache-kafka/
> >
> > Great little snippet of code. I think I can copy it and implement it
> > correctly.
> > You know, but if you read that section about the zombie fencing, you
> learn
> > that you either need to manually assign partitions or use the rebalance
> > listener and have N producers. Hunting around github is not super
> helpful,
> > some code snippets are less complete then even the snipped in the blog.
> >
> > I looked at what spring-kafka does. It does get the zombie fencing
> correct
> > with respect to fencing id, other bits other bits of the code seem
> > plausible.
> >
> > Notice I said "plausible", because I do not count a few end to end tests
> > running single VM as a solid enough evidence that this works in the face
> of
> > failures.
> >
> > I have been contemplating how one stress tests this exactly once concept,
> > with something Jepsen like or something brute force that I can run for 5
> > hours in a row
> >
> > If I faithfully implemented the code in the transactional read-write loop
> > and I feed it into my jepsen like black box tester it should:
> >
> > Create a topic with 10 partitions, Start launching read-write transaction
> > code, start feeding input data, Maybe strings like 1 -1000, now start
> > randomly killing vms with kill -9 kill graceful exits, maybe even killing
> > kafka, and make sure 1-1000, pop out on the other end.
> >
> > I thought of some other "crazy" ideas. One such idea:
> >
> > If I make a transactional "echo", read x; write x back to the same topic.
> > RunN instances of that and kill them randomly. If I am loosing messages
> > (and not duplicating messages) then the topic would eventually have no
> > data..
> >
> > Or should I make a program with some math formula like receive x write
> xx.
> > If duplication is happening I would start seeing multiple xx's
> >
> > Or send 1,000,000,000 messages through and consumer logs them to a file.
> > Then use an etl tool to validate messages come out on the other side.
> >
> > Or should I use a nosql with increments and count up and ensure no key
> has
> > been incremented twice.
> >
> > note: I realize I can just use kafka streams or storm, which has its own
> > systems to guarantee "at most once" but Iooking for a way to prove what
> can
> > be done with pure kafka. (and not just prove it adtech work (good enough
> 5%
> > here or there) )
> >
> > I imagine someone somewhere must be doing this. How where tips? Is it
> part
> > of some kafka release stress test? I'm down to write it if it does not
> > exist.
> >
> > Thanks,
> > Edward,
> >
> > Thanks,
> > Edward
> >
>

Boyang,

I just to summarize and make sure I understood your question, you want to
implement some Chaos testing to validate Kafka EOS model, but not sure how
to start or curious about whether there are already works in the community
doing that?

Yes.

I am not an expert in this field, but I know what distributed systems can
mask failures. For example if you have atomic increment you might unit test
it and it works fine, but if you ran it for 40 days it might double count 1
time.

 of Kafka EOS, we have tons of unit tests and system
tests to prove its functionality. They could be found inside the repo.

I've been a developer for a while so the phrase "there are tests" never
tells me everything. Tests reveal the presence of bugs not the absence.

Can you please point me at the tests? My curiosity is if there is a
systematic in-depth strategy here and how much rigor there is.

In my environment I need to quantify and use rigor to prove out these
things. Things that you might take for granted. For example, I have to
prove that zookeeper works as expected when we lose a datacenter. Most
people 'in the know' take it for granted that kafka and zk do what is
advertised when configured properly. I have to test that out and document
my findings.

For kafka transactions. The user space code needs to be written properly
and configured properly along with the server being 

Re: Attempt to prove Kafka transactions work

2019-10-27 Thread Boyang Chen
Hey Edward,

just to summarize and make sure I understood your question, you want to
implement some Chaos testing to validate Kafka EOS model, but not sure how
to start or curious about whether there are already works in the community
doing that?

For the correctness of Kafka EOS, we have tons of unit tests and system
tests to prove its functionality. They could be found inside the repo. You
could check them out and see if we still have gaps (which I believe we
definitely have).

Boyang

On Fri, Oct 25, 2019 at 7:25 PM Edward Capriolo 
wrote:

> Hello all,
>
> I used to work in adtech. Adtech was great. CPM for ads is 1-$5 per
> thousand ad impression. If numbers are 5% off you can blame javascript
> click trackers.
>
> Now, I work in a non addtech industry and they are really, really serious
> about exactly once.
>
> So there is this blog:
>
> https://www.confluent.io/blog/transactions-apache-kafka/
>
> Great little snippet of code. I think I can copy it and implement it
> correctly.
> You know, but if you read that section about the zombie fencing, you learn
> that you either need to manually assign partitions or use the rebalance
> listener and have N producers. Hunting around github is not super helpful,
> some code snippets are less complete then even the snipped in the blog.
>
> I looked at what spring-kafka does. It does get the zombie fencing correct
> with respect to fencing id, other bits other bits of the code seem
> plausible.
>
> Notice I said "plausible", because I do not count a few end to end tests
> running single VM as a solid enough evidence that this works in the face of
> failures.
>
> I have been contemplating how one stress tests this exactly once concept,
> with something Jepsen like or something brute force that I can run for 5
> hours in a row
>
> If I faithfully implemented the code in the transactional read-write loop
> and I feed it into my jepsen like black box tester it should:
>
> Create a topic with 10 partitions, Start launching read-write transaction
> code, start feeding input data, Maybe strings like 1 -1000, now start
> randomly killing vms with kill -9 kill graceful exits, maybe even killing
> kafka, and make sure 1-1000, pop out on the other end.
>
> I thought of some other "crazy" ideas. One such idea:
>
> If I make a transactional "echo", read x; write x back to the same topic.
> RunN instances of that and kill them randomly. If I am loosing messages
> (and not duplicating messages) then the topic would eventually have no
> data..
>
> Or should I make a program with some math formula like receive x write xx.
> If duplication is happening I would start seeing multiple xx's
>
> Or send 1,000,000,000 messages through and consumer logs them to a file.
> Then use an etl tool to validate messages come out on the other side.
>
> Or should I use a nosql with increments and count up and ensure no key has
> been incremented twice.
>
> note: I realize I can just use kafka streams or storm, which has its own
> systems to guarantee "at most once" but Iooking for a way to prove what can
> be done with pure kafka. (and not just prove it adtech work (good enough 5%
> here or there) )
>
> I imagine someone somewhere must be doing this. How where tips? Is it part
> of some kafka release stress test? I'm down to write it if it does not
> exist.
>
> Thanks,
> Edward,
>
> Thanks,
> Edward
>