[ 
https://issues.apache.org/jira/browse/KAFKA-12835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17598006#comment-17598006
 ] 

Sergey Ivanov commented on KAFKA-12835:
---------------------------------------

Hi All,

It seems this ticket already solved, but I'd like to add my case too.


We use Kafka 2.8.1, last upgrade from 2.7.1 to 2.8.1 was performed a few weeks 
ago. 
And yesterday we found that Kafka consumes all provided CPU, and a lot of Kafka 
clients don't work and failed by timeout. Default logs didn't show any useful 
information, but in state-change logs we found a lot of rows with:


{code:java}
[2022-08-30 17:33:19,804] ERROR [Broker id=3] Topic Id in memory: 
n1ld3W-pSMqWMS5ceX_v-Q does not match the topic Id for partition 
__consumer_offsets-14 provided in the request: LzjEgQA5Sw29x-cayfXYEw. 
(state.change.logger){code}

for different topics, but in the most part for __consumer_offsets. We checked 
and yes, topic_id was different for "patition.metadata" on broker side and 
zookeeper. 

Also we found, that a lof of topics were in under ISR status, for example:
{code:java}
Topic: __consumer_offsets       TopicId: LzjEgQA5Sw29x-cayfXYEw PartitionCount: 
50      ReplicationFactor: 3    Configs: 
compression.type=producer,min.insync.replicas=2,cleanup.policy=compact,segment.bytes=104857600,max.message.bytes=10000000,retention.bytes=1610612735
        Topic: __consumer_offsets       Partition: 0    Leader: 1       
Replicas: 1,3,2 Isr: 1
        Topic: __consumer_offsets       Partition: 1    Leader: 3       
Replicas: 2,1,3 Isr: 3
        Topic: __consumer_offsets       Partition: 2    Leader: 1       
Replicas: 3,2,1 Isr: 1
        Topic: __consumer_offsets       Partition: 3    Leader: 1       
Replicas: 1,2,3 Isr: 1
        Topic: __consumer_offsets       Partition: 4    Leader: 3       
Replicas: 2,3,1 Isr: 3
        Topic: __consumer_offsets       Partition: 5    Leader: 1       
Replicas: 3,1,2 Isr: 1
        Topic: __consumer_offsets       Partition: 6    Leader: 2       
Replicas: 1,3,2 Isr: 2
....
        Topic: __consumer_offsets       Partition: 48   Leader: 2       
Replicas: 1,3,2 Isr: 2
        Topic: __consumer_offsets       Partition: 49   Leader: 3       
Replicas: 2,1,3 Isr: 3{code}
and this didn't change for a lot of time. 


While Kafka was loaded in max CPU it couldn't answer clients correctly, 
especially consumers, because offsets topic was under ISR we think. So, all 
related to Kafka services are stuck.

Thanks to this ticket, we solved a part of issues with removing 
"partition.metadata" and restart cluster. But Kafka still feels bad and some of 
partitions still have only one ISR with issue "Non-monotonic update of high 
watermark". It looks like another issue KAFKA-13077

Is it expected, that we got this issue in 2.8.1 Kafka server?

> Topic IDs can mismatch on brokers (after interbroker protocol version update)
> -----------------------------------------------------------------------------
>
>                 Key: KAFKA-12835
>                 URL: https://issues.apache.org/jira/browse/KAFKA-12835
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 2.8.0
>            Reporter: Ivan Yurchenko
>            Assignee: Justine Olshan
>            Priority: Major
>             Fix For: 2.8.1, 3.0.0
>
>
> We had a Kafka cluster running 2.8 version with interbroker protocol set to 
> 2.7. It had a number of topics and everything was fine.
> Then we decided to update the interbroker protocol to 2.8 by the following 
> procedure:
> 1. Run new brokers with the interbroker protocol set to 2.8.
> 2. Move the data from the old brokers to the new ones (normal partition 
> reassignment API).
> 3. Decommission the old brokers.
> At the stage 2 we had the problem: old brokers started failing on 
> {{LeaderAndIsrRequest}} handling with
> {code:java}
> ERROR [Broker id=<...>] Topic Id in memory: <...> does not match the topic Id 
> for partition <...> provided in the request: <...>. (state.change.logger)
> {code}
> for multiple topics. Topics were not recreated.
> We checked {{partition.metadata}} files and IDs there were indeed different 
> from the values in ZooKeeper. It was fixed by deleting the metadata files 
> (and letting them be recreated).
>  
> The logs, unfortunately, didn't show anything that might point to the cause 
> of the issue (or it happened longer ago than we store the logs).
> We tried to reproduce this also, but no success.
> If the community can point out what to check or beware of in future, it will 
> be great. We'll be happy to provide additional information if needed. Thank 
> you! 
> Sorry for the ticket that might be not very actionable. We hope to at least 
> rise awareness of this issue.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to