[jira] [Updated] (KAFKA-14096) Race Condition in Log Rolling Leading to Disk Failure

2022-07-21 Thread Eric Azama (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14096?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eric Azama updated KAFKA-14096:
---
Description: 
We've recently encountered what appears to be a race condition that can lead to 
disk being marked offline. One of our brokers recently crashed because its log 
directory failed. We found the following in the server.log file
{code:java}
[2022-07-18 18:24:42,940] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Rolled new log segment at offset 141946850 in 37 ms. 
(kafka.log.Log)
[...]
[2022-07-18 18:24:47,782] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Scheduling segments for deletion 
List(LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678)) (kafka.log.Log)
[2022-07-18 18:24:48,024] ERROR Error while flushing log for TOPIC-REDACTED-15 
in dir /data1/kafka-logs with offset 141935201 
(kafka.server.LogDirFailureChannel)
java.nio.channels.ClosedChannelException
at java.base/sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:150)
at java.base/sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:452)
at org.apache.kafka.common.record.FileRecords.flush(FileRecords.java:176)
at kafka.log.LogSegment.$anonfun$flush$1(LogSegment.scala:472)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at kafka.log.LogSegment.flush(LogSegment.scala:471)
at kafka.log.Log.$anonfun$flush$4(Log.scala:1956)
at kafka.log.Log.$anonfun$flush$4$adapted(Log.scala:1955)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
at kafka.log.Log.$anonfun$flush$2(Log.scala:1955)
at kafka.log.Log.flush(Log.scala:2322)
at kafka.log.Log.$anonfun$roll$9(Log.scala:1925)
at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
[2022-07-18 18:24:48,036] ERROR Uncaught exception in scheduled task 
'flush-log' (kafka.utils.KafkaScheduler)
org.apache.kafka.common.errors.KafkaStorageException: Error while flushing log 
for TOPIC-REDACTED-15 in dir /data1/kafka-logs with offset 141935201{code}
and the following in the log-cleaner.log file
{code:java}
[2022-07-18 18:24:47,062] INFO Cleaner 0: Cleaning 
LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678) in log 
TOPIC-REDACTED-15 into 141935201 with deletion horizon 1658082163480, retaining 
deletes. (kafka.log.LogCleaner) {code}
The timing of the log-cleaner log shows that the log flush failed because the 
log segment had been cleaned and the underlying file was already renamed or 
deleted.

