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 so
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 correctl
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
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
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 current
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
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.
>
>
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
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
> doi
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 u
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.
11 matches
Mail list logo