[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17655155#comment-17655155 ] Chia-Ping Tsai commented on KAFKA-9087: --- [~junrao] Please take a look at [https://github.com/apache/kafka/pull/13075] > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 4224887 (kafka.cluster.Replica) > [2019-06-11 12:21:47,090] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:30:04,757] INFO [ReplicaFetcher replicaId=1, leaderId=2, > fetcherId=0] Re
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17655153#comment-17655153 ] Jun Rao commented on KAFKA-9087: [~chia7712] : Thanks for the explanation. Great find! I agree that this is a bug and the fix that you suggested makes sense. In alterReplicaLogDirs(), we initialize the initial offset for ReplicaAlterLogDirsThread with futureLog.highWatermark. We should do the same thing when handling the LeaderAndIsrRequest. It seems that the bug was introduced in this PR [https://github.com/apache/kafka/pull/6841|https://github.com/apache/kafka/pull/6841.]. Do you plan to submit a PR? > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654133#comment-17654133 ] Chia-Ping Tsai commented on KAFKA-9087: --- [~junrao] Sorry for late response. {quote}So, ReplicaAlterLogDirsThread is supposed to ignore the old fetched data and fetch again using the new fetch offset. I am wondering why that didn't happen. {quote} You are right. The true root cause is shown below. # tp-0 is located at broker-0:/tmp/data0 # move tp-0 from /tmp/data0 to /tmp/data1. It will create a new future log ([https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L765]) and ReplicaAlterLogDirsThread. The new future log does not have leader epoch before it sync data # file a partition reassignment to trigger LeaderAndIsrRequest request. The request will update the partition state of ReplicaAlterLogDirsThread ([https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L1565]), and the new offset of partition state is set with highWatermark of log # ReplicaAlterLogDirsThread uses the high watermark instead of OffsetsForLeaderEpoch API if there is no epoch cache. # The future log is new, so its end offset is 0. And the offset mismatch ( 0 v.s high watermark of log) causes the error. In short, the race condition of processing LeaderAndIsrRequest and AlterReplicaLogDirsRequest causes this error (on V2 message format). Also, the error can be reproduced easily on V1 since there is no epoch cache. I’m not sure why it used log.highWatermark ([https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L1559]). The ReplicaAlterLogDirsThread checks the offset of “future log” rather than “log. Hence, here is my two cents, we can replace log.highWatermark by futureLog.highWatermark to resolve this issue. I tested it on our cluster and it works well (on both V1 and V2). > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has bee
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17651012#comment-17651012 ] Jun Rao commented on KAFKA-9087: [~chia7712] : Thanks for the update. About the race condition. I am still wondering how we got into that state. Let's say ReplicaAlterLogDirsThread is in the about to append the fetched data to a future log when the log's dir is being changed. In this case, we will first remove the partition from the partitionState in ReplicaAlterLogDirsThread, recreate the future log and add the partition to ReplicaAlterLogDirsThread again. If ReplicaAlterLogDirsThread tries to append an old fetched data, it should fail the following test since the fetch offset in the fetch request and the currentFetchState will be different. [https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/AbstractFetcherThread.scala#L347] So, ReplicaAlterLogDirsThread is supposed to ignore the old fetched data and fetch again using the new fetch offset. I am wondering why that didn't happen. > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17650979#comment-17650979 ] Chia-Ping Tsai commented on KAFKA-9087: --- We encountered this error also. The root cause is about race condition. # ReplicaAlterLogDirsThread has fetched the data for topic partition # ReplicaManager#alterReplicaLogDirs changes the future log (the start offset is reset to 0) # ReplicaManager#alterReplicaLogDirs call AbstractFetcherManager#addFetcherForPartitions to add the topic partition (it just change the partition state in the ReplicaAlterLogDirsThread) # ReplicaAlterLogDirsThread starts to process the fetched data, and it throws IllegalStateException because the future log get renewed and start offset is zero. This bug causes the future log of topic partition can't get synced forever as the topic partition is marked as failed. It seems to me that we should return None instead of throwing IllegalStateException when start offset of future log is zero. [~junrao] WDYT? > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log st
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17575835#comment-17575835 ] Markus Werner commented on KAFKA-9087: -- I was able to reproduce the issue with Kafka 3.2 deployed via strimzi 0.30 in Kubernetes: # Create a second volume of type volume # Run `kafka-reassign-partitions.sh` moving partition from one volume to the other inside the replicas # The migration was stuck, so I deleted the broker {{2022-08-05 12:59:31,404 ERROR [ReplicaAlterLogDirsThread-1]: Unexpected error occurred while processing data for partition my-topic-0 at offset 59173818 (kafka.server.ReplicaAlterLogDirsThread) [ReplicaAlterLogDirsThread-1]}} {{java.lang.IllegalStateException: Offset mismatch for the future replica my-topic-0: fetched offset = 59173818, log end offset = 0.}} > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17249973#comment-17249973 ] Jun Rao commented on KAFKA-9087: Not sure if this is exactly the reason, but we did fix an issue (https://issues.apache.org/jira/browse/KAFKA-9654) related to ReplicaAlterLogDirs. Could you try and see if this issue still exists in some of the newer versions? > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 4224887 (kafka.cluster.Replica) > [2019-06-11 12:21:47,090] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17077566#comment-17077566 ] Matthew DiFabion commented on KAFKA-9087: - After many attempts over 2 weeks and lack of input here, I conclude that this method of partition migration is fundamentally broken. Here is what actually fixed my issue: # Create an additional replica across partitions in the desired destination log directory with kafka-reassign-partitions.sh. Wait until replication completes. # For each broker that has stuck migrations: ## Shut down the broker. ## Check that all partitions have an active leader (and that the shut down broker is not one of them). ## Delete both source and future (destination) topic partition directories (e.g. /data0/kafka/topic-0 and /data1/kafka/topic-0-aa308443b6ee4ec58a28d61abdde90ea-future) ## Start the broker. ## Ensure that the broker has created the destination topic partition directory only. ## Kafka will backfill this replica from the leader until both are in sync. # Use kafka-reassign-partitions.sh again to remove the replicas created in step 1. End state should be the replica reassignment state given in the original migration. In the future, I would stay away from migrating at all and just adding new replicas where they are needed before removing the unwanted ones. > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerSta
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17072960#comment-17072960 ] Matthew DiFabion commented on KAFKA-9087: - Has there been any movement on this? Also having the same issue. Would also welcome any workarounds; restarting the kafka daemon repeatedly to push a migration along is the only way that's working so far, but seems like a very risky and time-consuming method. > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 4224887 (kafka.cluster.Replica) > [2019-06-11 12:21:47,090] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data]
[jira] [Commented] (KAFKA-9087) ReplicaAlterLogDirs stuck and restart fails with java.lang.IllegalStateException: Offset mismatch for the future replica
[ https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16983768#comment-16983768 ] Vijay commented on KAFKA-9087: -- Is the above issue resolved? Am also facing same problem. [2019-11-27 17:07:24,201] INFO [Partition -1 broker=3] XXX-1 starts at Leader Epoch 103 from offset 10681297828. Previous Leader Epoch was: 102 (kafka.cluster.Partition) [2019-11-27 17:07:24,207] INFO [ReplicaAlterLogDirsThread-1]: Partition -1 has an older epoch (102) than the current leader. Will await the new LeaderAndIsr state before resuming fetching. (kafka.server.ReplicaAlterLogDirsThread) > ReplicaAlterLogDirs stuck and restart fails with > java.lang.IllegalStateException: Offset mismatch for the future replica > > > Key: KAFKA-9087 > URL: https://issues.apache.org/jira/browse/KAFKA-9087 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 2.2.0 >Reporter: Gregory Koshelev >Priority: Major > > I've started multiple replica movements between log directories and some > partitions were stuck. After the restart of the broker I've got exception in > server.log: > {noformat} > [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to > (kafka.server.ReplicaAlterLogDirsThread) > org.apache.kafka.common.KafkaException: Error processing data for partition > metrics_timers-35 offset 4224887 > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299) > at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) > at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299) > at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131) > at scala.Option.foreach(Option.scala:274) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131) > at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) > Caused by: java.lang.IllegalStateException: Offset mismatch for the future > replica metrics_timers-35: fetched offset = 4224887, log end offset = 0. > at > kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311) > ... 16 more > [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped > (kafka.server.ReplicaAlterLogDirsThread) > {noformat} > Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix > the problem. To fix it I've stopped the broker and remove all the stuck > future partitions. > Detailed log below > {noformat} > [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest > offset in the log is 4224886 (kafka.log.Log) > [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Loading producer state till offset 4224887 with > message format version 2 (kafka.log.Log) > [2019-06-11 12:21:34,980] INFO [ProducerStateManager > partition=metrics_timers-35] Loading producer state from snapshot file > '/storage2/kafka/data/metrics_timers-35/04224887.snapshot' > (kafka.log.ProducerStateManager) > [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35, > dir=/storage2/kafka/data] Completed load of log with 1 segments, log start > offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35 > with initial high watermark 0 (kafka.cluster.Replica) > [2019-06-11 12:21:45,307] INFO Replica loaded for