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

Gaël Fontenelle edited comment on KAFKA-6647 at 8/29/18 10:13 AM:
------------------------------------------------------------------

I have a similar issue too 
{code:java}
2018-08-29 12:06:34.749 INFO 38968 --- [e-CleanupThread] 
o.a.k.s.p.internals.StateDirectory : stream-thread 
[events-step-three-CleanupThread] Deleting obsolete state directory 8_0 for 
task 8_0 as 600014ms has elapsed (cleanup delay is 600000ms).
2018-08-29 12:06:34.749 ERROR 38968 --- [e-CleanupThread] 
o.a.k.s.p.internals.StateDirectory : stream-thread 
[events-step-three-CleanupThread] Failed to delete the state directory.

java.nio.file.DirectoryNotEmptyException: 
\tmp\kafka-streams\events-step-three\8_0
at 
sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:266)
 ~[na:1.8.0_171]
at 
sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
 ~[na:1.8.0_171]
at java.nio.file.Files.delete(Files.java:1126) ~[na:1.8.0_171]
at org.apache.kafka.common.utils.Utils$2.postVisitDirectory(Utils.java:740) 
~[kafka-clients-2.0.0.jar:na]
at org.apache.kafka.common.utils.Utils$2.postVisitDirectory(Utils.java:723) 
~[kafka-clients-2.0.0.jar:na]
at java.nio.file.Files.walkFileTree(Files.java:2688) ~[na:1.8.0_171]
at java.nio.file.Files.walkFileTree(Files.java:2742) ~[na:1.8.0_171]
at org.apache.kafka.common.utils.Utils.delete(Utils.java:723) 
~[kafka-clients-2.0.0.jar:na]
at 
org.apache.kafka.streams.processor.internals.StateDirectory.cleanRemovedTasks(StateDirectory.java:287)
 [kafka-streams-2.0.0.jar:na]
at 
org.apache.kafka.streams.processor.internals.StateDirectory.cleanRemovedTasks(StateDirectory.java:250)
 [kafka-streams-2.0.0.jar:na]
at org.apache.kafka.streams.KafkaStreams$2.run(KafkaStreams.java:800) 
[kafka-streams-2.0.0.jar:na]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
[na:1.8.0_171]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
[na:1.8.0_171]
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
 [na:1.8.0_171]
at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
 [na:1.8.0_171]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
[na:1.8.0_171]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[na:1.8.0_171]
at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
{code}
 



> KafkaStreams.cleanUp creates .lock file in directory its trying to clean 
> (Windows OS)
> -------------------------------------------------------------------------------------
>
>                 Key: KAFKA-6647
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6647
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 1.0.1
>         Environment: windows 10.
> java version "1.8.0_162"
> Java(TM) SE Runtime Environment (build 1.8.0_162-b12)
> Java HotSpot(TM) 64-Bit Server VM (build 25.162-b12, mixed mode)
> org.apache.kafka:kafka-streams:1.0.1
> Kafka commitId : c0518aa65f25317e
>            Reporter: George Bloggs
>            Priority: Minor
>              Labels: streams
>
> When calling kafkaStreams.cleanUp() before starting a stream the 
> StateDirectory.cleanRemovedTasks() method contains this check:
> {code:java}
> ... Line 240
>                   if (lock(id, 0)) {
>                         long now = time.milliseconds();
>                         long lastModifiedMs = taskDir.lastModified();
>                         if (now > lastModifiedMs + cleanupDelayMs) {
>                             log.info("{} Deleting obsolete state directory {} 
> for task {} as {}ms has elapsed (cleanup delay is {}ms)", logPrefix(), 
> dirName, id, now - lastModifiedMs, cleanupDelayMs);
>                             Utils.delete(taskDir);
>                         }
>                     }
> {code}
> The check for lock(id,0) will create a .lock file in the directory that 
> subsequently is going to be deleted. If the .lock file already exists from a 
> previous run the attempt to delete the .lock file fails with 
> AccessDeniedException.
> This leaves the .lock file in the taskDir. Calling Utils.delete(taskDir) will 
> then attempt to remove the taskDir path calling Files.delete(path).
> The call to files.delete(path) in postVisitDirectory will then fail 
> java.nio.file.DirectoryNotEmptyException as the failed attempt to delete the 
> .lock file left the directory not empty. (o.a.k.s.p.internals.StateDirectory  
>      : stream-thread [restartedMain] Failed to lock the state directory due 
> to an unexpected exception)
> This seems to then cause issues using streams from a topic to an inMemory 
> store.



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

Reply via email to