[jira] [Commented] (KAFKA-6264) Log cleaner thread may die on legacy segment containing messages whose offsets are too large
[ https://issues.apache.org/jira/browse/KAFKA-6264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16362105#comment-16362105 ] Ben Corlett commented on KAFKA-6264: In our production cluster we are running 0.10.2.1 and are seeing the log cleaning fail on 3 of the boxes with: {code:java} [2018-02-07 14:40:23,820] INFO Cleaner 0: Cleaning segment 0 in log __consumer_offsets-17 (largest timestamp Wed Aug 09 07:28:11 BST 2017) into 0, discarding deletes. (kafka.log.LogCleaner) [2018-02-07 14:40:23,830] ERROR [kafka-log-cleaner-thread-0]: Error due to (kafka.log.LogCleaner) java.lang.IllegalArgumentException: requirement failed: largest offset in message set can not be safely converted to relative offset. at scala.Predef$.require(Predef.scala:277) at kafka.log.LogSegment.append(LogSegment.scala:121) at kafka.log.Cleaner.cleanInto(LogCleaner.scala:551) at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:444) at kafka.log.Cleaner.$anonfun$doClean$6(LogCleaner.scala:385) at kafka.log.Cleaner.$anonfun$doClean$6$adapted(LogCleaner.scala:384) at scala.collection.immutable.List.foreach(List.scala:389) at kafka.log.Cleaner.doClean(LogCleaner.scala:384) at kafka.log.Cleaner.clean(LogCleaner.scala:361) at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:256) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:236) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) [2018-02-07 14:40:23,833] INFO [kafka-log-cleaner-thread-0]: Stopped (kafka.log.LogCleaner) {code} I can see high fd counts on these servers: {code:java} ssh xxx 'sudo ls -latr /proc/`pgrep java`/fd | wc -l' 130149 ssh xxx 'sudo ls -latr /proc/`pgrep java`/fd | wc -l' 147455 ssh xxx 'sudo ls -latr /proc/`pgrep java`/fd | wc -l' 155521 {code} I've tried several restarts. The log cleaner would fall over each time. I tried to upgrade one of the affected servers from 0.10.2.1 to 0.11.0.2. The log cleaner still failed. I'm guessing I'm going to have to hack the files on the filesystem. Looking at the affected partition: {code:java} -rw-r--r-- 1 kafka root 122372 Aug 10 2017 .log -rw-r--r-- 1 kafka root 2424 Aug 14 2017 004019345048.log -rw-r--r-- 1 kafka root 20956142 Aug 15 07:28 004020192019.log -rw-r--r-- 1 kafka root 20986067 Aug 16 07:28 004020403517.log -rw-r--r-- 1 kafka root 20984625 Aug 17 07:28 004020615318.log ... -rw-r--r-- 1 kafka kafka 184 Feb 7 14:39 .index -rw-r--r-- 1 kafka root 0 Feb 7 14:36 004019345048.index -rw-r--r-- 1 kafka root 40208 Feb 7 14:36 004020192019.index -rw-r--r-- 1 kafka root 40328 Feb 7 14:36 004020403517.index -rw-r--r-- 1 kafka root 40336 Feb 7 14:36 004020615318.index ... {code} I guess I'm looking for some advice on how to fix this. Should I just remove the "" files. Loosing Consumer offsets not updated since 10th of Aug 2017 shouldn't be an issue. Or should I try to empty these files? Try to figure out the starting offset of these files? Thanks > Log cleaner thread may die on legacy segment containing messages whose > offsets are too large > > > Key: KAFKA-6264 > URL: https://issues.apache.org/jira/browse/KAFKA-6264 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.10.2.1, 1.0.0, 0.11.0.2 >Reporter: Jiangjie Qin >Assignee: Jiangjie Qin >Priority: Critical > Fix For: 1.2.0 > > > We encountered a problem that some of the legacy log segments contains > messages whose offsets are larger than {{SegmentBaseOffset + Int.MaxValue}}. > Prior to 0.10.2.0, we do not assert the offset of the messages when appending > them to the log segments. Due to KAFKA-5413, the log cleaner may append > messages whose offset is greater than {{base_offset + Int.MaxValue}} into the > segment during the log compaction. > After the brokers are upgraded, those log segments cannot be compacted > anymore because the compaction will fail immediately due to the offset range > assertion we added to the LogSegment. > We have seen this issue in the __consumer_offsets topic so it could be a > general problem. There is no easy solution for the users to recover from this > case. > One solution is to split such log segments in the log cleaner once it sees a > message with problematic offset and append those messages to a separate log > segment with a larger base_offset. > Due to the impact of the issue. We may want to consider backporting the fix > to previous affected versions. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16251400#comment-16251400 ] Ben Corlett commented on KAFKA-6194: Topic1 from that log has settings: min.cleanable.dirty.ratio 0.05 cleanup.policy compact retention.ms1000 delete.retention.ms 60 segment.bytes 500 Yes. These settings were done by the devs presumably they were looking for very aggressive compaction. > 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 > Attachments: server.log.2017-11-14-03.gz > > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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)
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16251371#comment-16251371 ] Ben Corlett commented on KAFKA-6194: In the past topics have been deleted and recreated with the same name. Unfortunately this has resulted in "lingering" topics that had to manually be removed from zookeeper. These days we version the topics and create new ones with new names. It might still happen but very rarely will we do this. In the last 24 hours we have had another 15 failures. I've sanitised (renamed the topics) the log file from one of them and attached it to the issue. {code} [2017-11-14 03:33:30,238] ERROR Uncaught exception in scheduled task 'delete-file' (kafka.utils.KafkaScheduler) kafka.common.KafkaStorageException: Delete of log .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} I had a brief look though the code and I can see what you mean about the exception only happening if the delete fails and the file still exists. As you can see from the logs we have very aggressive rotation/deletion/compaction of 'topic1'. Probably way more than actually needed. But thats the devs for you. I wonder if we have a sequence of two deletes followed by a compaction finishing and a new .delete file appearing so: 1. delete the file (succeeds) 2. delete the file (fails) 3. compaction finishes and new .delete file created 4. check file exists... it does ... throw exception. Another thing. It always seems to be for .log.deleted on all the different servers. Maybe we could enable additional logging. > 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 > Attachments: server.log.2017-11-14-03.gz > > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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
[jira] [Updated] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Corlett updated KAFKA-6194: --- Attachment: server.log.2017-11-14-03.gz > 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 > Attachments: server.log.2017-11-14-03.gz > > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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 > xxx-83 (kafka.server.ReplicaFetcherThread) > org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an > offline log directory for partition xxx-83 > [2017-11-09 15:32:05,341] ER
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16249677#comment-16249677 ] Ben Corlett commented on KAFKA-6194: Just had the thought about if it was trying to delete the log file twice. Stripping some of the logs from that last incident. {code} [2017-11-11 00:51:47,940] INFO Deleting segment 29071939 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29073106 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29074245 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29075369 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29076527 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29077660 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29078814 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,940] INFO Deleting segment 29079944 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:47,938] INFO Deleting segment 29068465 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:48,133] INFO Deleting segment 29081095 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:49,224] INFO Deleting segment 0 from log topic2-37. (kafka.log.Log) [2017-11-11 00:51:49,226] INFO Deleting segment 7481858 from log topic2-37. (kafka.log.Log) [2017-11-11 00:51:49,373] INFO Deleting segment 0 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:49,374] INFO Deleting segment 29081095 from log topic1-90. (kafka.log.Log) [2017-11-11 00:51:49,379] ERROR Uncaught exception in scheduled task 'delete-file' (kafka.utils.KafkaScheduler) {code} 2 delete lines for 29081095 from log topic1-90 I looked at another incident {code} [2017-11-10 20:36:33,687] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29439517.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:33,775] INFO Deleting segment 29440823 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:33,776] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29440823.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:33,776] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29440823.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:33,894] INFO Deleting segment 29442113 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:33,896] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29442113.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:33,896] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29442113.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:34,013] INFO Rolled new log segment for 'topic1-84' in 0 ms. (kafka.log.Log) [2017-11-10 20:36:34,044] INFO Deleting segment 29443416 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,045] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29443416.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,045] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29443416.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:34,179] INFO Deleting segment 0 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,179] INFO Deleting segment 29443416 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,180] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,179] INFO Deleting segment 29442113 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,179] INFO Deleting segment 29440823 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,180] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29442113.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,180] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29440823.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,179] INFO Deleting segment 29439517 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,180] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29440823.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:34,181] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29439517.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,181] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29439517.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 20:36:34,179] INFO Deleting segment 29437182 from log topic1-84. (kafka.log.Log) [2017-11-10 20:36:34,181] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29437182.index.deleted (kafka.log.OffsetIndex) [2017-11-10 20:36:34,181] INFO Deleting index /mnt/secure/kafka/datalog/topic1-84/29437182.timeindex.deleted (kafka.log.TimeIndex) [2017-11-10 2
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ 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/XX-90/29081095.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:48,680] INFO Rolled new log segment for 'XX-90' in 4 ms. (kafka.log.Log) [2017-11-11 00:51:49,224] INFO Deleting segment 0 from log XXX-37. (kafka.log.Log) [2017-11-11 00:51:49,226] INFO Deleting index /mnt/secure/kafka/datalog/-37/.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,227] INFO Deleting index /mnt/secure/kafka/datalog/-37/.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,226] INFO Deleting segment 7481858 from log -37. (kafka.log.Log) [2017-11-11 00:51:49,230] INFO Deleting index /mnt/secure/kafka/datalog/X-37/07481858.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,230] INFO Deleting index /mnt/secure/kafka/datalog/X-37/07481858.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,373] INFO Deleting segment 0 from log X-90. (kafka.log.Log) [2017-11-11 00:51:49,374] INFO Deleting index /mnt/secure/kafka/datalog/X-90/.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,374] INFO Deleting segment 29081095 from log X-90. (kafka.log.Log) [2017-11-11 00:51:49,376] INFO Deleting index /mnt/secure/kafka/datalog/X-90/29081095.index.deleted (kafka.log.OffsetIndex) [2017-11-11 00:51:49,376] INFO Deleting index /mnt/secure/kafka/datalog/X-90/.timeindex.deleted (kafka.log.TimeIndex) [2017-11-11 00:51:49,377] INFO Deleting index /mnt/secure/kafka/datalog/X-90/29081095.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 .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
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247899#comment-16247899 ] Ben Corlett commented on KAFKA-6194: I see lots of "Deleting segment %d from log %s." before the crash. The server had been up and operating normally for several hours. I did not try to remove that file it wanted but I did verify that I could create and remove a file as the kafka user in that same directory. Most of the cluster is now back to 0.11.0.1. Been having a lot of issues with servers needing multiple restarts following: [2017-11-10 18:23:47,810] INFO Partition [xxx,97] on broker 110: Cached zkVersion [56] not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition) errors. Which I think you have an issue open already for. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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$Sche
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247621#comment-16247621 ] Ben Corlett commented on KAFKA-6194: No sign of "Failed to close" in the logs > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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 > xxx-83 (kafka.server.ReplicaFetcherThread) > org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an > offline log directory for partition xxx-83 > [2017-11-09 15:32:05,341]
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247338#comment-16247338 ] Ben Corlett commented on KAFKA-6194: So the drives are EBS volumes. They are set up with lvm2 and a logical volume formatted with ext4. They have been set up this way and running for over a year. I think we may end up downgrading back to 0.11.0.1 and the snapshot build of 1.0 (on broker 25) like we were at Tuesday. We'll need to bounce node 25 daily to avoid it running out of heap, but at least the cluster will stay up. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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(ThreadPoolExecut
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247311#comment-16247311 ] Ben Corlett commented on KAFKA-6194: Just had another incident after the chgrp. Which was done while the server was online. {code} [2017-11-10 10:13:59,297] ERROR Error while deleting segments for x-38 in dir /mnt/secure/kafka/datalog (kafka.server.LogDirFailureChannel) java.io.IOException: Delete of log .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-10 10:13:59,300] FATAL [LogDirFailureHandler]: Halting broker because dir /mnt/secure/kafka/datalog is offline (kafka.server.ReplicaManager$LogDirFailureHandler) [2017-11-10 10:13:59,301] ERROR Uncaught exception in scheduled task 'delete-file' (kafka.utils.KafkaScheduler) org.apache.kafka.common.errors.KafkaStorageException: Error while deleting segments for xx-38 in dir /mnt/secure/kafka/datalog Caused by: java.io.IOException: Delete of log .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) {code} > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .log.deleted failed. > at kafka.log.LogSegment.delete(LogSegment.scala:496) > at kafka.log.
[jira] [Comment Edited] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247242#comment-16247242 ] Ben Corlett edited comment on KAFKA-6194 at 11/10/17 9:15 AM: -- So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously directories would be written with: {code} drwxr-xr-x2 kafka root {code} after the change {code} drwxr-xr-x1 kafka kafka {code} Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. was (Author: corlettb): So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: {code} -rw-r--r--2 kafka root {code} after the change {code} -rw-r--r--1 kafka kafka {code} Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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.KafkaStor
[jira] [Comment Edited] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247242#comment-16247242 ] Ben Corlett edited comment on KAFKA-6194 at 11/10/17 9:13 AM: -- So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: {code} -rwxr-xr-x2 kafka root {code} after the change {code} -rw-r--r--1 kafka kafka {code} Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. was (Author: corlettb): So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: -rwxr-xr-x2 kafka root after the change -rw-r--r--1 kafka kafka Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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
[jira] [Comment Edited] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247242#comment-16247242 ] Ben Corlett edited comment on KAFKA-6194 at 11/10/17 9:13 AM: -- So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: {code} -rw-r--r--2 kafka root {code} after the change {code} -rw-r--r--1 kafka kafka {code} Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. was (Author: corlettb): So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: {code} -rwxr-xr-x2 kafka root {code} after the change {code} -rw-r--r--1 kafka kafka {code} Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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.KafkaStorageExc
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16247242#comment-16247242 ] Ben Corlett commented on KAFKA-6194: So we had another 8 incidents overnight. Maybe this is an issue of my own making. Another change was made shortly after upgrading to 1.0 to debug the heap issues (basically getting jmap to run properly). In our monit config this was changed start program = "/opt/kafka/bin/start.sh" as uid kafka with timeout 600 seconds to: start program = "/opt/kafka/bin/start.sh" as uid kafka and gid kafka with timeout 600 seconds Previously files would be written with: -rwxr-xr-x2 kafka root after the change -rw-r--r--1 kafka kafka Now as the uid has full access to all the files and directories I just assumed this wouldn't be an issue. Maybe something is odd about the way files are getting removed. I'm going to recursively chgrp the remaining files and see if we still get the issue. > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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.uti
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16246009#comment-16246009 ] Ben Corlett commented on KAFKA-6194: Quick question. We are currently running 1.0 on: inter.broker.protocol.version=1.0 log.message.format.version=0.11.0.1. Can we still downgrade to 0.11.0.1? By changing back the inter broker protocol and then switching the binaries back to 0.11.0.1 > 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 > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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 > xxx-83 (kafka.server
[jira] [Commented] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16245999#comment-16245999 ] Ben Corlett commented on KAFKA-6194: Checking for this file now it does exist: root@ds-rd-kafkacluster-a-22:/mnt/secure/kafka/datalog/xxx-49# ls -latr .log.deleted -rw-r--r-- 1 kafka kafka 2459 Nov 9 16:44 .log.deleted > 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 > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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 > xxx-83 (kafka.server.ReplicaFetcherT
[jira] [Updated] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Corlett updated KAFKA-6194: --- Description: 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 xx-49 in dir /mnt/secure/kafka/datalog (kafka.server.LogDirFailureChannel) java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog Caused by: java.io.IOException: Delete of log .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 xxx-83 (kafka.server.ReplicaFetcherThread) org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log directory for partition xxx-83 [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxx-89 (kafka.server.ReplicaFetcherThread) org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log directory for partition xxx-89 [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxx-76 (kafka.server.ReplicaFetcherThread) . 2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the replica LEO, the partition xxx-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 x-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} was: 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
[jira] [Updated] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Corlett updated KAFKA-6194: --- Description: 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 xx-49 in dir /mnt/secure/kafka/datalog (kafka.server.LogDirFailureChannel) java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog Caused by: java.io.IOException: Delete of log .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 xxx-83 (kafka.server.ReplicaFetcherThread) org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log directory for partition xxx-83 [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxx-89 (kafka.server.ReplicaFetcherThread) org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an offline log directory for partition xxx-89 [2017-11-09 15:32:05,341] ERROR Error while processing data for partition xxx-76 (kafka.server.ReplicaFetcherThread) . 2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the replica LEO, the partition xxx-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 x-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} was: 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 i
[jira] [Updated] (KAFKA-6194) Server crash while deleting segments
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Corlett updated KAFKA-6194: --- Summary: Server crash while deleting segments (was: Server crash) > 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 > > 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 > xx-49 in dir /mnt/secure/kafka/datalog > (kafka.server.LogDirFailureChannel) > java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog > Caused by: java.io.IOException: Delete of log > .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) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (KAFKA-6194) Server crash
Ben Corlett created KAFKA-6194: -- Summary: Server crash 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 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 xx-49 in dir /mnt/secure/kafka/datalog (kafka.server.LogDirFailureChannel) java.io.IOException: Delete of log .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 xx-49 in dir /mnt/secure/kafka/datalog Caused by: java.io.IOException: Delete of log .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) {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16225159#comment-16225159 ] Ben Corlett commented on KAFKA-6042: The new build was installed 14:10ish on the 23rd. It was restarted mid-day on the 29th. > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0, 0.11.0.2 > > Attachments: heapusage.png, thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.s
[jira] [Comment Edited] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16225144#comment-16225144 ] Ben Corlett edited comment on KAFKA-6042 at 10/30/17 3:33 PM: -- It might be a memory leak. I'll keep an eye on it over the next few days. !heapusage.png! was (Author: corlettb): It might be a memory leak. I'll keep an eye on it over the next few days. !heapusage.png|thumbnail! > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0, 0.11.0.2 > > Attachments: heapusage.png, thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetc
[jira] [Comment Edited] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16225144#comment-16225144 ] Ben Corlett edited comment on KAFKA-6042 at 10/30/17 3:33 PM: -- It might be a memory leak. I'll keep an eye on it over the next few days. !heapusage.jpg|thumbnail! was (Author: corlettb): heapusage > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0, 0.11.0.2 > > Attachments: heapusage.png, thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(Ab
[jira] [Comment Edited] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16225144#comment-16225144 ] Ben Corlett edited comment on KAFKA-6042 at 10/30/17 3:33 PM: -- It might be a memory leak. I'll keep an eye on it over the next few days. !heapusage.png|thumbnail! was (Author: corlettb): It might be a memory leak. I'll keep an eye on it over the next few days. !heapusage.jpg|thumbnail! > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0, 0.11.0.2 > > Attachments: heapusage.png, thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(Ab
[jira] [Updated] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Corlett updated KAFKA-6042: --- Attachment: heapusage.png heapusage > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0, 0.11.0.2 > > Attachments: heapusage.png, thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > As node 125 removed all the ISRs as it was locking up, a failover for any > partition without
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16224617#comment-16224617 ] Ben Corlett commented on KAFKA-6042: We had another issue on broker 125 again. Not the same this time. All other nodes in the cluster are on 0.11.0.1. Broker 125 is running a build from: https://github.com/apache/kafka/commits/2a321941387c7739f2fbbbe592d017b703223ada It ran out of heap space. We are currently running a heap of 5GB. This is the first time we've seen an out of heap issue with kafka. I don't know if this is related to this issue. It might be that 1.0 requires more heap space or that running mixed versions uses more heap. This issue affected the entire cluster and messages rates didn't go back to normal until broker 125 was restarted. I can increase the heap size. {code} [2017-10-28 16:19:31,061] ERROR [KafkaApi-125] Error when handling request {replica_id=-1,max_wait_time=500,min_bytes=1,max_bytes=52428800,topics=[{topic=X,partitions=[{partition=40,fetch_offset=153707886,max_bytes=1048576}]}]} (kafka.server.KafkaApis) java.lang.OutOfMemoryError: Java heap space at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) at org.apache.kafka.common.record.AbstractRecords.downConvert(AbstractRecords.java:101) at org.apache.kafka.common.record.FileRecords.downConvert(FileRecords.java:253) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$4(KafkaApis.scala:520) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$4$adapted(KafkaApis.scala:518) at kafka.server.KafkaApis$$Lambda$837/843104331.apply(Unknown Source) at scala.Option.map(Option.scala:146) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$3(KafkaApis.scala:518) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$3$adapted(KafkaApis.scala:508) at kafka.server.KafkaApis$$Lambda$836/1538921035.apply(Unknown Source) at scala.Option.flatMap(Option.scala:171) at kafka.server.KafkaApis.convertedPartitionData$1(KafkaApis.scala:508) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$12(KafkaApis.scala:556) at kafka.server.KafkaApis$$Lambda$833/1032345356.apply(Unknown Source) at scala.collection.Iterator.foreach(Iterator.scala:929) at scala.collection.Iterator.foreach$(Iterator.scala:929) at scala.collection.AbstractIterator.foreach(Iterator.scala:1417) at scala.collection.IterableLike.foreach(IterableLike.scala:71) at scala.collection.IterableLike.foreach$(IterableLike.scala:70) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at kafka.server.KafkaApis.createResponse$2(KafkaApis.scala:555) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$14(KafkaApis.scala:569) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$14$adapted(KafkaApis.scala:569) at kafka.server.KafkaApis$$Lambda$844/44004770.apply(Unknown Source) at kafka.server.KafkaApis.$anonfun$sendResponseMaybeThrottle$1(KafkaApis.scala:2034) at kafka.server.KafkaApis$$Lambda$439/940799008.apply$mcVI$sp(Unknown Source) at kafka.server.ClientRequestQuotaManager.maybeRecordAndThrottle(ClientRequestQuotaManager.scala:52) at kafka.server.KafkaApis.sendResponseMaybeThrottle(KafkaApis.scala:2034) at kafka.server.KafkaApis.fetchResponseCallback$1(KafkaApis.scala:569) at kafka.server.KafkaApis.$anonfun$handleFetchRequest$15(KafkaApis.scala:588) at kafka.server.KafkaApis$$Lambda$843/1757998472.apply$mcVI$sp(Unknown Source) {code} {code} 2017-10-28T16:19:31.207+0100: 439215.093: [GC pause (G1 Evacuation Pause) (young), 0.0025596 secs] [Parallel Time: 1.3 ms, GC Workers: 4] [GC Worker Start (ms): Min: 439215093.0, Avg: 439215093.0, Max: 439215093.0, Diff: 0.0] [Ext Root Scanning (ms): Min: 0.6, Avg: 0.6, Max: 0.7, Diff: 0.1, Sum: 2.5] [Update RS (ms): Min: 0.0, Avg: 0.4, Max: 0.5, Diff: 0.5, Sum: 1.6] [Processed Buffers: Min: 1, Avg: 1.2, Max: 2, Diff: 1, Sum: 5] [Scan RS (ms): Min: 0.0, Avg: 0.0, Max: 0.0, Diff: 0.0, Sum: 0.0] [Code Root Scanning (ms): Min: 0.0, Avg: 0.0, Max: 0.0, Diff: 0.0, Sum: 0.0] [Object Copy (ms): Min: 0.1, Avg: 0.1, Max: 0.1, Diff: 0.0, Sum: 0.3] [Termination (ms): Min: 0.0, Avg: 0.1, Max: 0.4, Diff: 0.4, Sum: 0.4] [Termination Attempts: Min: 1, Avg: 1.0, Max: 1, Diff: 0, Sum: 4] [GC Worker Other (ms): Min: 0.0, Avg: 0.0, Max: 0.0, Diff: 0.0, Sum: 0.0] [GC Worker Total (ms): Min: 1.2, Avg: 1.2, Max: 1.2, Diff: 0.0, Sum: 4.9] [GC Worker End (ms): Min: 439215094.2, Avg: 439215094.2, Max: 439215094.2, Diff: 0.0] [Code Root Fixup: 0.1 ms] [Code Root Purge: 0.0 ms] [Clear CT: 0.1 ms] [Other: 1.0 ms] [Choose
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16215143#comment-16215143 ] Ben Corlett commented on KAFKA-6042: I've deployed 1.0.0-SNAPSHOT to broker 25. I'll let you know how we get on. > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0 > Environment: kafka version: 0.11.0.1 > 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 >Assignee: Rajini Sivaram >Priority: Blocker > Fix For: 1.0.0 > > Attachments: thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > As node 125 removed all the
[jira] [Commented] (KAFKA-5970) Deadlock due to locking of DelayedProduce and group
[ https://issues.apache.org/jira/browse/KAFKA-5970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16212383#comment-16212383 ] Ben Corlett commented on KAFKA-5970: Unfortunately we've had another incident today on broker 125 after applying the changes from pull request 3956. You can see the changes here https://github.com/corlettb/kafka/commits/deadlock. {noformat} Found one Java-level deadlock: = "executor-Heartbeat": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" "kafka-request-handler-7": waiting to lock monitor 0x7fbe1942f698 (object 0x00068cd2c420, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-4" "kafka-request-handler-4": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" Java stack information for the threads listed above: === "executor-Heartbeat": at kafka.coordinator.group.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:776) - waiting to lock <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:34) at kafka.server.DelayedOperation.run(DelayedOperation.scala:120) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:748) "kafka-request-handler-7": at kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311) - waiting to lock <0x00068cd2c420> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10$adapted(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager$$Lambda$1045/747223912.apply(Unknown Source) at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:124) at kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:68) at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:106) at kafka.server.DelayedOperation.maybeTryComplete(DelayedOperation.scala:107) at kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:347) at kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:253) at kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250) at kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418) at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500) at kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:545) at kafka.server.ReplicaManager$$Lambda$909/475609331.apply(Unknown Source) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234) at scala.collection.TraversableLike$$Lambda$14/1859039536.apply(Unknown Source) at scala.collection.immutable.Map$Map1.foreach(Map.scala:120) at scala.collection.TraversableLike.map(TraversableLike.scala:234) at scala.collection.TraversableLike.map$(TraversableLike.scala:227) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:531) at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:373) at kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:245) at kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:465) - locked <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:429) at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:361) at kafka.server.KafkaApis.handle(KafkaApis.scala:105) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66) at java.lang.Thread.run(Thread.java:748) "kafka-request-handler-4": at kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311)
[jira] [Comment Edited] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16212375#comment-16212375 ] Ben Corlett edited comment on KAFKA-6042 at 10/20/17 8:48 AM: -- Unfortunately we've had another incident today on broker 125. Here is the github commits of the patched build I was using. https://github.com/corlettb/kafka/commits/deadlock. Basically 0.11.0.1 with a cherry pick of the changes in pull request 3956 (for kafka-5970). {noformat} Found one Java-level deadlock: = "executor-Heartbeat": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" "kafka-request-handler-7": waiting to lock monitor 0x7fbe1942f698 (object 0x00068cd2c420, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-4" "kafka-request-handler-4": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" Java stack information for the threads listed above: === "executor-Heartbeat": at kafka.coordinator.group.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:776) - waiting to lock <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:34) at kafka.server.DelayedOperation.run(DelayedOperation.scala:120) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:748) "kafka-request-handler-7": at kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311) - waiting to lock <0x00068cd2c420> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10$adapted(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager$$Lambda$1045/747223912.apply(Unknown Source) at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:124) at kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:68) at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:106) at kafka.server.DelayedOperation.maybeTryComplete(DelayedOperation.scala:107) at kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:347) at kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:253) at kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250) at kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418) at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500) at kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:545) at kafka.server.ReplicaManager$$Lambda$909/475609331.apply(Unknown Source) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234) at scala.collection.TraversableLike$$Lambda$14/1859039536.apply(Unknown Source) at scala.collection.immutable.Map$Map1.foreach(Map.scala:120) at scala.collection.TraversableLike.map(TraversableLike.scala:234) at scala.collection.TraversableLike.map$(TraversableLike.scala:227) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:531) at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:373) at kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:245) at kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:465) - locked <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:429) at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:361) at kafka.server.KafkaApis.handle(KafkaApis.scala:105) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66) at java.lang.Thread.run(Thread.java:748) "kafka-request-hand
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16212375#comment-16212375 ] Ben Corlett commented on KAFKA-6042: Unfortunately we've had another incident today on broker 125. Here is the github commits of the patched build I was using. https://github.com/corlettb/kafka/commits/deadlock. Basically 0.11.0.1 with a cherry pick of the changes in pull request 3956 (for kafka-5970). Found one Java-level deadlock: = "executor-Heartbeat": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" "kafka-request-handler-7": waiting to lock monitor 0x7fbe1942f698 (object 0x00068cd2c420, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-4" "kafka-request-handler-4": waiting to lock monitor 0x7fbd8c1834c8 (object 0x00068cccb590, a kafka.coordinator.group.GroupMetadata), which is held by "kafka-request-handler-7" Java stack information for the threads listed above: === "executor-Heartbeat": at kafka.coordinator.group.GroupCoordinator.onExpireHeartbeat(GroupCoordinator.scala:776) - waiting to lock <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.DelayedHeartbeat.onExpiration(DelayedHeartbeat.scala:34) at kafka.server.DelayedOperation.run(DelayedOperation.scala:120) 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:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:748) "kafka-request-handler-7": at kafka.coordinator.group.GroupMetadataManager.putCacheCallback$2(GroupMetadataManager.scala:311) - waiting to lock <0x00068cd2c420> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager.$anonfun$storeOffsets$10$adapted(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupMetadataManager$$Lambda$1045/747223912.apply(Unknown Source) at kafka.server.DelayedProduce.onComplete(DelayedProduce.scala:124) at kafka.server.DelayedOperation.forceComplete(DelayedOperation.scala:68) at kafka.server.DelayedProduce.tryComplete(DelayedProduce.scala:106) at kafka.server.DelayedOperation.maybeTryComplete(DelayedOperation.scala:107) at kafka.server.DelayedOperationPurgatory$Watchers.tryCompleteWatched(DelayedOperation.scala:347) at kafka.server.DelayedOperationPurgatory.checkAndComplete(DelayedOperation.scala:253) at kafka.server.ReplicaManager.tryCompleteDelayedProduce(ReplicaManager.scala:250) at kafka.cluster.Partition.tryCompleteDelayedRequests(Partition.scala:418) at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:500) at kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(ReplicaManager.scala:545) at kafka.server.ReplicaManager$$Lambda$909/475609331.apply(Unknown Source) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234) at scala.collection.TraversableLike$$Lambda$14/1859039536.apply(Unknown Source) at scala.collection.immutable.Map$Map1.foreach(Map.scala:120) at scala.collection.TraversableLike.map(TraversableLike.scala:234) at scala.collection.TraversableLike.map$(TraversableLike.scala:227) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:531) at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:373) at kafka.coordinator.group.GroupMetadataManager.appendForGroup(GroupMetadataManager.scala:245) at kafka.coordinator.group.GroupMetadataManager.storeOffsets(GroupMetadataManager.scala:380) at kafka.coordinator.group.GroupCoordinator.doCommitOffsets(GroupCoordinator.scala:465) - locked <0x00068cccb590> (a kafka.coordinator.group.GroupMetadata) at kafka.coordinator.group.GroupCoordinator.handleCommitOffsets(GroupCoordinator.scala:429) at kafka.server.KafkaApis.handleOffsetCommitRequest(KafkaApis.scala:361) at kafka.server.KafkaApis.handle(KafkaApis.scala:105) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66) at java.lang.Thread.run(Thread.java:748) "kafka-request-handler-4": at kafka.coordinator.group.GroupMetadataMana
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16209195#comment-16209195 ] Ben Corlett commented on KAFKA-6042: So far after a week. No issues. > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1 > Environment: kafka version: 0.11.0.1 > 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 >Priority: Critical > Attachments: thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > As node 125 removed all the ISRs as it was locking up, a failover for any > partition without an unclean leader election is not possible. This br
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16198640#comment-16198640 ] Ben Corlett commented on KAFKA-6042: I've deployed a build of 0.11.0.1 with the commits of pull request 3956 to node 125. Will let you know how I get on. > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1 > Environment: kafka version: 0.11.0.1 > 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 >Priority: Critical > Attachments: thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) > As node 125 removed all the ISRs as it was locking up, a fai
[jira] [Commented] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
[ https://issues.apache.org/jira/browse/KAFKA-6042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16198483#comment-16198483 ] Ben Corlett commented on KAFKA-6042: The second stack trace in that issue looks very familiar. You may well be right. I should have searched a bit harder. Do you know when 0.11.0.2 will be released. Wondering if I should create a build with that pull request? > Kafka Request Handler deadlocks and brings down the cluster. > > > Key: KAFKA-6042 > URL: https://issues.apache.org/jira/browse/KAFKA-6042 > Project: Kafka > Issue Type: Bug >Affects Versions: 0.11.0.0, 0.11.0.1 > Environment: kafka version: 0.11.0.1 > 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 >Priority: Critical > Attachments: thread_dump.txt.gz > > > We have been experiencing a deadlock that happens on a consistent server > within our cluster. This happens multiple times a week currently. It first > started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to > resolve the issue. > Sequence of events: > At a seemingly random time broker 125 goes into a deadlock. As soon as it is > deadlocked it will remove all the ISR's for any partition is its the leader > for. > [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: > Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) > > The other nodes fail to connect to the node 125 > [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch > to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, > minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, > logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, > logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, > logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, > logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, > logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, > logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) > java.io.IOException: Connection to 125 was disconnected before the response > was read > at > org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) > at > kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) > at > kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) > at kafka.utils.Shutdo
[jira] [Created] (KAFKA-6042) Kafka Request Handler deadlocks and brings down the cluster.
Ben Corlett created KAFKA-6042: -- Summary: Kafka Request Handler deadlocks and brings down the cluster. Key: KAFKA-6042 URL: https://issues.apache.org/jira/browse/KAFKA-6042 Project: Kafka Issue Type: Bug Affects Versions: 0.11.0.0, 0.11.0.1 Environment: kafka version: 0.11.0.1 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 Priority: Critical Attachments: thread_dump.txt.gz We have been experiencing a deadlock that happens on a consistent server within our cluster. This happens multiple times a week currently. It first started happening when we upgraded to 0.11.0.0. Sadly 0.11.0.1 failed to resolve the issue. Sequence of events: At a seemingly random time broker 125 goes into a deadlock. As soon as it is deadlocked it will remove all the ISR's for any partition is its the leader for. [2017-10-10 00:06:10,061] INFO Partition [XX,24] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,073] INFO Partition [XX,974] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,079] INFO Partition [XX,64] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,081] INFO Partition [XX,21] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,084] INFO Partition [XX,12] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,085] INFO Partition [XX,61] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,086] INFO Partition [XX,53] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,088] INFO Partition [XX,27] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,090] INFO Partition [XX,182] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) [2017-10-10 00:06:10,091] INFO Partition [XX,16] on broker 125: Shrinking ISR from 117,125 to 125 (kafka.cluster.Partition) The other nodes fail to connect to the node 125 [2017-10-10 00:08:42,318] WARN [ReplicaFetcherThread-0-125]: Error in fetch to broker 125, request (type=FetchRequest, replicaId=101, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={XX-94=(offset=0, logStartOffset=0, maxBytes=1048576), XX-22=(offset=0, logStartOffset=0, maxBytes=1048576), XX-58=(offset=0, logStartOffset=0, maxBytes=1048576), XX-11=(offset=78932482, logStartOffset=50881481, maxBytes=1048576), XX-55=(offset=0, logStartOffset=0, maxBytes=1048576), XX-19=(offset=0, logStartOffset=0, maxBytes=1048576), XX-91=(offset=0, logStartOffset=0, maxBytes=1048576), XX-5=(offset=903857106, logStartOffset=0, maxBytes=1048576), XX-80=(offset=0, logStartOffset=0, maxBytes=1048576), XX-88=(offset=0, logStartOffset=0, maxBytes=1048576), XX-34=(offset=308, logStartOffset=308, maxBytes=1048576), XX-7=(offset=369990, logStartOffset=369990, maxBytes=1048576), XX-0=(offset=57965795, logStartOffset=0, maxBytes=1048576)}) (kafka.server.ReplicaFetcherThread) java.io.IOException: Connection to 125 was disconnected before the response was read at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:93) at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:93) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:207) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:151) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:112) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64) As node 125 removed all the ISRs as it was locking up, a failover for any partition without an unclean leader election is not possible. This breaks any partition that this node was leader for. As we spread all topics across all servers this essentials brings down the entire cluster. Recovery: Unforuntately with broker 125 in its deadlocked state a clean shutdown does not work. A kill -9 is necessary. After an unclean shutdown the indexes must be rebuilt and start up time is around 10 minutes. After node 125 finally starts the cluster reco