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

Hans Schuell commented on KAFKA-6188:
-------------------------------------

I have also a Confluent 5.0.0 (Kafka 2.0.0) test system on Windows 10 and the 
broker stopped working after I have deleted a topic and the broker was 
restarted.

The stack trace from server.log is below. The german system message from the 
Windows operating system says, that the process cannot access the file, because 
it is still in use.

The english Windows system message would be "*the process cannot access the 
file because it is being used by another process*"

This is one major difference between *nix and Windows. Under Windows a file 
cannot be moved (in the case below) or deleted, when there is still an open 
file handle of the running current process or of another process. I know, that 
Windows is not supported, but it is very likely, that also under Linux it is a 
bit weird, that the file xxx.log is renamed to xxx.log.deleted by the 
"scheduler" while the file is not yet closed. 

{{[2018-11-27 17:11:04,811] INFO [Log partition=test-events-1-4, 
dir=C:\tmp\confluent\kafka-logs] Scheduling log segment [baseOffset 0, size 
219591] for deletion. (kafka.log.Log)}}
{{[2018-11-27 17:11:04,814] ERROR Error while deleting segments for 
test-events-1-4 in dir C:\tmp\confluent\kafka-logs 
(kafka.server.LogDirFailureChannel)}}
{{java.nio.file.FileSystemException: 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log -> 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log.deleted: 
Der Prozess kann nicht auf die Datei zugreifen, da sie von einem anderen 
Prozess verwendet wird.}}{{ at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)}}
{{ at 
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)}}
{{ at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:387)}}
{{ at 
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)}}
{{ at java.nio.file.Files.move(Files.java:1395)}}
{{ at 
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:786)}}
{{ at 
org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:211)}}
{{ at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:488)}}
{{ at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:1751)}}
{{ at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:1738)}}
{{ at 
kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1309)}}
{{ at 
kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1309)}}
{{ at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)}}
{{ at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply$mcI$sp(Log.scala:1309)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1300)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1300)}}
{{ at kafka.log.Log.maybeHandleIOException(Log.scala:1837)}}
{{ at kafka.log.Log.deleteSegments(Log.scala:1300)}}
{{ at kafka.log.Log.deleteOldSegments(Log.scala:1295)}}
{{ at kafka.log.Log.deleteRetentionMsBreachedSegments(Log.scala:1367)}}
{{ at kafka.log.Log.deleteOldSegments(Log.scala:1361)}}
{{ at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:874)}}
{{ at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:872)}}
{{ at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)}}
{{ at scala.collection.immutable.List.foreach(List.scala:392)}}
{{ at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)}}
{{ at kafka.log.LogManager.cleanupLogs(LogManager.scala:872)}}
{{ at 
kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:395)}}
{{ at 
kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:114)}}
{{ at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63)}}
{{ at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)}}
{{ at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)}}
{{ at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)}}
{{ at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)}}
{{ 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)}}
{{ Suppressed: java.nio.file.FileSystemException: 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log -> 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log.deleted: 
Der Prozess kann nicht auf die Datei zugreifen, da sie von einem anderen 
Prozess verwendet wird.}}{{  at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)}}
{{  at 
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)}}
{{  at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301)}}
{{  at 
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)}}
{{  at java.nio.file.Files.move(Files.java:1395)}}
{{  at 
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:783)}}
{{  ... 32 more}}
{{[2018-11-27 17:11:04,818] INFO [ReplicaManager broker=0] Stopping serving 
replicas in dir C:\tmp\confluent\kafka-logs (kafka.server.ReplicaManager)}}
{{[2018-11-27 17:11:04,819] ERROR Uncaught exception in scheduled task 
'kafka-log-retention' (kafka.utils.KafkaScheduler)}}
{{org.apache.kafka.common.errors.KafkaStorageException: Error while deleting 
segments for test-events-1-4 in dir C:\tmp\confluent\kafka-logs}}
{{Caused by: java.nio.file.FileSystemException: 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log -> 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log.deleted: 
Der Prozess kann nicht auf die Datei zugreifen, da sie von einem anderen 
Prozess verwendet wird.}}{{ at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)}}
{{ at 
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)}}
{{ at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:387)}}
{{ at 
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)}}
{{ at java.nio.file.Files.move(Files.java:1395)}}
{{ at 
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:786)}}
{{ at 
org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:211)}}
{{ at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:488)}}
{{ at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:1751)}}
{{ at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:1738)}}
{{ at 
kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1309)}}
{{ at 
kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1309)}}
{{ at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)}}
{{ at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply$mcI$sp(Log.scala:1309)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1300)}}
{{ at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1300)}}
{{ at kafka.log.Log.maybeHandleIOException(Log.scala:1837)}}
{{ at kafka.log.Log.deleteSegments(Log.scala:1300)}}
{{ at kafka.log.Log.deleteOldSegments(Log.scala:1295)}}
{{ at kafka.log.Log.deleteRetentionMsBreachedSegments(Log.scala:1367)}}
{{ at kafka.log.Log.deleteOldSegments(Log.scala:1361)}}
{{ at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:874)}}
{{ at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:872)}}
{{ at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)}}
{{ at scala.collection.immutable.List.foreach(List.scala:392)}}
{{ at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)}}
{{ at kafka.log.LogManager.cleanupLogs(LogManager.scala:872)}}
{{ at 
kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:395)}}
{{ at 
kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:114)}}
{{ at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63)}}
{{ at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)}}
{{ at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)}}
{{ at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)}}
{{ at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)}}
{{ 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)}}
{{ Suppressed: java.nio.file.FileSystemException: 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log -> 
C:\tmp\confluent\kafka-logs\test-events-1-4\00000000000000000000.log.deleted: 
Der Prozess kann nicht auf die Datei zugreifen, da sie von einem anderen 
Prozess verwendet wird.}}{{  at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)}}
{{  at 
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)}}
{{  at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301)}}
{{  at 
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)}}
{{  at java.nio.file.Files.move(Files.java:1395)}}
{{  at 
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:783)}}
{{  ... 32 more}}
{{[2018-11-27 17:11:04,824] INFO [ReplicaFetcherManager on broker 0] Removed 
fetcher for partitions 
__consumer_offsets-22,__consumer_offsets-30,test-events-1-2,test-events-1-4,__consumer_offsets-8,__consumer_offsets-21,test-events-2-3,__consumer_offsets-4,__consumer_offsets-27,__consumer_offsets-7,__consumer_offsets-9,__consumer_offsets-46,__consumer_offsets-25,__consumer_offsets-35,__consumer_offsets-41,test-events-2-2,__consumer_offsets-33,__consumer_offsets-23,__consumer_offsets-49,test-events-1-1,__consumer_offsets-47,__consumer_offsets-16,__consumer_offsets-28,__consumer_offsets-31,__consumer_offsets-36,__consumer_offsets-42,__consumer_offsets-3,__consumer_offsets-18,test-events-3,__consumer_offsets-37,__consumer_offsets-15,__consumer_offsets-24,test-events-0,test-events-4,test-events-1-0,__consumer_offsets-38,__consumer_offsets-17,__consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,test-events-2,__consumer_offsets-13,__consumer_offsets-2,__consumer_offsets-43,__consumer_offsets-6,__consumer_offsets-14,test-events-2-4,test-events-2-0,test-events-1-3,__consumer_offsets-20,__consumer_offsets-0,__consumer_offsets-44,__consumer_offsets-39,__consumer_offsets-12,test-events-1,__consumer_offsets-45,__consumer_offsets-1,__consumer_offsets-5,__consumer_offsets-26,__consumer_offsets-29,__consumer_offsets-34,test-events-2-1,__consumer_offsets-10,__consumer_offsets-32,__consumer_offsets-40
 (kafka.server.ReplicaFetcherManager)}}
{{[2018-11-27 17:11:04,825] INFO [ReplicaAlterLogDirsManager on broker 0] 
Removed fetcher for partitions 
__consumer_offsets-22,__consumer_offsets-30,test-events-1-2,test-events-1-4,__consumer_offsets-8,__consumer_offsets-21,test-events-2-3,__consumer_offsets-4,__consumer_offsets-27,__consumer_offsets-7,__consumer_offsets-9,__consumer_offsets-46,__consumer_offsets-25,__consumer_offsets-35,__consumer_offsets-41,test-events-2-2,__consumer_offsets-33,__consumer_offsets-23,__consumer_offsets-49,test-events-1-1,__consumer_offsets-47,__consumer_offsets-16,__consumer_offsets-28,__consumer_offsets-31,__consumer_offsets-36,__consumer_offsets-42,__consumer_offsets-3,__consumer_offsets-18,test-events-3,__consumer_offsets-37,__consumer_offsets-15,__consumer_offsets-24,test-events-0,test-events-4,test-events-1-0,__consumer_offsets-38,__consumer_offsets-17,__consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,test-events-2,__consumer_offsets-13,__consumer_offsets-2,__consumer_offsets-43,__consumer_offsets-6,__consumer_offsets-14,test-events-2-4,test-events-2-0,test-events-1-3,__consumer_offsets-20,__consumer_offsets-0,__consumer_offsets-44,__consumer_offsets-39,__consumer_offsets-12,test-events-1,__consumer_offsets-45,__consumer_offsets-1,__consumer_offsets-5,__consumer_offsets-26,__consumer_offsets-29,__consumer_offsets-34,test-events-2-1,__consumer_offsets-10,__consumer_offsets-32,__consumer_offsets-40
 (kafka.server.ReplicaAlterLogDirsManager)}}
{{[2018-11-27 17:11:04,866] INFO [ReplicaManager broker=0] Broker 0 stopped 
fetcher for partitions 
__consumer_offsets-22,__consumer_offsets-30,test-events-1-2,test-events-1-4,__consumer_offsets-8,__consumer_offsets-21,test-events-2-3,__consumer_offsets-4,__consumer_offsets-27,__consumer_offsets-7,__consumer_offsets-9,__consumer_offsets-46,__consumer_offsets-25,__consumer_offsets-35,__consumer_offsets-41,test-events-2-2,__consumer_offsets-33,__consumer_offsets-23,__consumer_offsets-49,test-events-1-1,__consumer_offsets-47,__consumer_offsets-16,__consumer_offsets-28,__consumer_offsets-31,__consumer_offsets-36,__consumer_offsets-42,__consumer_offsets-3,__consumer_offsets-18,test-events-3,__consumer_offsets-37,__consumer_offsets-15,__consumer_offsets-24,test-events-0,test-events-4,test-events-1-0,__consumer_offsets-38,__consumer_offsets-17,__consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,test-events-2,__consumer_offsets-13,__consumer_offsets-2,__consumer_offsets-43,__consumer_offsets-6,__consumer_offsets-14,test-events-2-4,test-events-2-0,test-events-1-3,__consumer_offsets-20,__consumer_offsets-0,__consumer_offsets-44,__consumer_offsets-39,__consumer_offsets-12,test-events-1,__consumer_offsets-45,__consumer_offsets-1,__consumer_offsets-5,__consumer_offsets-26,__consumer_offsets-29,__consumer_offsets-34,test-events-2-1,__consumer_offsets-10,__consumer_offsets-32,__consumer_offsets-40
 and stopped moving logs for partitions  because they are in the failed log 
directory C:\tmp\confluent\kafka-logs. (kafka.server.ReplicaManager)}}
{{[2018-11-27 17:11:04,892] INFO Stopping serving logs in dir 
C:\tmp\confluent\kafka-logs (kafka.log.LogManager)}}
{{[2018-11-27 17:11:04,905] ERROR Shutdown broker because all log dirs in 
C:\tmp\confluent\kafka-logs have failed (kafka.log.LogManager)}}

 