The stacktrace indicates that the log flush that triggered the exception was 
part of the process of rolling a new log segment. (at 
kafka.log.Log.$anonfun$roll$9([Log.scala:1925|https://github.com/apache/kafka/blob/2.5.1/core/src/main/scala/kafka/log/Log.scala#L1925]))]
 This is somewhat concerning because this flush should have been scheduled with 
no delay, but the exception occurred about 6 second after the most recent roll, 
and it wasn't even the most recent file. 

Our best guess is that the broker's Scheduler was overloaded to a point that 
even 0-delay tasks were backed up in the queue, but we're not aware of any 
metrics that would allow us to monitor scheduler health.

While we encountered this on a somewhat old (2.5.1) version of the Broker, 
there don't seem to be any changes in trunk that would protect against this 
kind of delay

  was:
We've recently encountered what appears to be a race condition that can lead to 
disk being marked offline. One of our brokers recently crashed because its log 
directory failed. We found the following in the server.log file
{code:java}
[2022-07-18 18:24:42,940] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Rolled new log segment at offset 141946850 in 37 ms. 
(kafka.log.Log)
[...]
[2022-07-18 18:24:47,782] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Scheduling segments for deletion 
List(LogSegment(baseOffset=141935201, size=1073560219, 

[jira] [Updated] (KAFKA-14096) Race Condition in Log Rolling Leading to Disk Failure

2022-07-21 Thread Eric Azama (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14096?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Eric Azama updated KAFKA-14096:
---
Description: 
We've recently encountered what appears to be a race condition that can lead to 
disk being marked offline. One of our brokers recently crashed because its log 
directory failed. We found the following in the server.log file
{code:java}
[2022-07-18 18:24:42,940] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Rolled new log segment at offset 141946850 in 37 ms. 
(kafka.log.Log)
[...]
[2022-07-18 18:24:47,782] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Scheduling segments for deletion 
List(LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678)) (kafka.log.Log)
[2022-07-18 18:24:48,024] ERROR Error while flushing log for TOPIC-REDACTED-15 
in dir /data1/kafka-logs with offset 141935201 
(kafka.server.LogDirFailureChannel)
java.nio.channels.ClosedChannelException
at java.base/sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:150)
at java.base/sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:452)
at org.apache.kafka.common.record.FileRecords.flush(FileRecords.java:176)
at kafka.log.LogSegment.$anonfun$flush$1(LogSegment.scala:472)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at kafka.log.LogSegment.flush(LogSegment.scala:471)
at kafka.log.Log.$anonfun$flush$4(Log.scala:1956)
at kafka.log.Log.$anonfun$flush$4$adapted(Log.scala:1955)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
at kafka.log.Log.$anonfun$flush$2(Log.scala:1955)
at kafka.log.Log.flush(Log.scala:2322)
at kafka.log.Log.$anonfun$roll$9(Log.scala:1925)
at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
[2022-07-18 18:24:48,036] ERROR Uncaught exception in scheduled task 
'flush-log' (kafka.utils.KafkaScheduler)
org.apache.kafka.common.errors.KafkaStorageException: Error while flushing log 
for TOPIC-REDACTED-15 in dir /data1/kafka-logs with offset 141935201{code}
and the following in the log-cleaner.log file
{code:java}
[2022-07-18 18:24:47,062] INFO Cleaner 0: Cleaning 
LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678) in log 
TOPIC-REDACTED-15 into 141935201 with deletion horizon 1658082163480, retaining 
deletes. (kafka.log.LogCleaner) {code}
The timing of the log-cleaner log shows that the log flush failed because the 
log segment had been cleaned and the underlying file was already renamed or 
deleted.

The stacktrace indicates that the log flush that triggered the exception was 
part of the process of rolling a new log segment. (at 
kafka.log.Log.$anonfun$roll$9([Log.scala:1925|#L1925]))] This is somewhat 
concerning because this flush should have been scheduled with no delay, but the 
exception occurred about 6 second after the most recent roll, and it wasn't 
even the most recent file. 

Our best guess is that the broker's Scheduler was overloaded to a point that 
even 0-delay tasks were backed up in the queue, but we're not aware of any 
metrics that would allow us to monitor scheduler health.

While we encountered this on a somewhat old (2.5.1) version of the Broker, 
there don't seem to be any changes in trunk that would protect against this 
kind of delay

  was:
We've recently encountered what appears to be a race condition that can lead to 
disk being marked offline. One of our brokers recently crashed because its log 
directory failed. We found the following in the server.log file
{code:java}
[2022-07-18 18:24:42,940] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Rolled new log segment at offset 141946850 in 37 ms. 
(kafka.log.Log)
[...]
[2022-07-18 18:24:47,782] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Scheduling segments for deletion 
List(LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678)) (kafka.log.Log)
[2022-07-18 

[jira] [Created] (KAFKA-14096) Race Condition in Log Rolling Leading to Disk Failure

2022-07-21 Thread Eric Azama (Jira)
Eric Azama created KAFKA-14096:
--

 Summary: Race Condition in Log Rolling Leading to Disk Failure
 Key: KAFKA-14096
 URL: https://issues.apache.org/jira/browse/KAFKA-14096
 Project: Kafka
  Issue Type: Bug
Affects Versions: 2.5.1
Reporter: Eric Azama


We've recently encountered what appears to be a race condition that can lead to 
disk being marked offline. One of our brokers recently crashed because its log 
directory failed. We found the following in the server.log file
{code:java}
[2022-07-18 18:24:42,940] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Rolled new log segment at offset 141946850 in 37 ms. 
(kafka.log.Log)
[...]
[2022-07-18 18:24:47,782] INFO [Log partition=TOPIC-REDACTED-15, 
dir=/data1/kafka-logs] Scheduling segments for deletion 
List(LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678)) (kafka.log.Log)
[2022-07-18 18:24:48,024] ERROR Error while flushing log for TOPIC-REDACTED-15 
in dir /data1/kafka-logs with offset 141935201 
(kafka.server.LogDirFailureChannel)
java.nio.channels.ClosedChannelException
at java.base/sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:150)
at java.base/sun.nio.ch.FileChannelImpl.force(FileChannelImpl.java:452)
at org.apache.kafka.common.record.FileRecords.flush(FileRecords.java:176)
at kafka.log.LogSegment.$anonfun$flush$1(LogSegment.scala:472)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at kafka.log.LogSegment.flush(LogSegment.scala:471)
at kafka.log.Log.$anonfun$flush$4(Log.scala:1956)
at kafka.log.Log.$anonfun$flush$4$adapted(Log.scala:1955)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
at kafka.log.Log.$anonfun$flush$2(Log.scala:1955)
at kafka.log.Log.flush(Log.scala:2322)
at kafka.log.Log.$anonfun$roll$9(Log.scala:1925)
at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)
[2022-07-18 18:24:48,036] ERROR Uncaught exception in scheduled task 
'flush-log' (kafka.utils.KafkaScheduler)
org.apache.kafka.common.errors.KafkaStorageException: Error while flushing log 
for TOPIC-REDACTED-15 in dir /data1/kafka-logs with offset 141935201{code}
and the following in the log-cleaner.log file
{code:java}
[2022-07-18 18:24:47,062] INFO Cleaner 0: Cleaning 
LogSegment(baseOffset=141935201, size=1073560219, 
lastModifiedTime=1658168598869, largestTime=1658168495678) in log 
TOPIC-REDACTED-15 into 141935201 with deletion horizon 1658082163480, retaining 
deletes. (kafka.log.LogCleaner) {code}
 

The timing of the log-cleaner log shows that the log flush failed because the 
log segment had been cleaned and the underlying file was already renamed or 
deleted.


The stacktrace indicates that the log flush that triggered the exception was 
part of the process of rolling a new log segment. (at 
kafka.log.Log.$anonfun$roll$9([Log.scala:1925|[https://github.com/apache/kafka/blob/2.5.1/core/src/main/scala/kafka/log/Log.scala#L1925]))]
 This is somewhat concerning because this flush should have been scheduled with 
no delay, but the exception occurred about 6 second after the most recent roll, 
and it wasn't even the most recent file. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)