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

George Bloggs commented on KAFKA-6647:
--------------------------------------

Guozhang. 

I have not submitted any PRs. I too commented on one PR to say I did not 
believe it would resolve the issue. Furthermore, I see moving the .lock file to 
a parent directory is not a full solution. 

In reply to:

`...By looking into your issue, I think the root cause maybe that there is 
still some un-closed handle on that file in which Windows 10 would not actually 
delete the file...`

I do not believe this is true.  I have patched the issue in our code calling 
Kafka Utils.delete directly just before performing KafkaStreams.start(). This 
works. I also believe the issue is happening on Linux. We run our code on a 
linux instance deploying using ansible. The code is showing the same issues on 
linux although I am not able to debug it on the linux boxes to prove through.

I have debugged this on Windows and from what I was able to tell, it is the 
lock code that is causing the isssue. This is bourne out by the fact that my 
patch in our code works. 

As further proof that no other process is holding a handle on the file, the 
parent directory can be deleted through Windows Explorer before 
KafkaStreams.start() is called. If a handle was being held on the .lock file 
Windows would prevent the deletion I believe. 

The shutdownHook is not overly important I believe but it simply has 3 ines of 
code:
```java
kafkaStreams.close();
kafkaStreams.cleanUp();
```

We also call kafkaStreams.cleanUp(); on the line *BEFORE* kafkaStreams.start() 
as per documentation.

`So I'd suggest we hold on the proposed PR and try to investigate further what 
actually causes AccessDeniedException.` 
I agree. The issue is more subtle than simply moving the .lock file to an 
alternative location. I am unable to access our GitLab repo at present but will 
copy my hack to allow our code to work tomorrow. This is merely a hack in our 
code but using Kafka Utils.delete without using KafkaStreams.cleanUp(). To be 
clear, I am not stating this is a perfect solution, its a hack to get our code 
working in the hope a full solution in KafkaStreams.cleanUp() can be found. It 
works, in the same codebase, with the only difference being my solution goes 
direct to Utils.delete() without checking the lock. I can do this as there is 
only one instance of our app running on one instance for now.
LOG.info("KafkaStream shutdown hook completed");

> KafkaStreams.cleanUp creates .lock file in directory its trying to clean
> ------------------------------------------------------------------------
>
>                 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