-
Notifications
You must be signed in to change notification settings - Fork 963
Fixed issue where entries could not be read due to mismatch between the ensemble on metadata and the actual written bookies. #4194
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
base: master
Are you sure you want to change the base?
Conversation
|
@jvrao @merlimat @nicoloboschi @dlg99 @hangc0276 @codelipenghui you may want to take a look |
|
@shustsud This fix seems to be related to PR #4171 by @graysonzeng . Is there any overlap? |
|
@lhotari I confirmed PR #4171 and knew that threads other than BookKeeperClientWorker-OrderedExecutor call LedgerHandle#sendAddSuccessCallbacks. Thanks. |
@shustsud @graysonzeng The reason why I thought it is related is that the bug that #4171 is attempting to fix seems to happen when the ensemble is replaced. Isn't the scenario common for both cases? |
|
@lhotari The cause of this issue is the following call to LedgerHandle#sendAddSuccessCallbacks. bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java Lines 203 to 206 in 234b817
If LedgerHandle#sendAddSuccessCallbacks is called at this timing, it may cause other entries that need to be written again to the Bookies to complete. Therefore, LedgerHandle#sendAddSuccessCallbacks must be called after all entries have been written again to Bookies. Also, while writing again to Bookies, LedgerHandle#sendAddSuccessCallbacks may be called in other threads. This point needs to be fixed and may overlap with the PR #4171. bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java Lines 2008 to 2012 in 13e7efa
In this fix, changing the timing of setting changingEnsemble to false may solve the issue. bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java Line 1997 in 13e7efa
|
|
Great work @shustsud . I hope @graysonzeng could also review this PR. bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java Lines 1822 to 1829 in 13e7efa
Why does that logic exist in the first place? Are entry ids guaranteed to be continuous? |
Sorry, I don't know if entry ids are guaranteed to be continuous. But this check looks like its premise. Also, ensemble on metadata would probably be structured as follows. This check may exist because entry ids must have been successfully written sequentially. |
|
@jvrao @merlimat @nicoloboschi @dlg99 @hangc0276 @codelipenghui |
|
well done @shustsud . I think that although both problems occur when ensemble is replaced, this PR fixes the calling sequence of unsetSuccessAndSendWriteRequest and sendAddSuccessCallbacks, and the #4171 tries to fixes the race conditions between unsetSuccessAndSendWriteRequest and writeComplete. If this PR is merged, I think the repair of #4171 may be improved. What do you think @lhotari |
Is the issue with #4171 that BookKeeperClientWorker-OrderedExecutor thread and pulsar-io thread called LedgerHandle#sendAddSuccessCallbacks at the same time? Even if this PR is merged, will BookKeeperClientWorker-OrderedExecutor thread still call LedgerHandle#sendAddSuccessCallbacks, but will #4171 improve? |
Yes.
I mean maybe we can use a simpler way to fix #4171 after this PR, for example add synchronized to the sendAddSuccessCallbacks may not lead to deadlock anymore? |
f47db3d to
63d3ccd
Compare
| ensembleChangeLoop(origEnsemble, toReplace); | ||
| } | ||
|
|
||
| if (newEnsemble != null) { // unsetSuccess outside of lock |
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.
There's a comment "unsetSuccess outside of lock". Is there a potential for a deadlock when "unsetSuccessAndSendWriteRequest" is called in a different location?
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.
solution was added in #1857
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.
Sorry for the late reply.
I believe the current code should not cause a deadlock, but I agree that calling unsetSuccessAndSendWriteRequest inside the lock might not be ideal.
I'll check whether it’s possible to safely move the unsetSuccessAndSendWriteRequest call outside the lock.
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.
@lhotari
I've updated the code to move the unsetSuccessAndSendWriteRequest() call outside the metadataLock, as I mentioned before.
Please take another look when you have a chance.
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.
Hi @lhotari, when you have a moment, could you please take another look at this PR?
Let me know if there's anything else you'd like me to adjust. Thanks!
|
Although this PR was originally submitted last year, I’ve tested it with the latest codebase and confirmed that the issue still persists. This PR fixes that issue. |
…he ensemble on metadata and the actual written bookies.
9cdf7e2 to
1fdb1c6
Compare
Related Issue: #4097
Motivation
If write to Bookies succeeds during replacing ensemble, there may be a mismatch between the ensemble on metadata and the actual written bookies.
This issue occurs in the following scenario.
Changes
LedgerHandle#sendAddSuccessCallbacksis called.