-
Notifications
You must be signed in to change notification settings - Fork 14k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-9618: Directory deletion failure leading to error task RocksDB open #8186
Conversation
@@ -83,8 +84,8 @@ static void wipeStateStores(final Logger log, final ProcessorStateManager stateM | |||
try { | |||
Utils.delete(stateMgr.baseDir()); | |||
} catch (final IOException fatalException) { | |||
// since it is only called under dirty close, we always swallow the exception |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we now always wipe the state stores (even on clean close)? cc/ @guozhangwang
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We do not wipe state stores on clean close, but only in dirty close.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
wipeStateStores
is only triggered if it is dirty close, under EOS.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I wasn't looking at the latest trunk
, I see you fixed that in a recent PR. In that case we should leave the comment?
// since it is only called under dirty close, we always swallow the exception | ||
log.warn("Failed to wiping state stores for task {}", stateMgr.taskId()); | ||
log.error("Failed to wiping state stores for task {} due to {}", stateMgr.taskId(), fatalException); | ||
throw new StreamsException(fatalException); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems like we should retry, at least if this is a clean close...otherwise this thread still dies, and also kills any other threads that then get assigned this task.
Or better yet, can we fix the state store to not blindly open non-existent files? If we successfully delete the checkpoint file but fail to delete the stores, Streams shouldn't attempt to open any store files.
If we fail at deleting the checkpoint, we're more at risk as the data may be corrupted (otherwise we wouldn't be trying to delete the task state to begin with). In that case it seems like the only safe thing to do would be to retry until we at least delete the checkpoint file
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My current feeling is that we should infinitely retry to make sure we clean up a directory completely, since in principal if this is not an atomic operation, the handling complexity during normal processing would be 2X IMHO. In this case, I would rather we stuck in here instead of proceed, as we potentially could already corrupt the state and what's even worse is that we don't know it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well, isn't the whole point of the checkpoint file to act as the source of truth? ie we should feel secure that deletion of the checkpoint file will always invalidate any leftover state, even if we fail to actually wipe it out -- whether the store or even the directory itself is left behind should not matter. We should absolutely retry indefinitely to delete the checkpoint file though. Once that's gone, either the cleanup thread will delete the directory or some thread will get the task re-assigned and delete the leftover store to recreate from scratch
@abbccdda Do we know what's the exception thrown here? I think if it is a transient error we should retry closing until succeed, OR if it is fatal we should throw it all the way up to thread as fatal and stop the thread (or even stop the instance based on how we want to proceed with KIP-572). |
Exception was due to non-empty directory:
|
I'm looking into it, and feel this is some bug from the directory deletion utility itself, as we are not failing every time. |
Do we have the guarantee that the file is not being accessed by JVM when deleting it? |
d210dc5
to
1904f24
Compare
Thinking about that a bit more, the
I think we should take a deeper look at the call trace of 4) and see if that's not the case and the file locking is actually not protecting us. |
@guozhangwang I think there actually is a race condition there as we first unlock the directory, and then try to delete it -- another thread may be spinning trying to obtain this lock and get it before the directory deletion |
Can we just delete the checkpoint file (and state) and then unlock the directory? We can attempt to also delete it after that, but if we fail just move on, as it's likely another thread came in and will reinitialize/restore the stores from scratch. If we fail for some other reason then thats also fine, the cleanup thread will remove the empty directory |
I updated the JIRA with the race condition reasoning. My current thinking was that we should be more careful in the deletion of states by making sure we hold the lock during the deletion and release it afterwards. This means we may potentially have a bunch of empty state directories, but not stepping on each other's toes. Note that some OS doesn't allow the deletion of a file which is locked by the current JVM thread, so it is slightly difficult. The other option is to have a directory like |
@ableegoldman Yeah when I discussed with @abbccdda offline on Friday we also found that, but at that time we still need to confirm from the logs that it was indeed the issue. @abbccdda From the trace you added on the JIRA ticket it seems our theory is indeed confirmed, thanks! |
8f97d35
to
ab48698
Compare
test this please |
// if EOS is enabled, we wipe out the whole state store for unclean close | ||
// since they are invalid to use anymore | ||
if (!clean && !eosDisabled) { | ||
StateManagerUtil.wipeStateStores(log, stateMgr); | ||
} | ||
|
||
// first close state manager (which is idempotent) then close the record collector (which could throw), | ||
// if the latter throws and we re-close dirty which would close the state manager again. | ||
StateManagerUtil.closeStateManager(log, logPrefix, clean, stateMgr, stateDirectory); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is that okay that we wipe out the directory already and then closing state manager, in which we would flush the stores and then close them? Would any exception be thrown?
15b7258
to
a6ec2ba
Compare
test this please |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Made a pass over the updated PR, overall looks good.
Could you also add an integration test that involves with the FS indeed to make sure this code path is okay? With mocked modulo unit tests I concern we do not have a good coverage here.
Also could you trigger a system test as well?
// first close state manager (which is idempotent) then close the record collector (which could throw), | ||
// if the latter throws and we re-close dirty which would close the state manager again. | ||
StateManagerUtil.closeStateManager(log, logPrefix, clean, | ||
wipeStateStore, stateMgr, stateDirectory, "active"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: use TaskType.ACTIVE
.
@@ -179,7 +179,7 @@ private void close(final boolean clean) { | |||
} | |||
|
|||
if (state() == State.CLOSING) { | |||
StateManagerUtil.closeStateManager(log, logPrefix, clean, stateMgr, stateDirectory); | |||
StateManagerUtil.closeStateManager(log, logPrefix, clean, false, stateMgr, stateDirectory, "standby"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use TaskType.STANDBY
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the added integration test! Left some more comments.
...ms/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java
Outdated
Show resolved
Hide resolved
...ms/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java
Show resolved
Hide resolved
The system tests also pass: https://jenkins.confluent.io/job/system-test-kafka-branch-builder/3801/ |
test this please |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM. Waiting for green build.
test this please |
The only real test failure: is fixed by last commit. |
This PR tries to reorder the closing behavior by doing the state store cleanup first before releasing the lock.
Committer Checklist (excluded from commit message)