unsubscribe On Sat, Jul 21, 2018 at 1:26 AM 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 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 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 > -- Anty Rao