[ 
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/00000000000029439517.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/00000000000029440823.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:33,776] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029440823.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/00000000000029442113.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:33,896] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029442113.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/00000000000029443416.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,045] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029443416.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/00000000000000000000.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/00000000000029442113.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,180] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029440823.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/00000000000029440823.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029439517.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029439517.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/00000000000029437182.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029437182.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,179] INFO Deleting segment 29435892 from log topic1-84. 
(kafka.log.Log)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029443416.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029435892.index.deleted 
(kafka.log.OffsetIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029435892.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,181] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029443416.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,180] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000029442113.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,180] INFO Deleting index 
/mnt/secure/kafka/datalog/topic1-84/00000000000000000000.timeindex.deleted 
(kafka.log.TimeIndex)
[2017-11-10 20:36:34,182] ERROR Uncaught exception in scheduled task 
'delete-file' (kafka.utils.KafkaScheduler)
kafka.common.KafkaStorageException: Delete of log 
00000000000000000000.log.deleted failed.
        at kafka.log.LogSegment.delete(LogSegment.scala:505)
        at kafka.log.Log.deleteSeg$1(Log.scala:1490)
        at kafka.log.Log.$anonfun$asyncDeleteSegment$2(Log.scala:1492)
        at 
kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
        at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:57)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:748)
[2017-11-10 20:36:34,288] INFO Deleting segment 29444711 from log topic1-84. 
(kafka.log.Log)
{code}

Note that topic1 is that same topic as the other incident. 

Again two "Deleting segment 29443416 from log topic1-84".

Sadly this topic has been deleted since then. There is a new topic with a 
similar name with the following settings:

min.cleanable.dirty.ratio       0.05
cleanup.policy  compact
retention.ms    1000
delete.retention.ms     600000
segment.bytes   5000000

This new topic is currently running at around 90,000 messages per sec.




> Server crash while deleting segments
> ------------------------------------
>
>                 Key: KAFKA-6194
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6194
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 1.0.0
>         Environment: kafka version: 1.0
> client versions: 0.8.2.1-0.10.2.1
> platform: aws (eu-west-1a)
> nodes: 36 x r4.xlarge
> disk storage: 2.5 tb per node (~73% usage per node)
> topics: 250
> number of partitions: 48k (approx)
> os: ubuntu 14.04
> jvm: Java(TM) SE Runtime Environment (build 1.8.0_131-b11), Java HotSpot(TM) 
> 64-Bit Server VM (build 25.131-b11, mixed mode)
>            Reporter: Ben Corlett
>              Labels: regression
>
> We upgraded our R+D cluster to 1.0 in the hope that it would fix the deadlock 
> from 0.11.0.*. Sadly our cluster has the memory leak issues with 1.0 most 
> likely from https://issues.apache.org/jira/browse/KAFKA-6185. We are running 
> one server on a patched version of 1.0 with the pull request from that.
> However today we have had two different servers fall over for non-heap 
> related reasons. The exceptions in the kafka log are :
> {code}
> [2017-11-09 15:32:04,037] ERROR Error while deleting segments for 
> xxxxxxxxxx-49 in dir /mnt/secure/kafka/datalog 
> (kafka.server.LogDirFailureChannel)
> java.io.IOException: Delete of log 00000000000000000000.log.deleted failed.
>         at kafka.log.LogSegment.delete(LogSegment.scala:496)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596)
>         at 
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
>         at kafka.log.Log.deleteSeg$1(Log.scala:1596)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599)
>         at 
> kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:748)
> [2017-11-09 15:32:04,040] INFO [ReplicaManager broker=122] Stopping serving 
> replicas in dir /mnt/secure/kafka/datalog (kafka.server.ReplicaManager)
> [2017-11-09 15:32:04,041] ERROR Uncaught exception in scheduled task 
> 'delete-file' (kafka.utils.KafkaScheduler)
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for xxxxxxxxxxxxxx-49 in dir /mnt/secure/kafka/datalog
> Caused by: java.io.IOException: Delete of log 
> 00000000000000000000.log.deleted failed.
>         at kafka.log.LogSegment.delete(LogSegment.scala:496)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$3(Log.scala:1596)
>         at 
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:12)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
>         at kafka.log.Log.deleteSeg$1(Log.scala:1596)
>         at kafka.log.Log.$anonfun$asyncDeleteSegment$4(Log.scala:1599)
>         at 
> kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
>         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:748)
> .....
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition 
> xxxxxxx-83 (kafka.server.ReplicaFetcherThread)
> org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an 
> offline log directory for partition xxxxxxx-83
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition 
> xxxxxxx-89 (kafka.server.ReplicaFetcherThread)
> org.apache.kafka.common.errors.KafkaStorageException: Replica 122 is in an 
> offline log directory for partition xxxxxxx-89
> [2017-11-09 15:32:05,341] ERROR Error while processing data for partition 
> xxxxxxx-76 (kafka.server.ReplicaFetcherThread)
> .....
> 2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording the 
> replica LEO, the partition xxxxxxx-27 hasn't been created. 
> (kafka.server.ReplicaManager)
> [2017-11-09 15:32:05,613] WARN [ReplicaManager broker=122] While recording 
> the replica LEO, the partition xxxxxxxxx-79 hasn't been created. 
> (kafka.server.ReplicaManager)
> [2017-11-09 15:32:05,622] FATAL Shutdown broker because all log dirs in 
> /mnt/secure/kafka/datalog have failed (kafka.log.LogManager)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to