> Broker fails with FATAL Shutdown - log dirs have failed
> -------------------------------------------------------
>
>                 Key: KAFKA-6188
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6188
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, log
>    Affects Versions: 1.0.0, 1.0.1
>         Environment: Windows 10
>            Reporter: Valentina Baljak
>            Assignee: Dong Lin
>            Priority: Blocker
>              Labels: windows
>         Attachments: Segments are opened before deletion, 
> kafka_2.10-0.10.2.1.zip, output.txt
>
>
> Just started with version 1.0.0 after a 4-5 months of using 0.10.2.1. The 
> test environment is very simple, with only one producer and one consumer. 
> Initially, everything started fine, stand alone tests worked as expected. 
> However, running my code, Kafka clients fail after approximately 10 minutes. 
> Kafka won't start after that and it fails with the same error. 
> Deleting logs helps to start again, and the same problem occurs.
> Here is the error traceback:
> [2017-11-08 08:21:57,532] INFO Starting log cleanup with a period of 300000 
> ms. (kafka.log.LogManager)
> [2017-11-08 08:21:57,548] INFO Starting log flusher with a default period of 
> 9223372036854775807 ms. (kafka.log.LogManager)
> [2017-11-08 08:21:57,798] INFO Awaiting socket connections on 0.0.0.0:9092. 
> (kafka.network.Acceptor)
> [2017-11-08 08:21:57,813] INFO [SocketServer brokerId=0] Started 1 acceptor 
> threads (kafka.network.SocketServer)
> [2017-11-08 08:21:57,829] INFO [ExpirationReaper-0-Produce]: Starting 
> (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> [2017-11-08 08:21:57,845] INFO [ExpirationReaper-0-DeleteRecords]: Starting 
> (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> [2017-11-08 08:21:57,845] INFO [ExpirationReaper-0-Fetch]: Starting 
> (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> [2017-11-08 08:21:57,845] INFO [LogDirFailureHandler]: Starting 
> (kafka.server.ReplicaManager$LogDirFailureHandler)
> [2017-11-08 08:21:57,860] INFO [ReplicaManager broker=0] Stopping serving 
> replicas in dir C:\Kafka\kafka_2.12-1.0.0\kafka-logs 
> (kafka.server.ReplicaManager)
> [2017-11-08 08:21:57,860] INFO [ReplicaManager broker=0] Partitions  are 
> offline due to failure on log directory C:\Kafka\kafka_2.12-1.0.0\kafka-logs 
> (kafka.server.ReplicaManager)
> [2017-11-08 08:21:57,860] INFO [ReplicaFetcherManager on broker 0] Removed 
> fetcher for partitions  (kafka.server.ReplicaFetcherManager)
> [2017-11-08 08:21:57,892] INFO [ReplicaManager broker=0] Broker 0 stopped 
> fetcher for partitions  because they are in the failed log dir 
> C:\Kafka\kafka_2.12-1.0.0\kafka-logs (kafka.server.ReplicaManager)
> [2017-11-08 08:21:57,892] INFO Stopping serving logs in dir 
> C:\Kafka\kafka_2.12-1.0.0\kafka-logs (kafka.log.LogManager)
> [2017-11-08 08:21:57,892] FATAL Shutdown broker because all log dirs in 
> C:\Kafka\kafka_2.12-1.0.0\kafka-logs have failed (kafka.log.LogManager)



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

Reply via email to