Skip to content

KAFKA-6647: Do note delete the lock file while holding the lock#8267

Merged
guozhangwang merged 10 commits intoapache:trunkfrom
guozhangwang:K6647-do-not-delete-lock-file
Mar 14, 2020
Merged

KAFKA-6647: Do note delete the lock file while holding the lock#8267
guozhangwang merged 10 commits intoapache:trunkfrom
guozhangwang:K6647-do-not-delete-lock-file

Conversation

@guozhangwang
Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang commented Mar 10, 2020

  1. Inside StateDirectory#cleanRemovedTasks, skip deleting the lock file (and hence the parent directory) until releasing the lock. And after the lock is released only go ahead and delete the parent directory if manualUserCall == true. That is, this is triggered from KafkaStreams#cleanUp and users are responsible to make sure that Streams instance is not started and hence there are no other threads trying to grab that lock.

  2. As a result, during scheduled cleanup the corresponding task.dir would not be empty but be left with only the lock file, so effectively we still achieve the goal of releasing disk spaces. For callers of listTaskDirectories like KIP-441 (cc @ableegoldman to take a look) I've introduced a new listNonEmptyTaskDirectories which excludes such dummy task.dirs with only the lock file left.

  3. Also fixed KAFKA-8999 along the way to expose the exception while traversing the directory.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

if (manualUserCall) {
throw e;
}
} catch (final OverlappingFileLockException | IOException e) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a minor code clean up.

throw exc;
}

if (rootFile.toPath().equals(path)) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a bit tricky: for the root file, we only consider deleting it if there's no specified skipping sub-files; otherwise we never try to delete since it would doom with DirectoryNotEmpty.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be good to avoid the unnecessary conversions in this method (rootFile.toPath, path.toFile). We can do some work at the start of the method to improve efficiency.

private final Time time;
private final String appId;
private final File stateDir;
private final boolean createStateDirectory;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What do you think about renaming this, eg something like isStateful, hasStatefulTopology`, etc?

It's really confusing to reason about in its usage, for example why createStateDirectory => #lock returns true. You have to backtrack to where createStateDirectory is set to understand this. However, I think it's easy to reason about why isStateful => should create state directory.

Alternatively, what if we have a stateless version of the StateDirectory class that just stubs things where appropriate. Then we could get rid of this altogether. WDYT?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense, I will do the renaming.

* @return The list of all the existing local directories for stream tasks
*/
File[] listTaskDirectories() {
File[] lisAllTaskDirectories() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
File[] lisAllTaskDirectories() {
File[] listAllTaskDirectories() {

@mjsax
Copy link
Copy Markdown
Member

mjsax commented Mar 11, 2020

@guozhangwang I opened a PR for something related some time ago: #4713

Does it overlap?

@guozhangwang
Copy link
Copy Markdown
Contributor Author

@guozhangwang I opened a PR for something related some time ago: #4713

Does it overlap?

Good call.

My proposal is to skip trying to delete the lock file, which I think is essentially similar to yours (capture the exception and then skip).

@guozhangwang
Copy link
Copy Markdown
Contributor Author

I've finally able to re-produce the issue with NTFS using this example code: https://github.com/simplesteph/kafka-streams-course/blob/1.1.0/word-count/src/test/java/com/github/simplesteph/udemy/kafka/streams/WordCountAppTest.java

Screen Shot 2020-03-12 at 4 06 40 PM

And with my PR now it is proven fixed:

Screen Shot 2020-03-12 at 4 09 30 PM

I'm going to add more unit tests in my PR but the non-testing part should be ready to review now cc @mjsax @vvcephei

@guozhangwang
Copy link
Copy Markdown
Contributor Author

Also cc @rodesai

@guozhangwang guozhangwang changed the title KAFKA-6647: Do note delete the lock file while holding the lock [WIP] KAFKA-6647: Do note delete the lock file while holding the lock Mar 13, 2020
@mjsax mjsax added the streams label Mar 13, 2020
Copy link
Copy Markdown
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, Guozhang!

Just a couple of questions, and I think Sophie’s question is a good one, too.

Otherwise, lgtm!


@Test
public void shouldListAllTaskDirectories() {
public void shouldNotListNonEmptyTaskDirectories() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the double negative intentional here?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call, will change.

} catch (final IOException e) {
log.error("{} Failed to release the state directory lock.", logPrefix());

// for manual user call, stream threads are not running so it is safe to delete
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment makes me wonder why we even bother with locking at all for manual calls. If we know there’s no app running, why not just delete the whole state directory and not bother with locks?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought about that too, and tried it out, it's just that based on manualUserCall to decide whether lock / unlock the block, we ended up with either much more code duplicates or clumsy and finer-grained if else condition, neither of which I like. On the other hand, since streams.cleanUp is usually a one-time thing compared with the periodic internal clean I think having two delete calls are okay to same some code duplication here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok!

@guozhangwang
Copy link
Copy Markdown
Contributor Author

@vvcephei @ableegoldman addressed your comments, please take another look.

Copy link
Copy Markdown
Member

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We just merged PR 8246, can you rebase and make the TaskManager#tryToLockAllTaskDirectoriesleverage the newlistNonEmptyTaskDirectories` method?
Otherwise LGTM

