[ https://issues.apache.org/jira/browse/KAFKA-6188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16789138#comment-16789138 ]
jaren commented on KAFKA-6188: ------------------------------ we use kafka_2.11_2.1.0 also have the problem. Failed to clean up log for __consumer_offsets-17 in dir E:\tmp\kafka-logs due to IOException (kafka.server.LogDirFailureChannel) java.nio.file.FileSystemException: E:\tmp\kafka-logs\__consumer_offsets-17\00000000000000000000.index.cleaned -> E:\tmp\kafka-logs\__consumer_offsets-17\00000000000000000000.index.swap: һٶԌѲֽ՚ʹԃՋτݾìԌϞרϊc 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:809) at kafka.log.AbstractIndex.renameTo(AbstractIndex.scala:205) at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:489) at kafka.log.Log.$anonfun$replaceSegments$4(Log.scala:1892) at kafka.log.Log.$anonfun$replaceSegments$4$adapted(Log.scala:1892) at scala.collection.immutable.List.foreach(List.scala:388) at kafka.log.Log.replaceSegments(Log.scala:1892) at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:583) at kafka.log.Cleaner.$anonfun$doClean$6(LogCleaner.scala:515) at kafka.log.Cleaner.$anonfun$doClean$6$adapted(LogCleaner.scala:514) at scala.collection.immutable.List.foreach(List.scala:388) at kafka.log.Cleaner.doClean(LogCleaner.scala:514) at kafka.log.Cleaner.clean(LogCleaner.scala:492) at kafka.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.scala:353) at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:319) at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:300) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82) Suppressed: java.nio.file.FileSystemException: E:\tmp\kafka-logs\__consumer_offsets-17\00000000000000000000.index.cleaned -> E:\tmp\kafka-logs\__consumer_offsets-17\00000000000000000000.index.swap: һٶԌѲֽ՚ʹԃՋτݾìԌϞרϊc 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:806) ... 16 more [2019-03-10 18:21:20,530] INFO [ReplicaManager broker=0] Stopping serving replicas in dir E:\tmp\kafka-logs (kafka.server.ReplicaManager) [2019-03-10 18:21:20,533] INFO [ReplicaFetcherManager on broker 0] Removed fetcher for partitions Set(__consumer_offsets-22, __consumer_offsets-30, OBSERVE_DEVICE-0, __consumer_offsets-8, __consumer_offsets-21, __consumer_offsets-4, __consumer_offsets-27, __consumer_offsets-7, __consumer_offsets-9, __consumer_offsets-46, FOTA_DOWNLOAD_ERROR-0, __consumer_offsets-25, DEVICE_DE_REGISTER-0, __consumer_offsets-35, DEVICE_REG_UPDATE-0, __consumer_offsets-41, __consumer_offsets-33, __consumer_offsets-23, __consumer_offsets-49, __consumer_offsets-47, __consumer_offsets-16, __consumer_offsets-28, FOTA_IMEI_MONITOR-0, __consumer_offsets-31, __consumer_offsets-36, __consumer_offsets-42, __consumer_offsets-3, __consumer_offsets-18, DATA_TO_DEVICE-0, __consumer_offsets-37, __consumer_offsets-15, __consumer_offsets-24, FOTA_PLAIN_MONITOR_FORCE-0, DEVICE_REGISTER-0, __consumer_offsets-38, __consumer_offsets-17, DEVICE_REP-0, __consumer_offsets-48, __consumer_offsets-19, __consumer_offsets-11, __consumer_offsets-13, __consumer_offsets-2, __consumer_offsets-43, __consumer_offsets-6, __consumer_offsets-14, FOTA_STATICS_MONITOR-0, __consumer_offsets-20, __consumer_offsets-0, __consumer_offsets-44, __consumer_offsets-39, FOTA_STATE_CHANGE-0, __consumer_offsets-12, FOTA_UPGRADE_NOTIFY-0, __consumer_offsets-45, __consumer_offsets-1, __consumer_offsets-5, __consumer_offsets-26, __consumer_offsets-29, __consumer_offsets-34, __consumer_offsets-10, __consumer_offsets-32, __consumer_offsets-40) (kafka.server.ReplicaFetcherManager) [2019-03-10 18:21:20,534] INFO [ReplicaAlterLogDirsManager on broker 0] Removed fetcher for partitions Set(__consumer_offsets-22, __consumer_offsets-30, OBSERVE_DEVICE-0, __consumer_offsets-8, __consumer_offsets-21, __consumer_offsets-4, __consumer_offsets-27, __consumer_offsets-7, __consumer_offsets-9, __consumer_offsets-46, FOTA_DOWNLOAD_ERROR-0, __consumer_offsets-25, DEVICE_DE_REGISTER-0, __consumer_offsets-35, DEVICE_REG_UPDATE-0, __consumer_offsets-41, __consumer_offsets-33, __consumer_offsets-23, __consumer_offsets-49, __consumer_offsets-47, __consumer_offsets-16, __consumer_offsets-28, FOTA_IMEI_MONITOR-0, __consumer_offsets-31, __consumer_offsets-36, __consumer_offsets-42, __consumer_offsets-3, __consumer_offsets-18, DATA_TO_DEVICE-0, __consumer_offsets-37, __consumer_offsets-15, __consumer_offsets-24, FOTA_PLAIN_MONITOR_FORCE-0, DEVICE_REGISTER-0, __consumer_offsets-38, __consumer_offsets-17, DEVICE_REP-0, __consumer_offsets-48, __consumer_offsets-19, __consumer_offsets-11, __consumer_offsets-13, __consumer_offsets-2, __consumer_offsets-43, __consumer_offsets-6, __consumer_offsets-14, FOTA_STATICS_MONITOR-0, __consumer_offsets-20, __consumer_offsets-0, __consumer_offsets-44, __consumer_offsets-39, FOTA_STATE_CHANGE-0, __consumer_offsets-12, FOTA_UPGRADE_NOTIFY-0, __consumer_offsets-45, __consumer_offsets-1, __consumer_offsets-5, __consumer_offsets-26, __consumer_offsets-29, __consumer_offsets-34, __consumer_offsets-10, __consumer_offsets-32, __consumer_offsets-40) (kafka.server.ReplicaAlterLogDirsManager) [2019-03-10 18:21:20,586] INFO [ReplicaManager broker=0] Broker 0 stopped fetcher for partitions __consumer_offsets-22,__consumer_offsets-30,OBSERVE_DEVICE-0,__consumer_offsets-8,__consumer_offsets-21,__consumer_offsets-4,__consumer_offsets-27,__consumer_offsets-7,__consumer_offsets-9,__consumer_offsets-46,FOTA_DOWNLOAD_ERROR-0,__consumer_offsets-25,DEVICE_DE_REGISTER-0,__consumer_offsets-35,DEVICE_REG_UPDATE-0,__consumer_offsets-41,__consumer_offsets-33,__consumer_offsets-23,__consumer_offsets-49,__consumer_offsets-47,__consumer_offsets-16,__consumer_offsets-28,FOTA_IMEI_MONITOR-0,__consumer_offsets-31,__consumer_offsets-36,__consumer_offsets-42,__consumer_offsets-3,__consumer_offsets-18,DATA_TO_DEVICE-0,__consumer_offsets-37,__consumer_offsets-15,__consumer_offsets-24,FOTA_PLAIN_MONITOR_FORCE-0,DEVICE_REGISTER-0,__consumer_offsets-38,__consumer_offsets-17,DEVICE_REP-0,__consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,__consumer_offsets-13,__consumer_offsets-2,__consumer_offsets-43,__consumer_offsets-6,__consumer_offsets-14,FOTA_STATICS_MONITOR-0,__consumer_offsets-20,__consumer_offsets-0,__consumer_offsets-44,__consumer_offsets-39,FOTA_STATE_CHANGE-0,__consumer_offsets-12,FOTA_UPGRADE_NOTIFY-0,__consumer_offsets-45,__consumer_offsets-1,__consumer_offsets-5,__consumer_offsets-26,__consumer_offsets-29,__consumer_offsets-34,__consumer_offsets-10,__consumer_offsets-32,__consumer_offsets-40 and stopped moving logs for partitions because they are in the failed log directory E:\tmp\kafka-logs. (kafka.server.ReplicaManager) [2019-03-10 18:21:20,587] INFO Stopping serving logs in dir E:\tmp\kafka-logs (kafka.log.LogManager) [2019-03-10 18:21:20,592] ERROR Shutdown broker because all log dirs in E:\tmp\kafka-logs have failed (kafka.log.LogManager) [2019-03-10 18:21:20,936] WARN Exception causing close of session 0x103d26725330007: ԶԌܺǿLjژҕһٶЖԐքlޓc (org.apache.zookeeper.server.NIOServerCnxn) [2019-03-10 18:21:20,938] INFO Closed socket connection for client /127.0.0.1:53874 which had sessionid 0x103d26725330007 (org.apache.zookeeper.server.NIOServerCnxn) [2019-03-10 18:21:27,764] INFO Expiring session 0x103d26725330007, timeout of 6000ms exceeded (org.apache.zookeeper.server.ZooKeeperServer) [2019-03-10 18:21:27,764] INFO Processed session termination for sessionid: 0x103d26725330007 (org.apache.zookeeper.server.PrepRequestProcessor) > 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)