kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Dong Lin (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-6624) log segment deletion could cause a disk to be marked offline incorrectly
Date Thu, 08 Mar 2018 04:54:00 GMT

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

Dong Lin commented on KAFKA-6624:
---------------------------------

[~junrao] It may be reasonable to grab the lock of the log during log.flush(). This will be
consistent with the idea that all write/read operation of log segments will be synchronized
using the per-log lock.

It will incur slightly higher overhead but personally I think it is OK. We will only flush
the log segments since the previous flushed offset (i.e. recovery offset). Alternatively we
can use the flag to optimize it. Not sure if the performance improvement with that optimization
is worth the extra complexity.

Can you review [https://github.com/apache/kafka/pull/4663/files?|https://github.com/apache/kafka/pull/4663/files]

 

> log segment deletion could cause a disk to be marked offline incorrectly
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-6624
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6624
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 1.1.0
>            Reporter: Jun Rao
>            Priority: Major
>
> Saw the following log.
> [2018-03-06 23:12:20,721] ERROR Error while flushing log for topic1-0 in dir /data01/kafka-logs
with offset 80993 (kafka.server.LogDirFailureChannel)
> java.nio.channels.ClosedChannelException
>         at sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:110)
>         at sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:379)
>         at org.apache.kafka.common.record.FileRecords.flush(FileRecords.java:163)
>         at kafka.log.LogSegment$$anonfun$flush$1.apply$mcV$sp(LogSegment.scala:375)
>         at kafka.log.LogSegment$$anonfun$flush$1.apply(LogSegment.scala:374)
>         at kafka.log.LogSegment$$anonfun$flush$1.apply(LogSegment.scala:374)
>         at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
>         at kafka.log.LogSegment.flush(LogSegment.scala:374)
>         at kafka.log.Log$$anonfun$flush$1$$anonfun$apply$mcV$sp$4.apply(Log.scala:1374)
>         at kafka.log.Log$$anonfun$flush$1$$anonfun$apply$mcV$sp$4.apply(Log.scala:1373)
>         at scala.collection.Iterator$class.foreach(Iterator.scala:891)
>         at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
>         at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>         at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>         at kafka.log.Log$$anonfun$flush$1.apply$mcV$sp(Log.scala:1373)
>         at kafka.log.Log$$anonfun$flush$1.apply(Log.scala:1368)
>         at kafka.log.Log$$anonfun$flush$1.apply(Log.scala:1368)
>         at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
>         at kafka.log.Log.flush(Log.scala:1368)
>         at kafka.log.Log$$anonfun$roll$2$$anonfun$apply$1.apply$mcV$sp(Log.scala:1343)
>         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)
> [2018-03-06 23:12:20,722] INFO [ReplicaManager broker=0] Stopping serving replicas in
dir /data01/kafka-logs (kafka.server.ReplicaManager)
> It seems that topic1 was being deleted around the time when flushing was called. Then
flushing hit an IOException, which caused the disk to be marked offline incorrectly.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message