You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Anty Rao <an...@gmail.com> on 2021/06/02 03:33:18 UTC

Re: UNKNOWN_PRODUCER_ID error when running Streams WordCount demo with processing.guarantee set to EXACTLY_ONCE

unsubscribe

On Sat, Jul 21, 2018 at 1:26 AM Guozhang Wang <wa...@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 <bi...@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(刘少波) <la...@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