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

 Brajesh Kumar commented on KAFKA-5381:
---------------------------------------

We have seen this issue with Kafka 2.3.1. Broker also experienced memory 
pressure on host(A day before we observed this problem).

 
{quote}[2020-11-06 16:55:07,593] ERROR Uncaught exception in scheduled task 
'isr-expiration' (kafka.utils.KafkaScheduler)
org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = 
Session expired for /brokers/topics/<topic_name>/partitions/0/state
 at org.apache.zookeeper.KeeperException.create(KeeperException.java:130)
 at org.apache.zookeeper.KeeperException.create(KeeperException.java:54)
 at kafka.zookeeper.AsyncResponse.resultException(ZooKeeperClient.scala:553)
 at kafka.zk.KafkaZkClient.conditionalUpdatePath(KafkaZkClient.scala:741)
 at kafka.utils.ReplicationUtils$.updateLeaderAndIsr(ReplicationUtils.scala:34)
 at kafka.cluster.Partition.updateIsr(Partition.scala:1021)
 at kafka.cluster.Partition.$anonfun$maybeShrinkIsr$1(Partition.scala:672)
 at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
 at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
 at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:261)
 at kafka.cluster.Partition.maybeShrinkIsr(Partition.scala:654)
 at 
kafka.server.ReplicaManager.$anonfun$maybeShrinkIsr$3(ReplicaManager.scala:1384)
 at 
kafka.server.ReplicaManager.$anonfun$maybeShrinkIsr$3$adapted(ReplicaManager.scala:1384)
 at scala.Option.foreach(Option.scala:407)
 at 
kafka.server.ReplicaManager.$anonfun$maybeShrinkIsr$2(ReplicaManager.scala:1384)
 at 
kafka.server.ReplicaManager.$anonfun$maybeShrinkIsr$2$adapted(ReplicaManager.scala:1383)
 at scala.collection.Iterator.foreach(Iterator.scala:941)
 at scala.collection.Iterator.foreach$(Iterator.scala:941)
 at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
 at scala.collection.IterableLike.foreach(IterableLike.scala:74)
 at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
 at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
 at kafka.server.ReplicaManager.maybeShrinkIsr(ReplicaManager.scala:1383)
{quote}

> ERROR Uncaught exception in scheduled task 'delete-expired-consumer-offsets' 
> (kafka.utils.KafkaScheduler)
> ---------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5381
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5381
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Jeff Widman
>            Priority: Major
>
> We have a 6 node cluster of 0.10.0.1 brokers. Broker 4 had a hardware 
> problem, so we re-assigned all its partitions to other brokers. We 
> immediately started observing the error described in KAFKA-4362 from several 
> of our consumers.
> However, on broker 6, we also started seeing the following exceptions in 
> {{KafkaScheduler}} which have a somewhat similar-looking traceback:
> {code}
> [2017-06-03 17:23:57,926] ERROR Uncaught exception in scheduled task 
> 'delete-expired-consumer-offsets' (kafka.utils.KafkaScheduler)
> java.lang.IllegalArgumentException: Message format version for partition 50 
> not found
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$14.apply(GroupMetadataManager.scala:633)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$14.apply(GroupMetadataManager.scala:633)
>         at scala.Option.getOrElse(Option.scala:121)
>         at 
> kafka.coordinator.GroupMetadataManager.kafka$coordinator$GroupMetadataManager$$getMessageFormatVersionAndTimestamp(GroupMetadataManager.scala:632)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$2$$anonfun$10.apply(GroupMetadataManager.scala:560)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$2$$anonfun$10.apply(GroupMetadataManager.scala:551)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
>         at scala.collection.immutable.List.foreach(List.scala:381)
>         at 
> scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>         at scala.collection.immutable.List.map(List.scala:285)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$2.apply$mcI$sp(GroupMetadataManager.scala:551)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$2.apply(GroupMetadataManager.scala:543)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$2.apply(GroupMetadataManager.scala:543)
>         at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:231)
>         at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:239)
>         at 
> kafka.coordinator.GroupMetadataManager.kafka$coordinator$GroupMetadataManager$$deleteExpiredOffsets(GroupMetadataManager.scala:543)
>         at 
> kafka.coordinator.GroupMetadataManager$$anonfun$1.apply$mcV$sp(GroupMetadataManager.scala:87)
>         at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:56)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:748)
> {code}
> Unsurprisingly, the error disappeared once {{offsets.retention.minutes}} 
> passed.
> This appears to be similar root cause to KAFKA-4362 where 
> {{GroupMetadataManager.getMessageFormatVersionAndTimestamp}} is throwing the 
> error due to the offset partition being moved, but I'm unclear whether the 
> fix for that version also fixed the {{KafkaScheduler}} or if more work needs 
> to be done here.
> We did the partition re-assignment by using the 
> {{kafka-reassign-partitions.sh}} script and giving it the five healthy 
> brokers. From my understanding, this would have randomly re-assigned all 
> partitions (I don't think its sticky), so probably at least one partition 
> from the {{__consumer_offsets}} topic was removed from broker 6. However, if 
> that was the case, I would have expected all brokers to have had these 
> partitions removed and be throwing this error. But our logging infrastructure 
> shows that this error was only happening on broker 6, not on the other 
> brokers. Not sure why that is.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to