[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16249661#comment-16249661 ]
Ben Corlett commented on KAFKA-6194: ------------------------------------ Sorry for not getting back to you about getting you the logs. I need to confirm with my manager I'm afraid. If I do get the go ahead they will need to be sent privately. On Friday I did downgrade to 0.11.0.1 on all servers apart from broker 125 which is on a heap patched build of 1.0.0. Thankfully all servers have remained up since then. Checking for that log message I can see 3 incidents since the servers have been downgraded. The last one was on broker 128. {code} [2017-11-11 00:51:48,135] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXX-90/00000000000029081095.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:48,680] INFO Rolled new log segment for 'XXXXXX-90' in 4 ms. (kafka.log.Log) [2017-11-11 00:51:49,224] INFO Deleting segment 0 from log XXXXXXX-37. (kafka.log.Log) [2017-11-11 00:51:49,226] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXX-37/00000000000000000000.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,227] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXX-37/00000000000000000000.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,226] INFO Deleting segment 7481858 from log XXXXXXXX-37. (kafka.log.Log) [2017-11-11 00:51:49,230] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-37/00000000000007481858.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,230] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-37/00000000000007481858.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,373] INFO Deleting segment 0 from log XXXXXXXXX-90. (kafka.log.Log) [2017-11-11 00:51:49,374] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-90/00000000000000000000.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,374] INFO Deleting segment 29081095 from log XXXXXXXXX-90. (kafka.log.Log) [2017-11-11 00:51:49,376] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-90/00000000000029081095.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,376] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-90/00000000000000000000.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,377] INFO Deleting index /mnt/secure/kafka/datalog/XXXXXXXXX-90/00000000000029081095.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,379] ERROR Uncaught exception in scheduled task 'delete-file' (kafka.utils.KafkaScheduler) kafka.common.KafkaStorageException: Delete of log 00000000000000000000.log.deleted failed. at kafka.log.LogSegment.delete(LogSegment.scala:505) at kafka.log.Log.deleteSeg$1(Log.scala:1490) at kafka.log.Log.$anonfun$asyncDeleteSegment$2(Log.scala:1492) at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:57) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) 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} As you said the server remained up and carried on. > Server crash while deleting segments > ------------------------------------ > > Key: KAFKA-6194 > URL: https://issues.apache.org/jira/browse/KAFKA-6194 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 1.0.0 > Environment: kafka version: 1.0 > client versions: 0.8.2.1-0.10.2.1 > platform: aws (eu-west-1a) > nodes: 36 x r4.xlarge > disk storage: 2.5 tb per node (~73% usage per node) > topics: 250 > number of partitions: 48k (approx) > os: ubuntu 14.04 > jvm: Java(TM) SE Runtime Environment (build 1.8.0_131-b11), Java HotSpot(TM) > 64-Bit Server VM (build 25.131-b11, mixed mode) > Reporter: Ben Corlett > Labels: regression > > We upgraded our R+D cluster to 1.0 in the hope that it would fix the deadlock > from 0.11.0.*. Sadly our cluster has the memory leak issues with 1.0 most > likely from https://issues.apache.org/jira/browse/KAFKA-6185. We are running > one server on a patched version of 1.0 with the pull request from that. > However today we have had two different servers fall over for non-heap > related reasons. The exceptions in the kafka log are : > {code} > [2017-11-09 15:32:04,037] ERROR Error while deleting segments for > xxxxxxxxxx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log 00000000000000000000.log.deleted failed. > at kafka.log.LogSegment.delete(LogSegment.scala:496) > at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596) > at > scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12) > at kafka.log.Log.maybeHandleIOException(Log.scala:1669) > at kafka.log.Log.deleteSeg$1(Log.scala:1596) > at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599) > at > kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110) > at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) > 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) > [2017-11-09 15:32:04,040] INFO [ReplicaManager broker=122] Stopping serving > replicas in dir /mnt/secure/kafka/datalog (kafka.server.ReplicaManager) > [2017-11-09 15:32:04,041] ERROR Uncaught exception in scheduled task > 'delete-file' (kafka.utils.KafkaScheduler) > org.apache.kafka.common.errors.KafkaStorageException: Error while deleting > segments for xxxxxxxxxxxxxx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > 00000000000000000000.log.deleted failed. > at kafka.log.LogSegment.delete(LogSegment.scala:496) > at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596) > at > scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12) > at kafka.log.Log.maybeHandleIOException(Log.scala:1669) > at kafka.log.Log.deleteSeg$1(Log.scala:1596) > at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599) > at > kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110) > at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) > 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) > ..... > [2017-11-09 15:32:05,341] ERROR Error while processing data for partition > xxxxxxx-83 (kafka.server.ReplicaFetcherThread) > org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an > offline log directory for partition xxxxxxx-83 > [2017-11-09 15:32:05,341] ERROR Error while processing data for partition > xxxxxxx-89 (kafka.server.ReplicaFetcherThread) > org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an > offline log directory for partition xxxxxxx-89 > [2017-11-09 15:32:05,341] ERROR Error while processing data for partition > xxxxxxx-76 (kafka.server.ReplicaFetcherThread) > ..... > 2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the > replica LEO, the partition xxxxxxx-27 hasn't been created. > (kafka.server.ReplicaManager) > [2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording > the replica LEO, the partition xxxxxxxxx-79 hasn't been created. > (kafka.server.ReplicaManager) > [2017-11-09 15:32:05,622] FATAL Shutdown broker because all log dirs in > /mnt/secure/kafka/datalog have failed (kafka.log.LogManager) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)