Thanks for the fast response Bill. I have read the comments on the Jira and I understood that the problem is complex (not much besides that :) )
I am begging to think that maybe our problem is different as this error happens all the time. Just for the small chance that it is different problem, I will describe it: *The errors:* *On the client side we are getting WARN messages that look like that:* [kafka-producer-network-thread | ******-91d95eb6-a6df-4c0a-aa48-a48272089d06-StreamThread-5-0_8-producer] [WARN ] [o.a.k.c.p.i.Sender] - [Producer clientId=******-91d95eb6-a6df-4c0a-aa48-a48272089d06-StreamThread-5-0_8-producer, transactionalId=******-0_8] Got error produce response with correlation id 1201 on topic-partition ******-******-repartition-4, retrying (2147483646 attempts left). Error: UNKNOWN_PRODUCER_ID *On the broker we are getting ERROR messages that look like that:* [2018-11-22 15:59:11,263] ERROR [ReplicaManager broker=2] Error processing append operation on partition ******-******-repartition-10 (kafka.server.ReplicaManager) org.apache.kafka.common.errors.UnknownProducerIdException: Found no record of producerId=3000 on the broker. It is possible that the last message with the producerId=3000 has been removed due to hitting the retention limit. *Our setup* We are running Kafka 2.1.0 cluster with 3 brokers and currently 3 clients (Scala 2.11 w\ kafka-streams 2.1.0). Since the upgrade to *2.1.0*, we are getting those errors in our local environment, automation environments and also in production (I find this strange because of the fact that in the local environment and the automation environments we are running on a single Kafka node). When we tried *1.1.0* / *1.1.1* / *2.0.0* / *2.0.1* - We got the error *ONLY* in production. Using *1.0.2* we don't see this error, but we did lose a big chunk of data (*in shorts*: we have compacted topic behind a reduce function that sums up numbers using + and somehow we got a smaller number to appear after the previously much higher number). Reading the description of the problems that can occur because of the way the UNKNOWN_PRODUCER error is being handled + all the bugs that got fixed in the new versions we believe this might be the reason for the data loss. Does this sound like the problem being addressed in the Jira? on our production environment, we definitely don't have "low rate" of messages. Maybe i am missing some kind of configuration? (as the error suggest, maybe some retention configuration) Currently, we are holding our progress because of it (we are thinking of going live and just log-rotate all day long :/ ) I hope that you find my message well, Let me know if you need more info/need to me to publish it somewhere else (should I send it as a new message in the mailing group?) Thanks, Nitay On Wed, Nov 21, 2018 at 6:30 PM Bill Bejeck <b...@confluent.io> wrote: > Hi Nitay, > > This is a fairly complex issue, you can catch up on the work done so far by > reading the Jira comments https://issues.apache.org/jira/browse/KAFKA-7190 > and you can read the associated KIP > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+handling+of+unknown+producer > . > > Thanks, > Bill > > On Wed, Nov 21, 2018 at 5:33 AM Nitay Kufert <nita...@ironsrc.com> wrote: > > > Hello everyone, > > Do we have any update on the subject in hand? > > > > Every time we are trying to bump our Kafka streams version in production > > past 1.0.2 we are getting the UNKNOWN_PRODUCER_ID WARN (I've tried 1.1.0, > > 1.1.1, 2.0.0 & now 2.0.1). > > In our case, we are definitely NOT under low traffic, but I think this is > > already known after reading > > https://issues.apache.org/jira/browse/KAFKA-7190 > > > > On Mon, Jul 23, 2018 at 4:56 PM lambdaliu(刘少波) <lambda...@tencent.com> > > wrote: > > > > > Hello Guozhang, Bill > > > > > > Thanks much for your explanation. I assigned this JIRA issue to myself > > > and will > > > try to fix it. > > > > > > Regards, > > > Lambdaliu(Shaobo Liu) > > > > > > On 2018-07-21 at 02:24 Bill Bejeck <b...@confluent.io> wrote: > > > > > > >Created JIRA https://issues.apache.org/jira/browse/KAFKA-7190 > > > > > > > >-Bill > > > > > > > >On Fri, Jul 20, 2018 at 1:26 PM Guozhang Wang <wangg...@gmail.com> > > wrote: > > > > > > > >> Hello Shaobo, > > > >> > > > >> The situation you observed is a valid case: when a streams > application > > > has > > > >> very low traffic (like the quickstart example, where you manually > type > > > one > > > >> message at a time), then it is possible that consumer purging would > > > delete > > > >> even the last message sent by this producer (i.e. all the messages > > sent > > > by > > > >> this producer have been consumed and committed), and as a result the > > > broker > > > >> would delete that producer's ID. The next time when this producer > > tries > > > to > > > >> send, it will get this UNKNOWN_PRODUCER_ID error code, but in this > > case > > > >> this error is retriable: the producer would just get a new producer > id > > > and > > > >> retries, and then this time it will succeed. So the results you > > > observed is > > > >> still correct. > > > >> > > > >> > > > >> We can, probably, improve this situation either in broker side or > > > streams > > > >> client side: on broker side, we can consider delaying the deletion > of > > > the > > > >> producer ID for a while; on streams client side, we can consider > > > purging in > > > >> a bit conservative manner (but it is still a bit tricky, since > > multiple > > > >> producers may be sending to the same inner topic, so just leaving > the > > > last > > > >> N messages to not be purged may not be safe still). > > > >> > > > >> > > > >> Bill, could we create a JIRA for this? > > > >> > > > >> > > > >> Guozhang > > > >> > > > >> > > > >> On Thu, Jul 19, 2018 at 7:55 AM, Bill Bejeck <b...@confluent.io> > > wrote: > > > >> > > > >> > Hi > > > >> > > > > >> > Thanks for reporting this. Just off the top of my head, I'm > > thinking > > > it > > > >> > may have to do with using a console producer, but I'll have to > take > > a > > > >> > deeper look. > > > >> > > > > >> > Thanks, > > > >> > Bill > > > >> > > > > >> > On Thu, Jul 19, 2018 at 9:59 AM lambdaliu(刘少波) < > > lambda...@tencent.com > > > > > > > >> > wrote: > > > >> > > > > >> > > Hi, > > > >> > > > > > >> > > I test the Kafka Streams WordCount demo follow the steps > described > > > in > > > >> > > http://kafka.apache.org/11/documentation/streams/quickstart > with > > > >> change > > > >> > > the processing.guarantee property to EXACTLY_ONCE. > > > >> > > > > > >> > > And seeing the following WARN message in streams demo app logs: > > > >> > > [2018-07-18 21:08:03,510] WARN The configuration 'admin.retries' > > was > > > >> > > supplied but isn't a known config. > > > >> > > (org.apache.kafka.clients.consumer.ConsumerConfig) > > > >> > > [2018-07-18 21:11:29,218] WARN [Producer > > > >> > > clientId=apache-wordcount-2a671de0-d2b7-404f-bfe8- > > > >> > 9e8cad5008d4-StreamThread-1-0_0-producer, > > > >> > > transactionalId=apache-wordcount-0_0] Got error produce response > > > with > > > >> > > correlation id 15 on topic-partition > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0, > > > >> > > retrying (2147483646 <(214)%20748-3646> <(214)%20748-3646> > attempts left). Error: > > > UNKNOWN_PRODUCER_ID > > > >> > > (org.apache.kafka.clients.producer.internals.Sender) > > > >> > > [2018-07-18 21:15:04,092] WARN [Producer > > > >> > > clientId=apache-wordcount-2a671de0-d2b7-404f-bfe8- > > > >> > 9e8cad5008d4-StreamThread-1-0_0-producer, > > > >> > > transactionalId=apache-wordcount-0_0] Got error produce response > > > with > > > >> > > correlation id 21 on topic-partition > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0, > > > >> > > retrying (2147483646 <(214)%20748-3646> <(214)%20748-3646> > attempts left). Error: > > > UNKNOWN_PRODUCER_ID > > > >> > > (org.apache.kafka.clients.producer.internals.Sender) > > > >> > > > > > >> > > There are also some ERROR message in the broker logs: > > > >> > > [2018-07-18 21:10:16,463] INFO Updated PartitionLeaderEpoch. > New: > > > >> > > {epoch:0, offset:0}, Current: {epoch:-1, offset:-1} for > Partition: > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0. > > > >> > > Cache now contains 0 entries. > > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:10:16,965] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 5 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:10:16,966] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 5 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:11:29,217] ERROR [ReplicaManager broker=0] Error > > > >> > processing > > > >> > > append operation on partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.ReplicaManager) > > > >> > > org.apache.kafka.common.errors.UnknownProducerIdException: Found > > no > > > >> > record > > > >> > > of producerId=5000 on the broker. It is possible that the last > > > message > > > >> > with > > > >> > > the producerId=5000 has been removed due to hitting the > retention > > > >> limit. > > > >> > > [2018-07-18 21:11:29,331] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 9 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:11:29,332] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 9 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:15:04,091] ERROR [ReplicaManager broker=0] Error > > > >> > processing > > > >> > > append operation on partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.ReplicaManager) > > > >> > > org.apache.kafka.common.errors.UnknownProducerIdException: Found > > no > > > >> > record > > > >> > > of producerId=5000 on the broker. It is possible that the last > > > message > > > >> > with > > > >> > > the producerId=5000 has been removed due to hitting the > retention > > > >> limit. > > > >> > > [2018-07-18 21:15:04,204] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 13 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:15:04,205] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 13 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > > > > >> > > I found the outputs of the WordCount app is correct. But each > > time I > > > >> send > > > >> > > a line to streams-wordcount-input, the Streams App throw a new > > > >> > > UNKNOWN_PRODUCER_ID error, and Broker also throw a new > > > >> > > UnknownProducerIdException. > > > >> > > The broker version I use is 1.1.0. Have anyone ecountered this > > > problem > > > >> > > before or can give me any hints about what might be causing this > > > >> > behaviour? > > > >> > > > > > >> > > Thanks, > > > >> > > lambdaliu > > > >> > > > > > >> > > > > >> > > > >> > > > >> > > > >> -- > > > >> -- Guozhang > > > >> > > > > > > >Created JIRA https://issues.apache.org/jira/browse/KAFKA-7190 > > > > > > > >-Bill > > > > > > > >On Fri, Jul 20, 2018 at 1:26 PM Guozhang Wang <wangg...@gmail.com> > > wrote: > > > > > > > >> Hello Shaobo, > > > >> > > > >> The situation you observed is a valid case: when a streams > application > > > has > > > >> very low traffic (like the quickstart example, where you manually > type > > > one > > > >> message at a time), then it is possible that consumer purging would > > > delete > > > >> even the last message sent by this producer (i.e. all the messages > > sent > > > by > > > >> this producer have been consumed and committed), and as a result the > > > broker > > > >> would delete that producer's ID. The next time when this producer > > tries > > > to > > > >> send, it will get this UNKNOWN_PRODUCER_ID error code, but in this > > case > > > >> this error is retriable: the producer would just get a new producer > id > > > and > > > >> retries, and then this time it will succeed. So the results you > > > observed is > > > >> still correct. > > > >> > > > >> > > > >> We can, probably, improve this situation either in broker side or > > > streams > > > >> client side: on broker side, we can consider delaying the deletion > of > > > the > > > >> producer ID for a while; on streams client side, we can consider > > > purging in > > > >> a bit conservative manner (but it is still a bit tricky, since > > multiple > > > >> producers may be sending to the same inner topic, so just leaving > the > > > last > > > >> N messages to not be purged may not be safe still). > > > >> > > > >> > > > >> Bill, could we create a JIRA for this? > > > >> > > > >> > > > >> Guozhang > > > >> > > > >> > > > >> On Thu, Jul 19, 2018 at 7:55 AM, Bill Bejeck <b...@confluent.io> > > wrote: > > > >> > > > >> > Hi > > > >> > > > > >> > Thanks for reporting this. Just off the top of my head, I'm > > thinking > > > it > > > >> > may have to do with using a console producer, but I'll have to > take > > a > > > >> > deeper look. > > > >> > > > > >> > Thanks, > > > >> > Bill > > > >> > > > > >> > On Thu, Jul 19, 2018 at 9:59 AM lambdaliu(刘少波) < > > lambda...@tencent.com > > > > > > > >> > wrote: > > > >> > > > > >> > > Hi, > > > >> > > > > > >> > > I test the Kafka Streams WordCount demo follow the steps > described > > > in > > > >> > > http://kafka.apache.org/11/documentation/streams/quickstart > with > > > >> change > > > >> > > the processing.guarantee property to EXACTLY_ONCE. > > > >> > > > > > >> > > And seeing the following WARN message in streams demo app logs: > > > >> > > [2018-07-18 21:08:03,510] WARN The configuration 'admin.retries' > > was > > > >> > > supplied but isn't a known config. > > > >> > > (org.apache.kafka.clients.consumer.ConsumerConfig) > > > >> > > [2018-07-18 21:11:29,218] WARN [Producer > > > >> > > clientId=apache-wordcount-2a671de0-d2b7-404f-bfe8- > > > >> > 9e8cad5008d4-StreamThread-1-0_0-producer, > > > >> > > transactionalId=apache-wordcount-0_0] Got error produce response > > > with > > > >> > > correlation id 15 on topic-partition > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0, > > > >> > > retrying (2147483646 <(214)%20748-3646> <(214)%20748-3646> > attempts left). Error: > > > UNKNOWN_PRODUCER_ID > > > >> > > (org.apache.kafka.clients.producer.internals.Sender) > > > >> > > [2018-07-18 21:15:04,092] WARN [Producer > > > >> > > clientId=apache-wordcount-2a671de0-d2b7-404f-bfe8- > > > >> > 9e8cad5008d4-StreamThread-1-0_0-producer, > > > >> > > transactionalId=apache-wordcount-0_0] Got error produce response > > > with > > > >> > > correlation id 21 on topic-partition > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0, > > > >> > > retrying (2147483646 <(214)%20748-3646> <(214)%20748-3646> > attempts left). Error: > > > UNKNOWN_PRODUCER_ID > > > >> > > (org.apache.kafka.clients.producer.internals.Sender) > > > >> > > > > > >> > > There are also some ERROR message in the broker logs: > > > >> > > [2018-07-18 21:10:16,463] INFO Updated PartitionLeaderEpoch. > New: > > > >> > > {epoch:0, offset:0}, Current: {epoch:-1, offset:-1} for > Partition: > > > >> > > > > > >> > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0. > > > >> > > Cache now contains 0 entries. > > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:10:16,965] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 5 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:10:16,966] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 5 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:11:29,217] ERROR [ReplicaManager broker=0] Error > > > >> > processing > > > >> > > append operation on partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.ReplicaManager) > > > >> > > org.apache.kafka.common.errors.UnknownProducerIdException: Found > > no > > > >> > record > > > >> > > of producerId=5000 on the broker. It is possible that the last > > > message > > > >> > with > > > >> > > the producerId=5000 has been removed due to hitting the > retention > > > >> limit. > > > >> > > [2018-07-18 21:11:29,331] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 9 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:11:29,332] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 9 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > [2018-07-18 21:15:04,091] ERROR [ReplicaManager broker=0] Error > > > >> > processing > > > >> > > append operation on partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.ReplicaManager) > > > >> > > org.apache.kafka.common.errors.UnknownProducerIdException: Found > > no > > > >> > record > > > >> > > of producerId=5000 on the broker. It is possible that the last > > > message > > > >> > with > > > >> > > the producerId=5000 has been removed due to hitting the > retention > > > >> limit. > > > >> > > [2018-07-18 21:15:04,204] INFO [Log > > > >> > > partition=apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE- > > > >> > 0000000003-repartition-0, > > > >> > > dir=/tmp/kafka-logs0] Incrementing log start offset to 13 > > > >> (kafka.log.Log) > > > >> > > [2018-07-18 21:15:04,205] INFO Cleared earliest 0 entries from > > epoch > > > >> > cache > > > >> > > based on passed offset 13 leaving 1 in EpochFile for partition > > > >> > > > > > apache-wordcount-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition-0 > > > >> > > (kafka.server.epoch.LeaderEpochFileCache) > > > >> > > > > > >> > > I found the outputs of the WordCount app is correct. But each > > time I > > > >> send > > > >> > > a line to streams-wordcount-input, the Streams App throw a new > > > >> > > UNKNOWN_PRODUCER_ID error, and Broker also throw a new > > > >> > > UnknownProducerIdException. > > > >> > > The broker version I use is 1.1.0. Have anyone ecountered this > > > problem > > > >> > > before or can give me any hints about what might be causing this > > > >> > behaviour? > > > >> > > > > > >> > > Thanks, > > > >> > > lambdaliu > > > >> > > > > > >> > > > > >> > > > >> > > > >> > > > >> -- > > > >> -- Guozhang > > > >> > > > > -- > > Nitay Kufert > > Backend Developer > > [image: ironSource] <http://www.ironsrc.com/> > > > > email nita...@ironsrc.com > > mobile +972-54-5480021 <+972%2054-548-0021> > > fax +972-77-5448273 <+972%2077-544-8273> > > skype nitay.kufert.ssa > > 9 Ehad Ha'am st. Tel- Aviv > > ironsrc.com <http://www.ironsrc.com/> > > [image: linkedin] <https://www.linkedin.com/company/ironsource>[image: > > twitter] <https://twitter.com/ironsource>[image: facebook] > > <https://www.facebook.com/ironSource>[image: googleplus] > > <https://plus.google.com/+ironsrc> > > This email (including any attachments) is for the sole use of the > intended > > recipient and may contain confidential information which may be protected > > by legal privilege. If you are not the intended recipient, or the > employee > > or agent responsible for delivering it to the intended recipient, you are > > hereby notified that any use, dissemination, distribution or copying of > > this communication and/or its content is strictly prohibited. If you are > > not the intended recipient, please immediately notify us by reply email > or > > by telephone, delete this email and destroy any copies. Thank you. > > > -- Nitay Kufert Backend Developer [image: ironSource] <http://www.ironsrc.com/> email nita...@ironsrc.com mobile +972-54-5480021 fax +972-77-5448273 skype nitay.kufert.ssa 9 Ehad Ha'am st. Tel- Aviv ironsrc.com <http://www.ironsrc.com/> [image: linkedin] <https://www.linkedin.com/company/ironsource>[image: twitter] <https://twitter.com/ironsource>[image: facebook] <https://www.facebook.com/ironSource>[image: googleplus] <https://plus.google.com/+ironsrc> This email (including any attachments) is for the sole use of the intended recipient and may contain confidential information which may be protected by legal privilege. If you are not the intended recipient, or the employee or agent responsible for delivering it to the intended recipient, you are hereby notified that any use, dissemination, distribution or copying of this communication and/or its content is strictly prohibited. If you are not the intended recipient, please immediately notify us by reply email or by telephone, delete this email and destroy any copies. Thank you.