[jira] [Commented] (KAFKA-8604) kafka log dir was marked as offline because of deleting segments of __consumer_offsets failed

2019-07-13 Thread songyingshuan (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16884547#comment-16884547
 ] 

songyingshuan commented on KAFKA-8604:
--

I think we have got the reason : in our kafka cluster, there is a topic with 
high throughout. 
And the consumer commit its offset everytime it is going to poll() more 
messages. 
This consumer group commit about 7.5 million records in about 5 minutes, 
which cause the __consuemr_group-38 partition rolls a new segment file about 
every 20 seconds.
Whenever there is a new segment created, the log cleaner thread tried to clean 
this partition (we config 'log.cleaner.threads
=11', and there is only a few topics are 'compact' type).
At the end of cleaning process, a asyncDeleteSegment task will be sheduled 
(default : 60s later), if the next two tasks have the same file to delete, the 
latter will fail。

Based on the analysis result,we first modified the consumer's code,auto commit 
was used ant the interval was fix to 3 seconds.
And the LogDirFailuer have not appeared yet.

So, we think the asyncDeleteSegment should be asyncDeleteSegment or decrease 
the default interval of asyn delete operation.


> kafka log dir was marked as offline because of deleting segments of 
> __consumer_offsets failed
> -
>
> Key: KAFKA-8604
> URL: https://issues.apache.org/jira/browse/KAFKA-8604
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 1.0.1
>Reporter: songyingshuan
>Priority: Major
> Attachments: error-logs.log
>
>
> We encountered a problem in our product env without any foresight. When kafka 
> broker trying to clean __consumer_offsets-38 (and only happents to this 
> partition), the log shows
> it failed, and marking the whole disk/log dir offline, and this leads to a 
> negative impact on some normal partitions (because of the ISR list of those 
> partitions decrease).
> we had to restart the broker server to reuse the disk/dir which was marked as 
> offline. BUT!! this problem occurs periodically with the same reason so we 
> have to restart broker periodically.
> we read some source code of kafka-1.0.1, but cannot make sure why this 
> happends. And The cluster status had been good until this problem suddenly 
> attacked us.
> the error log is something like this :
>  
> {code:java}
> 2019-06-25 00:11:26,241 INFO kafka.log.TimeIndex: Deleting index 
> /data6/kafka/data/__consumer_offsets-38/012855596978.timeindex.deleted
> 2019-06-25 00:11:26,258 ERROR kafka.server.LogDirFailureChannel: Error while 
> deleting segments for __consumer_offsets-38 in dir /data6/kafka/data
> java.io.IOException: Delete of log .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,265 ERROR kafka.utils.KafkaScheduler: Uncaught exception 
> in scheduled task 'delete-file'
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for __consumer_offsets-38 in dir /data6/kafka/data
> Caused by: java.io.IOException: Delete of log 
> .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> 

[jira] [Commented] (KAFKA-8604) kafka log dir was marked as offline because of deleting segments of __consumer_offsets failed

2019-07-13 Thread songyingshuan (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16884548#comment-16884548
 ] 

songyingshuan commented on KAFKA-8604:
--

It is worth mentioning that we have another kafka cluster specifically used to 
perform kafka-streaming/sql tasks. the same problem appeared in that cluster. 
And looking back now, it is most likely because the streaming/ksql tasks 
consume a topic with high throughput and update with high frequency.

How do u think?
[~junrao]
[~huxi_2b]

> kafka log dir was marked as offline because of deleting segments of 
> __consumer_offsets failed
> -
>
> Key: KAFKA-8604
> URL: https://issues.apache.org/jira/browse/KAFKA-8604
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 1.0.1
>Reporter: songyingshuan
>Priority: Major
> Attachments: error-logs.log
>
>
> We encountered a problem in our product env without any foresight. When kafka 
> broker trying to clean __consumer_offsets-38 (and only happents to this 
> partition), the log shows
> it failed, and marking the whole disk/log dir offline, and this leads to a 
> negative impact on some normal partitions (because of the ISR list of those 
> partitions decrease).
> we had to restart the broker server to reuse the disk/dir which was marked as 
> offline. BUT!! this problem occurs periodically with the same reason so we 
> have to restart broker periodically.
> we read some source code of kafka-1.0.1, but cannot make sure why this 
> happends. And The cluster status had been good until this problem suddenly 
> attacked us.
> the error log is something like this :
>  
> {code:java}
> 2019-06-25 00:11:26,241 INFO kafka.log.TimeIndex: Deleting index 
> /data6/kafka/data/__consumer_offsets-38/012855596978.timeindex.deleted
> 2019-06-25 00:11:26,258 ERROR kafka.server.LogDirFailureChannel: Error while 
> deleting segments for __consumer_offsets-38 in dir /data6/kafka/data
> java.io.IOException: Delete of log .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,265 ERROR kafka.utils.KafkaScheduler: Uncaught exception 
> in scheduled task 'delete-file'
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for __consumer_offsets-38 in dir /data6/kafka/data
> Caused by: java.io.IOException: Delete of log 
> .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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.

[jira] [Commented] (KAFKA-8604) kafka log dir was marked as offline because of deleting segments of __consumer_offsets failed

2019-07-15 Thread Alex Bertzeletos (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16885316#comment-16885316
 ] 

Alex Bertzeletos commented on KAFKA-8604:
-

[~ymxz] So, in your case, setting the log.retention.check.interval.ms=3000 
helped as a workaround?

> kafka log dir was marked as offline because of deleting segments of 
> __consumer_offsets failed
> -
>
> Key: KAFKA-8604
> URL: https://issues.apache.org/jira/browse/KAFKA-8604
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 1.0.1
>Reporter: songyingshuan
>Priority: Major
> Attachments: error-logs.log
>
>
> We encountered a problem in our product env without any foresight. When kafka 
> broker trying to clean __consumer_offsets-38 (and only happents to this 
> partition), the log shows
> it failed, and marking the whole disk/log dir offline, and this leads to a 
> negative impact on some normal partitions (because of the ISR list of those 
> partitions decrease).
> we had to restart the broker server to reuse the disk/dir which was marked as 
> offline. BUT!! this problem occurs periodically with the same reason so we 
> have to restart broker periodically.
> we read some source code of kafka-1.0.1, but cannot make sure why this 
> happends. And The cluster status had been good until this problem suddenly 
> attacked us.
> the error log is something like this :
>  
> {code:java}
> 2019-06-25 00:11:26,241 INFO kafka.log.TimeIndex: Deleting index 
> /data6/kafka/data/__consumer_offsets-38/012855596978.timeindex.deleted
> 2019-06-25 00:11:26,258 ERROR kafka.server.LogDirFailureChannel: Error while 
> deleting segments for __consumer_offsets-38 in dir /data6/kafka/data
> java.io.IOException: Delete of log .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,265 ERROR kafka.utils.KafkaScheduler: Uncaught exception 
> in scheduled task 'delete-file'
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for __consumer_offsets-38 in dir /data6/kafka/data
> Caused by: java.io.IOException: Delete of log 
> .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,268 INFO kafka.server.ReplicaManager: [Re

[jira] [Commented] (KAFKA-8604) kafka log dir was marked as offline because of deleting segments of __consumer_offsets failed

2019-08-02 Thread songyingshuan (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16899374#comment-16899374
 ] 

songyingshuan commented on KAFKA-8604:
--

[~abertzeletos] Thx for your reply and sorry for not replying for such a long 
time, i'm so busy recently.

we didn't alter the config 'log.retention.check.interval.ms', and it remains as 
the default value which fixed to 3000_000 ms.

we just slowed down the frequency of commits of consumer group's offsets 
record. and the LogDirFailuer has never occured again.

the meaning of  'log.retention.check.interval.ms' is described like this :  The 
frequency in milliseconds that the log cleaner checks whether any log is 
eligible for deletion.  I think this cannot provide any help because this is 
just the interval of judging is there is any '.delete' file to delete, but the 
root cause of this problem i think is the async delete strategy which is the 
previous step of truely file deletion。

> kafka log dir was marked as offline because of deleting segments of 
> __consumer_offsets failed
> -
>
> Key: KAFKA-8604
> URL: https://issues.apache.org/jira/browse/KAFKA-8604
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 1.0.1
>Reporter: songyingshuan
>Priority: Major
> Attachments: error-logs.log
>
>
> We encountered a problem in our product env without any foresight. When kafka 
> broker trying to clean __consumer_offsets-38 (and only happents to this 
> partition), the log shows
> it failed, and marking the whole disk/log dir offline, and this leads to a 
> negative impact on some normal partitions (because of the ISR list of those 
> partitions decrease).
> we had to restart the broker server to reuse the disk/dir which was marked as 
> offline. BUT!! this problem occurs periodically with the same reason so we 
> have to restart broker periodically.
> we read some source code of kafka-1.0.1, but cannot make sure why this 
> happends. And The cluster status had been good until this problem suddenly 
> attacked us.
> the error log is something like this :
>  
> {code:java}
> 2019-06-25 00:11:26,241 INFO kafka.log.TimeIndex: Deleting index 
> /data6/kafka/data/__consumer_offsets-38/012855596978.timeindex.deleted
> 2019-06-25 00:11:26,258 ERROR kafka.server.LogDirFailureChannel: Error while 
> deleting segments for __consumer_offsets-38 in dir /data6/kafka/data
> java.io.IOException: Delete of log .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,265 ERROR kafka.utils.KafkaScheduler: Uncaught exception 
> in scheduled task 'delete-file'
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for __consumer_offsets-38 in dir /data6/kafka/data
> Caused by: java.io.IOException: Delete of log 
> .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
> at kafka.utils.CoreUtils$$anon$1.

[jira] [Commented] (KAFKA-8604) kafka log dir was marked as offline because of deleting segments of __consumer_offsets failed

2019-06-26 Thread songyingshuan (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-8604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16873734#comment-16873734
 ] 

songyingshuan commented on KAFKA-8604:
--

The exceptions was thrown from here:
 
{code:scala}
  def delete() {
val deletedLog = log.delete()
val deletedIndex = index.delete()
val deletedTimeIndex = timeIndex.delete()
val deletedTxnIndex = txnIndex.delete()
if (!deletedLog && log.file.exists) // from here
  throw new IOException("Delete of log " + log.file.getName + " failed.")
if (!deletedIndex && index.file.exists)
  throw new IOException("Delete of index " + index.file.getName + " 
failed.")
if (!deletedTimeIndex && timeIndex.file.exists)
  throw new IOException("Delete of time index " + timeIndex.file.getName + 
" failed.")
if (!deletedTxnIndex && txnIndex.file.exists)
  throw new IOException("Delete of transaction index " + 
txnIndex.file.getName + " failed.")
  }
{code}

it means deletedLog == false && log.file.exists == true,that is the log file to 
be deleted failed to delete and the file is still in the disk.
but we have checked the partition file dir and the file did not exist. So 
confused !!


> kafka log dir was marked as offline because of deleting segments of 
> __consumer_offsets failed
> -
>
> Key: KAFKA-8604
> URL: https://issues.apache.org/jira/browse/KAFKA-8604
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 1.0.1
>Reporter: songyingshuan
>Priority: Major
> Attachments: error-logs.log
>
>
> We encountered a problem in our product env without any foresight. When kafka 
> broker trying to clean __consumer_offsets-38 (and only happents to this 
> partition), the log shows
> it failed, and marking the whole disk/log dir offline, and this leads to a 
> negative impact on some normal partitions (because of the ISR list of those 
> partitions decrease).
> we had to restart the broker server to reuse the disk/dir which was marked as 
> offline. BUT!! this problem occurs periodically with the same reason so we 
> have to restart broker periodically.
> we read some source code of kafka-1.0.1, but cannot make sure why this 
> happends. And The cluster status had been good until this problem suddenly 
> attacked us.
> the error log is something like this :
>  
> {code:java}
> 2019-06-25 00:11:26,241 INFO kafka.log.TimeIndex: Deleting index 
> /data6/kafka/data/__consumer_offsets-38/012855596978.timeindex.deleted
> 2019-06-25 00:11:26,258 ERROR kafka.server.LogDirFailureChannel: Error while 
> deleting segments for __consumer_offsets-38 in dir /data6/kafka/data
> java.io.IOException: Delete of log .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log.scala:1595)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$asyncDeleteSegment$1.apply$mcV$sp(Log.scala:1599)
> at 
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(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:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2019-06-25 00:11:26,265 ERROR kafka.utils.KafkaScheduler: Uncaught exception 
> in scheduled task 'delete-file'
> org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
> segments for __consumer_offsets-38 in dir /data6/kafka/data
> Caused by: java.io.IOException: Delete of log 
> .log.deleted failed.
> at kafka.log.LogSegment.delete(LogSegment.scala:496)
> at 
> kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply$mcV$sp(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log$$anonfun$kafka$log$Log$$deleteSeg$1$1.apply(Log.scala:1596)
> at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
> at kafka.log.Log.kafka$log$Log$$deleteSeg$1(Log