Hi Liam,
Brokers are on Apache Kafka v2.7.0
However, the Producer client is using the v2.6 libraries.

Regards,
Neeraj     On Monday, 4 April, 2022, 02:17:42 pm GMT+10, Liam Clarke-Hutchinson 
<lclar...@redhat.com> wrote:  
 
 Hi Neeraj,

Not sure just yet, I'm diving into the code to find out. Oh, what version
Kafka are you running please?

Cheers,

Liam

On Mon, 4 Apr 2022 at 14:50, Neeraj Vaidya
<neeraj.vai...@yahoo.co.in.invalid> wrote:

>  Hi Liam,
> Thanks for getting back.
>
> 1) Producer settings ( I am guessing these are the ones you are interested
> in)
> enable.idempotence=true
> max.in.flight.requests.per.connection=5
>
> 2) Sample broker logs corresponding to the timestamp in the application
> logs of the Producer
>
> [2022-04-03 15:56:39,587] ERROR [ReplicaManager broker=5] Error processing
> append operation on partition input-topic-114 (kafka.server.ReplicaManager)
> org.apache.kafka.common.errors.OutOfOrderSequenceException: Invalid
> sequence number for new epoch at offset 967756 in partition
> input-topic-114: 158 (request epoch), 3 (seq. number)
>
> Do the producer errors indicate that these messages never made it to the
> Kafka topic at all ?
>
> Regards,
> Neeraj
>      On Monday, 4 April, 2022, 12:23:30 pm GMT+10, Liam Clarke-Hutchinson <
> lclar...@redhat.com> wrote:
>
>  Hi Neeraj,
>
> First off, what are your producer settings?
> Secondly, do you have brokers logs for the leaders of some of your affected
> topics on hand at all?
>
> Cheers,
>
> Liam Clarke-Hutchinson
>
> On Mon, 4 Apr 2022 at 14:04, Neeraj Vaidya
> <neeraj.vai...@yahoo.co.in.invalid> wrote:
>
> > Hi All,
> > For one of the Kafka producers that I have, I see that the Producer
> Record
> > Error rate is non-zero i.e. out of the expected 3000 messages per second
> > which I a expect to be producing to the topic, I can see that this metric
> > shows a rate of about 200.
> > Does this indicate that the records failed to be sent to the Kafka topic,
> > or does this metric show up even for each retry in the Producer.Send
> > operation ?
> >
> > Notes :
> > 1) I have distributed  8 brokers equally across 2 sites. Using
> > rack-awareness, I am making Kafka position replicas equally across both
> > sites. My min.isr=2 and replication factor = 4. This makes 2 replicas to
> be
> > located in each site.
> > 2) The scenario I am testing is that of shutting down a set of 4 brokers
> > in one site (out of 8) for an extended period of time and then bringing
> > them back up after say 2 hours. This causes the the follower replicas on
> > those brokers to try and catch-up with the leader replicas on the other
> > brokers. The error rate that I am referring to shows up under this
> scenario
> > of restarting the brokers. It does not show up when I have just the other
> > set of (4) brokers.
> >
> > To be specific, here are the errors that I see in the Kafka producer log
> > file:
> >
> > 2022-04-03 15:56:39.613  WARN --- [-thread | producer-1]
> > o.a.k.c.p.i.Sender                      : [Producer clientId=producer-1]
> > Got error produce response with correlation id 16512434 on
> topic-partition
> > input-topic-114, retrying (2147483646 attempts left). Error:
> > OUT_OF_ORDER_SEQUENCE_NUMBER
> > 2022-04-03 15:56:39.613  WARN --- [-thread | producer-1]
> > o.a.k.c.p.i.Sender                      : [Producer clientId=producer-1]
> > Got error produce response with correlation id 16512434 on
> topic-partition
> > input-topic-58, retrying (2147483646 attempts left). Error:
> > OUT_OF_ORDER_SEQUENCE_NUMBER
> > 2022-04-03 15:56:39.613  INFO --- [-thread | producer-1]
> > o.a.k.c.p.i.TransactionManager          : [Producer clientId=producer-1]
> > ProducerId set to 2040 with epoch 159
> > 2022-04-03 15:56:39.613  INFO --- [-thread | producer-1]
> > o.a.k.c.p.i.ProducerBatch                : Resetting sequence number of
> > batch with current sequence 3 for partition input-topic-114 to 0
> > 2022-04-03 15:56:39.613  INFO --- [-thread | producer-1]
> > o.a.k.c.p.i.ProducerBatch                : Resetting sequence number of
> > batch with current sequence 5 for partition input-topic-114 to 2
> > 2022-04-03 15:56:39.613  INFO --- [-thread | producer-1]
> > o.a.k.c.p.i.ProducerBatch                : Resetting sequence number of
> > batch with current sequence 6 for partition input-topic-114 to 3
> > 2022-04-03 15:56:39.613  INFO --- [-thread | producer-1]
> > o.a.k.c.p.i.ProducerBatch                : Resetting sequence number of
> > batch with current sequence 1 for partition input-topic-58 to 0
> > 2022-04-03 15:56:39.739  WARN --- [-thread | producer-1]
> > o.a.k.c.p.i.Sender                      : [Producer clientId=producer-1]
> > Got error produce response with correlation id 16512436 on
> topic-partition
> > input-topic-82, retrying (2147483646 attempts left). Error:
> > OUT_OF_ORDER_SEQUENCE_NUMBER
> >
> > Regards,
> > Neeraj
> >
> >
>
  

Reply via email to