Bharat Kondeti created KAFKA-8172: ------------------------------------- Summary: FileSystemException: The process cannot access the file because it is being used by another process Key: KAFKA-8172 URL: https://issues.apache.org/jira/browse/KAFKA-8172 Project: Kafka Issue Type: Bug Components: core Affects Versions: 2.1.1, 2.2.0, 1.1.1 Environment: Windows Reporter: Bharat Kondeti Fix For: 2.1.1, 2.2.0, 1.1.1
Fix to close file handlers before renaming files / directories and open them back if required Following are the file renaming scenarios: * Files are renamed to .deleted so they can be deleted * .cleaned files are renamed to .swap as part of Log.replaceSegments flow * .swap files are renamed to original files as part of Log.replaceSegments flow Following are the folder renaming scenarios: * When a topic is marked for deletion, folder is renamed * As part of replacing current logs with future logs in LogManager In above scenarios, if file handles are not closed, we get file access violation exception Idea is to close the logs and file segments before doing a rename and open them back up if required. *Segments Deletion Scenario* [2018-06-01 17:00:07,566] ERROR Error while deleting segments for test4-1 in dir D:\data\Kafka\kafka-logs (kafka.server.LogDirFailureChannel) java.nio.file.FileSystemException: D:\data\Kafka\kafka-logs\test4-1\00000000000000000000.log -> D:\data\Kafka\kafka-logs\test4-1\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process. 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:697) at org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:212) at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:415) at kafka.log.Log.kafka$log$Log$$asyncDeleteSegment(Log.scala:1601) at kafka.log.Log.kafka$log$Log$$deleteSegment(Log.scala:1588) at kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1170) at kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$apply$mcI$sp$1.apply(Log.scala:1170) 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:1170) at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1161) at kafka.log.Log$$anonfun$deleteSegments$1.apply(Log.scala:1161) at kafka.log.Log.maybeHandleIOException(Log.scala:1678) at kafka.log.Log.deleteSegments(Log.scala:1161) at kafka.log.Log.deleteOldSegments(Log.scala:1156) at kafka.log.Log.deleteRetentionMsBreachedSegments(Log.scala:1228) at kafka.log.Log.deleteOldSegments(Log.scala:1222) at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:854) at kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager.scala:852) 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:852) at kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(LogManager.scala:385) at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:62) 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: D:\data\Kafka\kafka-logs\test4-1\00000000000000000000.log -> D:\data\Kafka\kafka-logs\test4-1\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process. 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:694) ... 32 more *Topic deletion scenario* 2018-06-07 15:05:17,805] ERROR Error while renaming dir for test5-1 in log dir D:\data\Kafka\kafka-logs (kafka.server.LogDirFailureChannel) java.nio.file.AccessDeniedException: D:\data\Kafka\kafka-logs\test5-1 -> D:\data\Kafka\kafka-logs\test5-1.87985bad40e441e1a4d08af4541db7ce-delete at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83) 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:697) at kafka.log.Log$$anonfun$renameDir$1.apply$mcV$sp(Log.scala:579) at kafka.log.Log$$anonfun$renameDir$1.apply(Log.scala:577) at kafka.log.Log$$anonfun$renameDir$1.apply(Log.scala:577) at kafka.log.Log.maybeHandleIOException(Log.scala:1678) at kafka.log.Log.renameDir(Log.scala:577) at kafka.log.LogManager.asyncDelete(LogManager.scala:813) at kafka.cluster.Partition$$anonfun$delete$1.apply(Partition.scala:240) at kafka.cluster.Partition$$anonfun$delete$1.apply(Partition.scala:235) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:250) at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:258) at kafka.cluster.Partition.delete(Partition.scala:235) at kafka.server.ReplicaManager.stopReplica(ReplicaManager.scala:347) at kafka.server.ReplicaManager$$anonfun$stopReplicas$2.apply(ReplicaManager.scala:377) at kafka.server.ReplicaManager$$anonfun$stopReplicas$2.apply(ReplicaManager.scala:375) 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.server.ReplicaManager.stopReplicas(ReplicaManager.scala:375) at kafka.server.KafkaApis.handleStopReplicaRequest(KafkaApis.scala:198) at kafka.server.KafkaApis.handle(KafkaApis.scala:109) at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69) at java.lang.Thread.run(Thread.java:748) Suppressed: java.nio.file.AccessDeniedException: D:\data\Kafka\kafka-logs\test5-1 -> D:\data\Kafka\kafka-logs\test5-1.87985bad40e441e1a4d08af4541db7ce-delete at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83) 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:694) ... 23 more *File Compaction Scenario* [2019-03-11 16:02:00,021] ERROR Error while loading log dir D:\data\Kafka\kafka-datalogs (kafka.log.LogManager) java.nio.file.FileSystemException: D:\data\Kafka\kafka-datalogs\kattesttopic-23\00000000000003505795.log: The process cannot access the file because it is being used by another process. at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97) at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102) at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:376) 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:697) at org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:223) at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:415) at kafka.log.Log.replaceSegments(Log.scala:1697) at kafka.log.Log$$anonfun$completeSwapOperations$1.apply(Log.scala:391) at kafka.log.Log$$anonfun$completeSwapOperations$1.apply(Log.scala:380) at scala.collection.immutable.Set$Set1.foreach(Set.scala:94) at kafka.log.Log.completeSwapOperations(Log.scala:380) at kafka.log.Log.loadSegments(Log.scala:408) at kafka.log.Log.<init>(Log.scala:216) at kafka.log.Log$.apply(Log.scala:1788) at kafka.log.LogManager.kafka$log$LogManager$$loadLog(LogManager.scala:260) at kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$11$$anonfun$apply$15$$anonfun$apply$2.apply$mcV$sp(LogManager.scala:340) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:62) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) 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.AccessDeniedException: D:\data\Kafka\kafka-datalogs\kattesttopic-23\00000000000003505795.log.swap -> D:\data\Kafka\kafka-datalogs\kattesttopic-23\00000000000003505795.log at sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83) 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:694) ... 18 more -- This message was sent by Atlassian JIRA (v7.6.3#76005)