[ https://issues.apache.org/jira/browse/KAFKA-15851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17807561#comment-17807561 ]
Yu Wang commented on KAFKA-15851: --------------------------------- got similar stack trace in 2.5.1, recovered after restart. {code:java} java.nio.BufferOverflowException at java.nio.Buffer.nextPutIndex(Buffer.java:533) at java.nio.DirectByteBuffer.putLong(DirectByteBuffer.java:796) at kafka.log.TimeIndex.$anonfun$maybeAppend$1(TimeIndex.scala:134) at kafka.log.TimeIndex.maybeAppend(TimeIndex.scala:114) at kafka.log.LogSegment.onBecomeInactiveSegment(LogSegment.scala:507) at kafka.log.Log.$anonfun$roll$8(Log.scala:1900) at kafka.log.Log.$anonfun$roll$2(Log.scala:1900) at kafka.log.Log.roll(Log.scala:2322) at kafka.log.Log.maybeRoll(Log.scala:1849) at kafka.log.Log.$anonfun$append$2(Log.scala:1148) at kafka.log.Log.append(Log.scala:2322) at kafka.log.Log.appendAsFollower(Log.scala:1017) at kafka.cluster.Partition.$anonfun$doAppendRecordsToFollowerOrFutureReplica$1(Partition.scala:924) at kafka.cluster.Partition.doAppendRecordsToFollowerOrFutureReplica(Partition.scala:917) at kafka.cluster.Partition.appendRecordsToFollowerOrFutureReplica(Partition.scala:931) at kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:167) at kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:332) at kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:320) at kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:319) at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:553) at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:551) at scala.collection.AbstractIterable.foreach(Iterable.scala:921) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:319) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:135) at kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:134) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:117) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) {code} > broker under replicated due to error java.nio.BufferOverflowException > --------------------------------------------------------------------- > > Key: KAFKA-15851 > URL: https://issues.apache.org/jira/browse/KAFKA-15851 > Project: Kafka > Issue Type: Bug > Affects Versions: 3.3.2 > Environment: Kafka Version: 3.3.2 > Deployment mode: zookeeper > Reporter: wangliucheng > Priority: Major > Attachments: p1.png > > > In my kafka cluster, kafka update 2.0 to 3.3.2 version > {*}first start failed{*}, because the same directory was configured > The error is as follows: > > {code:java} > [2023-11-16 10:04:09,952] ERROR (main kafka.Kafka$ 159) Exiting Kafka due to > fatal exception during startup. > java.lang.IllegalStateException: Duplicate log directories for > skydas_sc_tdevirsec-12 are found in both > /data01/kafka/log/skydas_sc_tdevirsec-12 and > /data07/kafka/log/skydas_sc_tdevirsec-12. It is likely because log directory > failure happened while broker was replacing current replica with future > replica. Recover broker from this failure by manually deleting one of the two > directories for this partition. It is recommended to delete the partition in > the log directory that is known to have failed recently. > at kafka.log.LogManager.loadLog(LogManager.scala:305) > at kafka.log.LogManager.$anonfun$loadLogs$14(LogManager.scala:403) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > [2023-11-16 10:04:09,953] INFO (kafka-shutdown-hook kafka.server.KafkaServer > 66) [KafkaServer id=1434] shutting down {code} > > > *second,* remove /data07/kafka/log in log.dirs and start kafka also reported > an error : > > {code:java} > [2023-11-16 10:13:10,713] INFO (ReplicaFetcherThread-3-1008 > kafka.log.UnifiedLog 66) [UnifiedLog partition=ty_udp_full-60, > dir=/data04/kafka/log] Rolling new log segment (log_size = > 755780551/1073741824}, offset_index_size = 2621440/2621440, time_index_size = > 1747626/1747626, inactive_time_ms = 2970196/604800000). > [2023-11-16 10:13:10,714] ERROR (ReplicaFetcherThread-3-1008 > kafka.server.ReplicaFetcherThread 76) [ReplicaFetcher replicaId=1434, > leaderId=1008, fetcherId=3] Unexpected error occurred while processing data > for partition ty_udp_full-60 at offset 2693467479 > java.nio.BufferOverflowException > at java.nio.Buffer.nextPutIndex(Buffer.java:555) > at java.nio.DirectByteBuffer.putLong(DirectByteBuffer.java:794) > at kafka.log.TimeIndex.$anonfun$maybeAppend$1(TimeIndex.scala:135) > at kafka.log.TimeIndex.maybeAppend(TimeIndex.scala:114) > at kafka.log.LogSegment.onBecomeInactiveSegment(LogSegment.scala:510) > at kafka.log.LocalLog.$anonfun$roll$9(LocalLog.scala:529) > at kafka.log.LocalLog.$anonfun$roll$9$adapted(LocalLog.scala:529) > at scala.Option.foreach(Option.scala:437) > at kafka.log.LocalLog.$anonfun$roll$2(LocalLog.scala:529) > at kafka.log.LocalLog.roll(LocalLog.scala:786) > at kafka.log.UnifiedLog.roll(UnifiedLog.scala:1537) > at kafka.log.UnifiedLog.maybeRoll(UnifiedLog.scala:1523) > at kafka.log.UnifiedLog.append(UnifiedLog.scala:919) > at kafka.log.UnifiedLog.appendAsFollower(UnifiedLog.scala:778) > at > kafka.cluster.Partition.doAppendRecordsToFollowerOrFutureReplica(Partition.scala:1121) > at > kafka.cluster.Partition.appendRecordsToFollowerOrFutureReplica(Partition.scala:1128) > at > kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:121) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:336) > at scala.Option.foreach(Option.scala:437) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:325) > at > kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:324) > at > kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62) > at > scala.collection.convert.JavaCollectionWrappers$JMapWrapperLike.foreachEntry(JavaCollectionWrappers.scala:359) > at > scala.collection.convert.JavaCollectionWrappers$JMapWrapperLike.foreachEntry$(JavaCollectionWrappers.scala:355) > at > scala.collection.convert.JavaCollectionWrappers$AbstractJMapWrapper.foreachEntry(JavaCollectionWrappers.scala:309) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:324) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:124) > at > kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:123) > at scala.Option.foreach(Option.scala:437) > at > kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:123) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:106) > at > kafka.server.ReplicaFetcherThread.doWork(ReplicaFetcherThread.scala:97) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) > [2023-11-16 10:13:10,714] WARN (ReplicaFetcherThread-3-1008 > kafka.server.ReplicaFetcherThread 70) [ReplicaFetcher replicaId=1434, > leaderId=1008, fetcherId=3] Partition ty_udp_full-60 marked as failed > {code} > > start again and finally run normally but log rolled > I want to know how this error occurred, I suspect that the status of the > timeindex file is incorrect, > Thanks > The detailed log information can be found in server.log > > > -- This message was sent by Atlassian Jira (v8.20.10#820010)