Hello Pieter, Please feel free to apply this two-liner to your branch and see if it helps: https://github.com/apache/kafka/pull/6951
Guozhang On Thu, Jun 13, 2019 at 12:30 AM Pieter Hameete <pieter.hame...@blockbax.com> wrote: > Hi Guozhang, > > We can run a shadow setup to test the workaround hotfix if that is helpful > for you. For the production cluster we'll wait for an official artifact to > be released. > > It is not super urgent for us because records are still being > produced/consumed. The main issue is many unnecessary error and warn logs, > and the unnecessary retries that follow from it. If there will be a fix for > this eventually that would make us very happy :-) > > Thanks for your efforts so far, and let me know if I can do anything to > assist. We could continue this work via Slack if you prefer. > > Best, > > Pieter > > -----Oorspronkelijk bericht----- > Van: Guozhang Wang <wangg...@gmail.com> > Verzonden: Wednesday, 12 June 2019 20:30 > Aan: users@kafka.apache.org > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka streams > applications > > @Jonathan Actually after double checking the code I realized > https://issues.apache.org/jira/browse/KAFKA-7190 would not completely > resolve this issue, as explained in my previous email to Pieter. > > @Pieter Your concerns about this issue makes sense to me. Are you willing > to compile from source code than from a downloaded artifact? I can try to > provide you a quick workaround hotfix if you're willing to do the former. > > > Guozhang > > On Wed, Jun 12, 2019 at 4:13 AM Pieter Hameete < > pieter.hame...@blockbax.com> > wrote: > > > Hey Jonathan, > > > > Thanks for the input. We've looked into the issue you linked but > > changing the topic config didn’t work for us (see earlier mails in this > thread). > > Could you elaborate a bit more on the specifics of your situation? > > > > How many partitions do you have for the repartition topics? What is > > your commit.interval.ms setting for the streams application? What is > > the frequency (volume) at which records are produced to each partition > > in your case? > > > > I'm trying to get a better idea of what 'low rate' and 'high volume' are. > > Specifically in comparison to your topic and Kafka streams app > > configuration. We produce records to a partition every 5 seconds, and > > this results in errors which is unexpected to us because we run a EOS > > configuration with default settings and we don't regard 1 record per 5 > > seconds per partition as super low volume with respect to these > > default settings. > > > > Best, > > > > Pieter > > > > -----Oorspronkelijk bericht----- > > Van: Jonathan Santilli <jonathansanti...@gmail.com> > > Verzonden: Tuesday, 11 June 2019 14:14 > > Aan: Kafka Users <users@kafka.apache.org> > > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka streams > > applications > > > > Hello, > > > > we currently see the same WARN logs when the App is processing low > > rate of records, as soon the app starts processing a high volume of > > records, those WARN stop showing in the logs. > > According to other email threads, this should be fixed with > > https://issues.apache.org/jira/browse/KAFKA-7190. > > > > We use version 2.2.1, with EOS > > > > Cheers! > > -- > > Jonathan > > > > > > On Tue, Jun 11, 2019 at 11:04 AM Pieter Hameete < > > pieter.hame...@blockbax.com> > > wrote: > > > > > Hi Guozhang, > > > > > > Thanks that clarifies a lot. The segment size or segment ms is not > > > the issue here then, changing these properties does not make the > > > errors go away either 😊 > > > > > > Regarding the proposed solutions: > > > > > > 1) We use EOS, so the commit interval is defaulted to 100 ms in our > case. > > > We don't want to delay consumers that respect transactions in their > > > consumption of the produced events so increasing this config is not > > > a good option for us. We have tried temporarily increasing it to 30s > > > and indeed partitions receiving records at a higher frequency (i.e. > > > 1 per > > > 5 seconds) stopped resulting in errors. Partitions receiving records > > > at > > a lower (i.e. > > > 1 per minute) frequency were still resulting in errors. > > > > > > 2) We have a small disparity in keys, so the data is already spread > > > over a small (3 or 4) number of partitions. I don't think reducing > > > the number of partitions to for example 4 will make a difference in > > > this > > case? > > > > > > My attempt to summarize the issue here would be the following: > > > > > > If exactly once is enabled and records are produced on a partition > > > less frequent than the commit interval, then the brokers/producers > > > will produce an UNKNOWN_PRODUCER_ID error/warning on each produced > > > record and the producers need to retry once for every produced record. > > > This happens because commits trigger truncation of the producer ID > > > map causing producers that did not produce any messages since last > > > commit to have their ID removed. > > > > > > In our case we use a Kafka Streams application with exactly once > > > enabled and a default commit.interval.ms of 100ms. We don't wish to > > > increase this default to allow the consumers to be as real-time as > > > possible. Even if we would scale down to 1 partition (which is > > > undesirable) I don't think the producers would produce a record to > > > this one partition between every offset commit. Basically we can't > > > resolve this problem unless we would have a volume large enough to > > > produce a record on each partition between every offset commit. > > > > > > Our only options seem to be to move from exactly once to at least > > > once, or to wait for > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+h > > andling+of+unknown+producer > > . > > > > > > > > > Best, > > > > > > Pieter > > > > > > -----Oorspronkelijk bericht----- > > > Van: Guozhang Wang <wangg...@gmail.com> > > > Verzonden: Monday, 10 June 2019 22:19 > > > Aan: users@kafka.apache.org > > > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > streams applications > > > > > > Hi Pieter, > > > > > > My reasoning is that the most recent segment (called the `active` > > > segment`) would not be deleted for immediately since it is still > > > being appended. I.e. > > > say you have two segments offset range at [0, 100), [100, 180). And > > > if the delete records API is issued to delete any records older than > > > 150, then the first segment would be deleted, but the second (and > > > the active one) would not be deleted. Instead we would only set the > > log-starting-offset at 150. > > > > > > But when I double checked the code again, I realized that even if we > > > do not delete the segment, we would still "truncate" the maintained > > > producer ID map, and hence what you've seen is correct: although the > > > data cleaner did not delete the segment file yet, corresponding > > > producer ID would still be removed. > > > > > > So for now what I can think of to reduce this issue, are that you > > > can either consider 1) increase your commit interval, and hence > > > reduce the delete-records frequency, or 2) decrease your > > > num.partitions so that the per-partition traffic from a single > producer can be reasonably large. > > > > > > > > > Guozhang > > > > > > > > > On Fri, Jun 7, 2019 at 2:09 AM Pieter Hameete > > > <pieter.hame...@blockbax.com > > > > > > > wrote: > > > > > > > Hi Guozhang, > > > > > > > > I've checked the log-cleaner.log on the brokers for cleanup of > > > > repartition topics for the event-rule-engine application, but I > > > > found no reports of frequent cleanup there. > > > > > > > > Also forgive me if I am wrong, because I don’t understand the > > > > exact mechanics of the log cleanup processes. But didn't > > > > KAFKA-6150 ( > > > > https://github.com/apache/kafka/pull/4270/) introduce a log > > > > cleanup mechanic for repartition topics that cleans up the logs > > > > more aggressively (i.e. not following the regular LogCleaner > > > > cleanup process that is triggered by segment.ms and segment.bytes) > > > > by deleting records via the admin client based on committed input > > > > topic > > offsets. > > > > This would explain producer id's not being known if the logs get > > > > cleaned up almost instantly via this process. > > > > > > > > Best, > > > > > > > > Pieter > > > > > > > > -----Oorspronkelijk bericht----- > > > > Van: Guozhang Wang <wangg...@gmail.com> > > > > Verzonden: Thursday, 6 June 2019 18:16 > > > > Aan: users@kafka.apache.org > > > > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > > streams applications > > > > > > > > Pieter, > > > > > > > > KIP-360 should be able to fix it, yes, but it won't be completed > > > > soon, and the earliest release it may get in is in 2.4. At the > > > > mean time, I thought that be increasing the segment size, hence > > > > reducing the frequency that records being truncated and hence > > > > causing producer ID to be removed to be much smaller. > > > > > > > > If you look at the broker-side logs, could you see if the log > > > > cleanup threads logs that truncating the > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartitio > > > > n- xx partitions? If it is too frequent that would cause the > > > > producer ID stored on brokers to be removed too early and > > > > therefore UNKNOWN_PRODUCER_ID, but overriding the segment size / > > > > ms should prevent it from being triggered too often. > > > > > > > > Guozhang > > > > > > > > > > > > On Thu, Jun 6, 2019 at 5:10 AM Pieter Hameete > > > > <pieter.hame...@blockbax.com > > > > > > > > > wrote: > > > > > > > > > Hello, > > > > > > > > > > We also found this earlier email in the archives which looks > > > > > very much like what we are experiencing: > > > > > > > > > > > > > > > http://mail-archives.apache.org/mod_mbox/kafka-users/201811.mbox > > > > > /% > > > > > 3C > > > > > CA > > > > > M0VdefApmc5wBZQaJmQtbcnZ_OOgGv84qCuPoJS-KU4B=e...@mail.gmail.com > > > > > %3 > > > > > E > > > > > > > > > > So it seems like: > > > > > > > > > > * It only happens with EXACTLY_ONCE processing > > > > > * It only happens on repartition topics (probably due to the > > > > > changes in > > > > > https://issues.apache.org/jira/browse/KAFKA-6150) > > > > > * The errors are happening continuously (on each produced > > > > > message) > > > > > > > > > > The email I referred to points to > > > > > https://issues.apache.org/jira/browse/KAFKA-7190 for ongoing > > > > > work on solving this issue. However, it seems that this 7190 > > > > > solves the UNKNOWN_PRODUCER_ID with a different cause (low > > > > > traffic). The hotfix of changing the retention settings on the > > > > > topic does not seem to resolve the errors in our case. > > > > > > > > > > Do you think the changes listed under KIP-360 > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Imp > > > > > ro > > > > > ve > > > > > +h > > > > > andling+of+unknown+producer > > > > > could solve what we are seeing here? > > > > > > > > > > Best, > > > > > > > > > > Pieter > > > > > > > > > > -----Oorspronkelijk bericht----- > > > > > Van: Pieter Hameete <pieter.hame...@blockbax.com> > > > > > Verzonden: Thursday, 6 June 2019 13:39 > > > > > Aan: users@kafka.apache.org > > > > > Onderwerp: RE: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > > > streams applications > > > > > > > > > > Hi Guozhang, Matthias, > > > > > > > > > > @Guozhang The brokers are on version 2.2.0-cp2 and the clients > > > > > are version 2.2.1. We have upgraded yesterday from 2.1.1 to this > > > > > version to see if it would make a difference, but unfortunately > > > > > not. After restarting the brokers and streams applications the > > > > > broker errors and client warnings continued as before. > > > > > > > > > > @Matthias I've increased the segment.bytes to be 10 times the > > > > > default but this does not change anything. > > > > > > > > > > For my understanding: why are retention settings important here? > > > > > Because the producer which is logging the UNKNOWN_PRODUCER_ID > > > > > exception produced a message 5 seconds earlier (with the same > > > > > exception though..). I remember from one of your comments > > > > > (Matthias) that the brokers build up the set of known producers > > > > > from > > the log. > > > > > If a message was produced on a partition 5 seconds before by the > > > > > same producer (confirmed via kafkacat), how can it be the broker > > > > > throws an > > > > UNKNOWN_PRODUCER_ID exception? > > > > > > > > > > Best, > > > > > > > > > > Pieter > > > > > > > > > > -----Oorspronkelijk bericht----- > > > > > Van: Guozhang Wang <wangg...@gmail.com> > > > > > Verzonden: Thursday, 6 June 2019 01:39 > > > > > Aan: users@kafka.apache.org > > > > > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > > > streams applications > > > > > > > > > > Hello Pieter, > > > > > > > > > > It's normal to see offset increasing by 2: that is because each > > > > > message on that partition is from a separate transaction which > > > > > Streams EOS leverage on, and whenever a a txn commits a commit > > > > > marker, written as a special message would be appended but not > > > > > exposed > > > to consumers. > > > > > And hence that txn marker will take one offset as well. > > > > > > > > > > Your overridden configs look good to me, and it's indeed weird > > > > > why it did not mitigate your observed issue. Which Kafka version > > > > > did you > > > use? > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > On Wed, Jun 5, 2019 at 4:43 AM Pieter Hameete > > > > > <pieter.hame...@blockbax.com > > > > > > > > > > > wrote: > > > > > > > > > > > Hi Guozhang, > > > > > > > > > > > > Some additional finding: it seems to only happen on Kakfa > > > > > > Streams repartition topics. We haven't seen this happening for > > > > > > any other topics so far. > > > > > > > > > > > > Best, > > > > > > > > > > > > Pieter > > > > > > > > > > > > -----Oorspronkelijk bericht----- > > > > > > Van: Pieter Hameete <pieter.hame...@blockbax.com> > > > > > > Verzonden: Wednesday, 5 June 2019 11:23 > > > > > > Aan: users@kafka.apache.org > > > > > > Onderwerp: RE: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > > > > streams applications > > > > > > > > > > > > Hi Guozhang, > > > > > > > > > > > > Thanks for your reply! I noticed my original mail went out > > > > > > twice by accident, sorry for that. > > > > > > > > > > > > We currently have a small variety of keys so not all > > > > > > partitions are 'actively used' indeed. The strange thing is > > > > > > though is that the errors occur for the partitions that > > > > > > actively receive records every few seconds. I have checked > > > > > > this using kafkacat to consume the specific > > > > > partitions. > > > > > > Something I noticed was that for each received record the > > > > > > partition offset was 2 higher than the previous record, > > > > > > instead of the expected 1. Could that be due to the producers > > > > > > retrying (see warning logs in my original mail)? > > > > > > > > > > > > I've done the override for the configs in the repartition > > > > > > topics as follows, on one of the brokers: > > > > > > > > > > > > The values are taken from your KIP-443 > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-443%3A+R > > > > > > et > > > > > > ur > > > > > > n+ > > > > > > to > > > > > > +default+segment.ms+and+segment.index.bytes+in+Streams+reparti > > > > > > +default+ti > > > > > > +default+on > > > > > > +default++t > > > > > > +default+op > > > > > > ics > > > > > > > > > > > > kafka-topics --zookeeper localhost:2181 --alter --topic > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart > > > > > > it io n --config segment.index.bytes=10485760 kafka-topics > > > > > > --zookeeper > > > > > > localhost:2181 --alter --topic > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart > > > > > > it io n --config segment.bytes= 52428800 kafka-topics > > > > > > --zookeeper > > > > > > localhost:2181 --alter --topic > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart > > > > > > it io n --config segment.ms=604800000 kafka-topics --zookeeper > > > > > > localhost:2181 --alter --topic > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart > > > > > > it > > > > > > io > > > > > > n > > > > > > --config retention.ms=-1 > > > > > > > > > > > > Verifying afterwards: > > > > > > > > > > > > kafka-topics --zookeeper localhost:2181 --describe --topic > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart > > > > > > it > > > > > > io > > > > > > n > > > > > > > > > > > > Topic:event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015- > > > > > > re > > > > > > pa > > > > > > rt > > > > > > it > > > > > > ion > > > > > > > > > > > > PartitionCount:32 ReplicationFactor:3 > > > > > > Configs:segment.bytes=52428800,retention.ms > > > > > > =-1,segment.index.bytes=10485760,segment.ms > > > > > > =604800000,cleanup.policy=delete > > > > > > > > > > > > Is there anything that seems off to you? Or something else I > > > > > > can investigate further? We'd really like to nail this issue > down. > > > > > > Especially because the cause seems different than the 'low > traffic' > > > > > > cause in JIRA issue KAFKA-7190 as the partitions for which > > > > > > errors are thrown are receiving data. > > > > > > > > > > > > Best, > > > > > > > > > > > > Pieter > > > > > > > > > > > > -----Oorspronkelijk bericht----- > > > > > > Van: Guozhang Wang <wangg...@gmail.com> > > > > > > Verzonden: Wednesday, 5 June 2019 02:23 > > > > > > Aan: users@kafka.apache.org > > > > > > Onderwerp: Re: Repeating UNKNOWN_PRODUCER_ID errors for Kafka > > > > > > streams applications > > > > > > > > > > > > Hello Pieter, > > > > > > > > > > > > If you only have one record every few seconds that may be too > > > > > > small given you have at least 25 partitions (as I saw you have > > > > > > a > > > > > > xxx--repartition-24 partition), which means that for a single > > > > > > partition, it may not see any records for a long time, and in > > > > > > this case you may need to override it to very large values. On > > > > > > the other hand, if you can reduce your num.partitions that may > > > > > > also help > > > > > increasing the traffic per partition. > > > > > > > > > > > > Also could you show me how did you override the configs in the > > > > > > repartition topics? > > > > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > On Tue, Jun 4, 2019 at 2:10 AM Pieter Hameete > > > > > > <pieter.hame...@blockbax.com > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > Hello, > > > > > > > > > > > > > > Our Kafka streams applications are showing the following > > > > > > > warning every few seconds (on each of our 3 brokers, and on > > > > > > > each of the > > > > > > > 2 instances of the streams application): > > > > > > > > > > > > > > > > > > > > > [Producer > > > > > > > clientId=event-rule-engine-dd71ae9b-523c-425d-a7c0-c62993315 > > > > > > > b3 > > > > > > > 0- > > > > > > > St > > > > > > > re am Thread-1-1_24-producer, > > > > > > > transactionalId=event-rule-engine-1_24] > > > > > > > Resetting sequence number of batch with current sequence 1 > > > > > > > for partition > > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repa > > > > > > > rt > > > > > > > it > > > > > > > io > > > > > > > n- > > > > > > > 24 > > > > > > > to 0 > > > > > > > > > > > > > > > > > > > > > > > > > > > > Followed by: > > > > > > > > > > > > > > > > > > > > > > > > > > > > [Producer > > > > > > > clientId=event-rule-engine-dd71ae9b-523c-425d-a7c0-c62993315 > > > > > > > b3 > > > > > > > 0- > > > > > > > St > > > > > > > re am Thread-1-1_24-producer, > > > > > > > transactionalId=event-rule-engine-1_24] > > > > > > > Got error produce response with correlation id 5902 on > > > > > > > topic-partition > > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repa > > > > > > > rt > > > > > > > it > > > > > > > io > > > > > > > n- > > > > > > > 24 , retrying (2147483646 attempts left). Error: > > > > > > > UNKNOWN_PRODUCER_ID > > > > > > > > > > > > > > The brokers are showing errors that look related: > > > > > > > > > > > > > > > > > > > > > Error processing append operation on partition > > > > > > > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repa > > > > > > > rt > > > > > > > it > > > > > > > io > > > > > > > n- > > > > > > > 24 > > > > > > > (kafka.server.ReplicaManager) > > > > > > > > > > > > > > org.apache.kafka.common.errors.UnknownProducerIdException: > > > > > > > Found no record of producerId=72 on the broker. It is > > > > > > > possible that the last message with the producerId=72 has > > > > > > > been removed due to hitting the > > > > > > retention limit. > > > > > > > > > > > > > > > > > > > > > > > > > > > > We would expect the UNKNOWN_PRODUCER_ID error to occur once. > > > > > > > After a retry the record would be published on the partition > > > > > > > and the PRODUCER_ID would be known. However, this error > > > > > > > keeps occurring every few seconds. This is roughly at the > > > > > > > same rate at which records are produced on the input topics > > > > > > > partitions, so it seems like it occurs for > > > > > > (nearly) every input record. > > > > > > > > > > > > > > > > > > > > > > > > > > > > The following JIRA issue: > > > > > > > https://issues.apache.org/jira/browse/KAFKA-7190 > > > > > > > looks related. Except the Jira issue mentions ‘little > > > > > > > traffic’, and I am not sure if a message per every few > > > > > > > seconds is regarded as little > > > > > > traffic. > > > > > > > Matthias mentions in the issue that a workaround seems to be > > > > > > > to increase topic configs `segment.bytes`, > > > > > > > `segment.index.bytes`, and `segment.ms` for the > > > > > > > corresponding repartition topics. We’ve tried manually > > > > > > > overriding these configs for a relevant topic to the config > > > > > > > values in the linked pull request > > > > > > > (https://github.com/apache/kafka/pull/6511) but this did not > > > > > > > result in > > > > > > the errors disappearing. > > > > > > > > > > > > > > > > > > > > > > > > > > > > Could anyone help us to figure out what is happening here, > > > > > > > and why the proposed fix for the above JIRA issue is not > > > > > > > working in this > > > > case? > > > > > > > > > > > > > > > > > > > > > > > > > > > > Best, > > > > > > > > > > > > > > > > > > > > > > > > > > > > Pieter > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > -- > > > > > -- Guozhang > > > > > > > > > > > > > > > > > -- > > > > -- Guozhang > > > > > > > > > > > > > -- > > > -- Guozhang > > > > > > > > > -- > > Santilli Jonathan > > > > > -- > -- Guozhang > -- -- Guozhang