en idempotency was added, the behavior was just kept as-is.
>
>
> -Matthias
>
>
> On 4/21/25 1:27 AM, Stig Rohde Døssing wrote:
> > Thanks, this was very useful.
> >
> >> Well, the producer does not have x=1 in it's buffer any longer --
> >&g
ordering of those messages the
> > same?
>
> Well, the producer does not have x=1 in it's buffer any longer --
> otherwise, it our-of-sequence-error would not have be given to the
> application, and the producer could just retry internally.
>
>
> > Is it fair to on
> already buffered data in the producer send buffer, would use this new
> epoch and reset sequence numbers.
>
> By closing the producer though, you also throw away pending writes for
> x=2 and x=3, and would call producer.send() for all three messages
> again, and thus can again s
Hi,
If I understand correctly, the idempotent producer should be able to
guarantee that messages are not duplicated when the producer retries sends,
and as of https://issues.apache.org/jira/browse/KAFKA-5494, it should allow
for the producer to have multiple in flight requests at a time without
r
I've had something similar on a different embedded kafka project. Most
likely your issue is that you are putting kafka-clients 3.0.0 on the
classpath alongside the Kafka server in version 2.7.1, which is the version
brought in by your spring-kafka-test dependency. Since the Kafka server
itself depe
Hi,
We have a topic with min.insync.replicas = 2 where each partition is
replicated to 3 nodes. We write to it using acks=all.
We experienced a network malfunction, where leader node 1 could not reach
replica 2 and 3, and vice versa. Nodes 2 and 3 could reach each other. The
controller broker co
Thanks.
Den fre. 11. dec. 2020 kl. 13.52 skrev Fabio Pardi :
>
>
> On 11/12/2020 13:20, Stig Rohde Døssing wrote:
> > Hi,
> >
> > We have a topic with min.insync.replicas = 2 where each partition is
> > replicated to 3 nodes.
> >
> > When we send
Hi,
We have a topic with min.insync.replicas = 2 where each partition is
replicated to 3 nodes.
When we send a produce request with acks=all, the request should fail if
the records don't make it to at least 2 nodes.
If the produce request fails, what does the partition leader do with the
records
s that people end up
> using it inappropriately and ending up with very unbalanced clusters in
> production.
>
> Kind regards,
>
> Tom
>
> On Mon, Oct 5, 2020 at 10:05 AM Stig Rohde Døssing >
> wrote:
>
> > Hi,
> >
> > Kafka can recomm
Hi,
Kafka can recommend an assignment via the ReassignPartitionsCommand (with
--generate), and it is also possible to get this assignment directly by
calling AdminUtils.assignReplicasToBrokers.
We would like our application to be able to get the suggested partition
assignment and apply it to a to
Hi,
We are expanding a 3-node cluster to a 5-node cluster, and have encountered
an issue where a follower node is fetching offsets out of order. We are on
2.4.0.
We've used the kafka-reassign-partitions tool. Several partitions are
affected. Picking an example partition (11), it was configured to
and mark them as done in my offset tracker?
2017-11-03 19:50 GMT+01:00 Stig Rohde Døssing :
> Hi,
>
> I'm working on the Kafka connector for Apache Storm, which pulls messages
> from Kafka and emits them into a Storm topology. The connector uses manual
> offset control sin
Hi,
I'm working on the Kafka connector for Apache Storm, which pulls messages
from Kafka and emits them into a Storm topology. The connector uses manual
offset control since message processing happens asynchronously to pulling
messages from Kafka, and we hit an issue a while back related to topic
13 matches
Mail list logo