Hi Neeraj, Sorry, I was wrong, the error rate is only records that failed to send after all retries were exhausted (or if retries weren't possible). So if that's gone up, then records have been dropped by the producer's sender.
Cheers, Liam On Tue, 5 Apr 2022 at 10:35, Neeraj Vaidya <neeraj.vai...@yahoo.co.in.invalid> wrote: > Thanks Liam. > Yes, I do believe the following should really help: > A producer metric which shows records which did not make their way to the > topic because of retries being exhausted or timeout being exhausted . > > If the metric is at a batch level, then we will need to work out the math > to calculate exactly how many records were dropped. > > Regards, > Neeraj > On Tuesday, 5 April, 2022, 07:47:21 am GMT+10, Liam Clarke-Hutchinson > <lclar...@redhat.com> wrote: > > Hi Neeraj, > > However, I am unclear what the record-error-rate|total metric for a > > producer means, > > Does the metric get incremented only when the record could not make it to > > the topic or even when there was a transient/retriable error trying to > send > > the message to the topic ? > > > The latter - so in your example, the error rate and retry rate metrics > would both show an increase, but the records were eventually successfully > sent. Would a metric for "batches that exhausted retries and so were > dropped" be of any use to you? If so, I can propose adding one, and see > what people think. > > Cheers, > > Liam Clarke-Hutchinson > > On Mon, 4 Apr 2022 at 19:29, Neeraj Vaidya > <neeraj.vai...@yahoo.co.in.invalid> wrote: > > > Thank you David and Liam for your excellent responses. > > Checking in the consumer will be extremely difficult. > > However, I am unclear what the record-error-rate|total metric for a > > producer means, > > Does the metric get incremented only when the record could not make it to > > the topic or even when there was a transient/retriable error trying to > send > > the message to the topic ? > > > > I am posting below the producer properties that I am using. > > > > acks = -1 > > batch.size = 16384 > > bootstrap.servers = [##MASKED##] > > buffer.memory = 23622320128 > > client.dns.lookup = use_all_dns_ips > > client.id = producer-1 > > compression.type = none > > connections.max.idle.ms = 540000 > > delivery.timeout.ms = 2880000 > > enable.idempotence = true > > interceptor.classes = [] > > internal.auto.downgrade.txn.commit = false > > key.serializer = class > > org.apache.kafka.common.serialization.StringSerializer > > linger.ms = 0 > > max.block.ms = 1440000 > > max.in.flight.requests.per.connection = 5 > > max.request.size = 1048576 > > metadata.max.age.ms = 7200000 > > metadata.max.idle.ms = 7200000 > > metric.reporters = [] > > metrics.num.samples = 2 > > metrics.recording.level = INFO > > metrics.sample.window.ms = 30000 > > partitioner.class = class > > org.apache.kafka.clients.producer.internals.DefaultPartitioner > > receive.buffer.bytes = 32768 > > reconnect.backoff.max.ms = 1000 > > reconnect.backoff.ms = 50 > > request.timeout.ms = 30000 > > retries = 2147483647 > > retry.backoff.ms = 100 > > sasl.client.callback.handler.class = null > > sasl.jaas.config = null > > sasl.kerberos.kinit.cmd = /usr/bin/kinit > > sasl.kerberos.min.time.before.relogin = 60000 > > sasl.kerberos.service.name = null > > sasl.kerberos.ticket.renew.jitter = 0.05 > > sasl.kerberos.ticket.renew.window.factor = 0.8 > > sasl.login.callback.handler.class = null > > sasl.login.class = null > > sasl.login.refresh.buffer.seconds = 300 > > sasl.login.refresh.min.period.seconds = 60 > > sasl.login.refresh.window.factor = 0.8 > > sasl.login.refresh.window.jitter = 0.05 > > sasl.mechanism = GSSAPI > > security.protocol = PLAINTEXT > > security.providers = null > > send.buffer.bytes = 131072 > > ssl.cipher.suites = null > > ssl.enabled.protocols = [TLSv1.2, TLSv1.3] > > ssl.endpoint.identification.algorithm = https > > ssl.engine.factory.class = null > > ssl.key.password = null > > ssl.keymanager.algorithm = SunX509 > > ssl.keystore.location = null > > ssl.keystore.password = null > > ssl.keystore.type = JKS > > ssl.protocol = TLSv1.3 > > ssl.provider = null > > ssl.secure.random.implementation = null > > ssl.trustmanager.algorithm = PKIX > > ssl.truststore.location = null > > ssl.truststore.password = null > > ssl.truststore.type = JKS > > transaction.timeout.ms = 60000 > > transactional.id = null > > value.serializer = class > > io.vertx.kafka.client.serialization.JsonObjectSerializer > > > > Regards, > > Neeraj On Monday, 4 April, 2022, 03:19:08 pm GMT+10, David Finnie < > > david.fin...@infrasoft.com.au> wrote: > > > > Hi Neeraj, > > > > I don't know what might be causing the first Produce error. Is the > > OUT_OF_ORDER_SEQUENCE_NUMBER the first Produce error? From the error > > that you included (Invalid sequence number for new epoch) it would seem > > that the broker doesn't (yet) know about the Producer's epoch - possibly > > because it is still catching up after you restarted it? Note that the > > first sequence number for a new epoch must be 0, so if the broker thinks > > that it is a new epoch, but the sequence number is 3, it will cause this > > error. > > > > I can explain more about the relationship of Producer ID, Producer Epoch > > and Sequence Number if you want. > > > > With 5 in-flight requests per connection, if any Produce is rejected, > > all other in-flight Produce requests will be rejected with > > OUT_OF_ORDER_SEQUENCE_NUMBER because the first rejected Produce > > request's sequence number range never got stored, so all subsequent > > in-flight Produce requests are out of sequence. (e.g. if a Produce with > > sequence number 2 is rejected, and further Produce requests have already > > been sent with sequence numbers 3, 4 and 5, after sequence number 2 is > > rejected, the broker is still expecting the next sequence number to be > > 2, and so will reject sequence numbers 3, 4, 5 with this error.) > > > > When a Produce request is rejected, the Producer code first waits for > > responses to arrive for all in-flight Produce requests, and does not > > send any new Produce requests until all responses are received. The > > reason is that it doesn't know whether the other requests will receive > > rejections simply of OUT_OF_ORDER_SEQUENCE_NUMBER, or some other > > rejection reason. It needs to wait so that it knows which record batches > > to retry. > > > > It is most likely that the poor performance you are seeing is due to the > > need to wait for in-flight Produce requests to receive responses. There > > is therefore less throughput achievable during such error recovery > > times. Additionally, any in-flight Produce requests that are deemed > > retriable need to be re-sent, so there is a doubling up of their network > > traffic. > > > > Does this poor performance last for a long time? I would have thought > > that it should be just a minor hiccup in performance, because the > > Producer will increment the epoch, and reset sequence numbers from 0. > > That should then allow for resumption of normal traffic for that > > Producer ID on that partition. > > > > Re. whether the records made it to the topic, they should have. The log > > messages indicate that it is retrying the records, and incrementing the > > epoch and resequencing is part of that process. Of course, you should > > probably check by setting up a consumer to ensure that all messages made > > it to the topic, if that is feasible. > > > > David Finnie > > > > Infrasoft Pty Limited > > > > On 4/04/2022 12:42, Neeraj Vaidya 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 > > >> > > >> > > > > > >