@guozhangwang
Copy link
Copy Markdown
Contributor Author

guozhangwang commented Mar 14, 2020

@ableegoldman Just clarifying on the releaseLockedUnassignedTaskDirectories side, it's okay to just iterate lockedTaskDirectories.iterator() and not stateDirectory.listNonEmptyTaskDirectories() right?

Also I renamed tryToLockAllTaskDirectories to tryToLockAllNonEmptyTaskDirectories.

@guozhangwang guozhangwang merged commit 605d55d into apache:trunk Mar 14, 2020
guozhangwang added a commit that referenced this pull request Mar 14, 2020
1. Inside StateDirectory#cleanRemovedTasks, skip deleting the lock file (and hence the parent directory) until releasing the lock. And after the lock is released only go ahead and delete the parent directory if manualUserCall == true. That is, this is triggered from KafkaStreams#cleanUp and users are responsible to make sure that Streams instance is not started and hence there are no other threads trying to grab that lock.

2. As a result, during scheduled cleanup the corresponding task.dir would not be empty but be left with only the lock file, so effectively we still achieve the goal of releasing disk spaces. For callers of listTaskDirectories like KIP-441 (cc @ableegoldman to take a look) I've introduced a new listNonEmptyTaskDirectories which excludes such dummy task.dirs with only the lock file left.

3. Also fixed KAFKA-8999 along the way to expose the exception while traversing the directory.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <vvcephei@apache.org>
@guozhangwang
Copy link
Copy Markdown
Contributor Author

Merged to trunk and 2.5 after tests green locally.

ijuma added a commit to confluentinc/kafka that referenced this pull request Mar 17, 2020
* apache-github/trunk: (39 commits)
  MINOR: cleanup and add tests to StateDirectoryTest (apache#8304)
  HOTFIX: StateDirectoryTest should use Set instead of List (apache#8305)
  MINOR: Fix build and JavaDoc warnings (apache#8291)
  MINOR: Fix kafka.server.RequestQuotaTest missing new ApiKeys. (apache#8302)
  KAFKA-9712: Catch and handle exception thrown by reflections scanner (apache#8289)
  KAFKA-9670; Reduce allocations in Metadata Response preparation (apache#8236)
  MINOR: fix Scala 2.13 build error introduced in apache#8083 (apache#8301)
  MINOR: enforce non-negative invariant for checkpointed offsets (apache#8297)
  MINOR: comment apikey types in generated switch (apache#8201)
  MINOR: Fix typo in CreateTopicsResponse.json (apache#8300)
  KIP-546: Implement describeClientQuotas and alterClientQuotas. (apache#8083)
  KAFKA-6647: Do note delete the lock file while holding the lock (apache#8267)
  KAFKA-9677: Fix consumer fetch with small consume bandwidth quotas (apache#8290)
  KAFKA-9533: Fix JavaDocs of KStream.transformValues (apache#8298)
  MINOR: reuse pseudo-topic in FKJoin (apache#8296)
  KAFKA-6145: Pt 2. Include offset sums in subscription (apache#8246)
  KAFKA-9714; Eliminate unused reference to IBP in `TransactionStateManager` (apache#8293)
  KAFKA-9718; Don't log passwords for AlterConfigs in request logs (apache#8294)
  KAFKA-8768: DeleteRecords request/response automated protocol (apache#7957)
  KAFKA-9685: Solve Set concatenation perf issue in AclAuthorizer
  ...
@SaberZA
Copy link
Copy Markdown

SaberZA commented Mar 25, 2020

Thank you so much for this fix!! Any idea when this will get into the next release? or how I could integrate the change sooner?

@guozhangwang
Copy link
Copy Markdown
Contributor Author

It is merged into 2.5 and branch, so it will be at least in 2.5.1 and 2.6.0; if 2.5.0 RC0 gets vetoed and a new one is cut, then it would also be in 2.5.0 as well.

@guozhangwang guozhangwang deleted the K6647-do-not-delete-lock-file branch April 25, 2020 00:00
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants