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-repartition-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/%3CCA
> M0VdefApmc5wBZQaJmQtbcnZ_OOgGv84qCuPoJS-KU4B=e...@mail.gmail.com%3E
>
>  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+Improve+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+Return+
> > to
> > +default+segment.ms+and+segment.index.bytes+in+Streams+repartition+t
> > +default+op
> > ics
> >
> > kafka-topics --zookeeper localhost:2181 --alter --topic 
> > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartition
> > --config segment.index.bytes=10485760 kafka-topics --zookeeper
> > localhost:2181 --alter --topic
> > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartition
> > --config segment.bytes= 52428800 kafka-topics --zookeeper
> > localhost:2181 --alter --topic
> > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartition
> > --config segment.ms=604800000 kafka-topics --zookeeper 
> > localhost:2181 --alter --topic 
> > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartition
> > --config retention.ms=-1
> >
> > Verifying afterwards:
> >
> > kafka-topics --zookeeper localhost:2181 --describe --topic 
> > event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repartition
> >
> > Topic:event-rule-engine-KSTREAM-REDUCE-STATE-STORE-0000000015-repart
> > 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-c62993315b30-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-repartitio
> > > n-
> > > 24
> > > to 0
> > >
> > >
> > >
> > > Followed by:
> > >
> > >
> > >
> > > [Producer
> > > clientId=event-rule-engine-dd71ae9b-523c-425d-a7c0-c62993315b30-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-repartitio
> > > 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-repartitio
> > > 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

Reply via email to