[ https://issues.apache.org/jira/browse/KAFKA-6647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16819732#comment-16819732 ]
ASF GitHub Bot commented on KAFKA-6647: --------------------------------------- jukkakarvanen commented on pull request #6569: KAFKA-6647 KafkaStreams.cleanUp creates .lock file in directory its trying to clean (Windows OS) URL: https://github.com/apache/kafka/pull/6569 This KAFKA-6647 KafkaStreams.cleanUp creates .lock file in directory its trying to clean (Windows OS) has been open for a while, and long conversation in previous Pull Requests. Anyway this problems is causing the TopologyTestDriver driver based unit test failing in Windows if not adding extra exception handling in there. This PR version is trying to keep the general functionality as similar as earlier. Only add one extra retry of delete, if first failed due to DirectoryNotExmptyException. When added the retry logic only at the end of finally, caused checkstyle CyclomaticComplexity and NPathComplexity to go above threshold. After it extracted cleanRemovedTaskDir and deleteTaskDir methods to avoid complexity. Also time condition evaluation changed to be first before locking, so no need to lock if inner block is doing nothing. No external functionality changed, so no additional test cases added. Following five test in StateDirectoryTest failed earlier in Windows. StateDirectoryTest.shouldCleanUpTaskStateDirectoriesThatAreNotCurrentlyLocked StateDirectoryTest.shouldCleanupStateDirectoriesWhenLastModifiedIsLessThanNowMinusCleanupDelay StateDirectoryTest.shouldNotLockStateDirLockedByAnotherThread StateDirectoryTest.shouldNotUnLockStateDirLockedByAnotherThread StateDirectoryTest.shouldCleanupAllTaskDirectoriesIncludingGlobalOne Test shouldNotLockStateDirLockedByAnotherThread is still failing due to there is no way to unlock task of already dead thread without adding extra code to StateDirectory class. The test left as is, but explanatory comment added. The rest four of those five tests are now successful also in Windows. Also shouldUseSerdesDefinedInMaterializedToConsumeGlobalTable test in StreamsBuilderTest failed in Windows before this change. After the change all the tests in StreamsBuilderTest are succesful. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > 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 > > 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)