Skip to content
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

Fixed issue where entries could not be read due to mismatch between the ensemble on metadata and the actual written bookies. #4194

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

shustsud
Copy link
Contributor

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.

STEP1:
Write entry0,1,2,3 to Bookies, but the entries are in the following state.
 - entry: 0(Bookie0, Bookie1) -> Waiting for successful write to Bookie0.
 - entry: 1(Bookie1, Bookie2) -> Writing to Bookie1,2 was successful, but its completion is pending.
 - entry: 2(Bookie2, Bookie3) -> Writing to Bookie2,3 was successful, but its completion is pending.
 - entry: 3(Bookie3, Bookie0) -> Waiting for successful write to Bookie0.

STEP2:
Write entry4,5,6 to Bookies, but the entries are in the following state.
 - entry: 0(Bookie0, Bookie1) -> Waiting for successful write to Bookie0.
 - entry: 1(Bookie1, Bookie2) -> Writing to Bookie1,2 was successful, but its completion is pending.
 - entry: 2(Bookie2, Bookie3) -> Writing to Bookie2,3 was successful, but its completion is pending.
 - entry: 3(Bookie3, Bookie0) -> Waiting for successful write to Bookie0.
 - entry: 4(Bookie0, Bookie1) -> Waiting for successful write to Bookie0.
 - entry: 5(Bookie1, Bookie2) -> Failed to write to Bookie2.
 - entry: 6(Bookie2, Bookie3) -> Failed to write to Bookie2,3.

Writing of entry5,6 failed, so ensemble replacement is started.

STEP3:
Write entry0,3,4 succeeded, but its completion is pending because ensemble is in the process of being replaced.

entries are in the following state.
 - entry: 0(Bookie0, Bookie1) -> Writing to Bookie0,1 was successful, but its completion is pending.
 - entry: 1(Bookie1, Bookie2) -> Writing to Bookie1,2 was successful, but its completion is pending.
 - entry: 2(Bookie2, Bookie3) -> Writing to Bookie2,3 was successful, but its completion is pending.
 - entry: 3(Bookie3, Bookie0) -> Writing to Bookie3,0 was successful, but its completion is pending.
 - entry: 4(Bookie0, Bookie1) -> Writing to Bookie0,1 was successful, but its completion is pending.
 - entry: 5(Bookie1, Bookie2) -> Failed to write to Bookie2.
 - entry: 6(Bookie2, Bookie3) -> Failed to write to Bookie2,3.

STEP4:
The ensemble replacement is completed and LedgerHandle#unsetSuccessAndSendWriteRequest is called.
 https://github.com/apache/bookkeeper/blob/13e7efaa971cd3613b065ac50836c5ee98985d13/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2007-L2013

Entry0 is processed first, but since entry0 is not written to Bookie2,3, LedgerHandle#sendAddSuccessCallbacks is called.
 https://github.com/apache/bookkeeper/blob/234b817cdb4e054887ffd5e42eaed25dc02daf63/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L203-L206

Entry1,2,3 must be written again to Bookies after replacing ensemble, but writing of entry0,1,2,3,4 is completed at the above timing.
 https://github.com/apache/bookkeeper/blob/13e7efaa971cd3613b065ac50836c5ee98985d13/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L1814-L1839

As a result, entry1,2,3 is mismatched between the ensemble on metadata and the actual written bookies. 
  • Running the test added in this PR with the unfixed source reproduces the issue.

Changes

  • Change the timing of when LedgerHandle#sendAddSuccessCallbacks is called.

…he ensemble on metadata and the actual written bookies.
@eolivelli
Copy link
Contributor

@jvrao @merlimat @nicoloboschi @dlg99 @hangc0276 @codelipenghui you may want to take a look

@lhotari
Copy link
Member

lhotari commented Feb 8, 2024

@shustsud This fix seems to be related to PR #4171 by @graysonzeng . Is there any overlap?

@shustsud
Copy link
Contributor Author

shustsud commented Feb 9, 2024

@lhotari
I think this fix is not related to PR #4171.
This issue also occurs when LedgerHandle#sendAddSuccessCallbacks is called singly.

I confirmed PR #4171 and knew that threads other than BookKeeperClientWorker-OrderedExecutor call LedgerHandle#sendAddSuccessCallbacks.
I think the current fix is not sufficient, so I will fix this PR.

Thanks.

@lhotari
Copy link
Member

lhotari commented Feb 9, 2024

I think this fix is not related to PR #4171.

@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?

@shustsud
Copy link
Contributor Author

@lhotari
I see.

The cause of this issue is the following call to LedgerHandle#sendAddSuccessCallbacks.

if (!lh.distributionSchedule.hasEntry(entryId, bookieIndex)) {
lh.sendAddSuccessCallbacks();
return;
}

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.

for (PendingAddOp pendingAddOp : pendingAddOps) {
for (Integer bookieIndex: bookies) {
pendingAddOp.unsetSuccessAndSendWriteRequest(ensemble, bookieIndex);
}
}

In this fix, changing the timing of setting changingEnsemble to false may solve the issue.

@shustsud
Copy link
Contributor Author

I confirmed PR #4171 and knew that threads other than BookKeeperClientWorker-OrderedExecutor call LedgerHandle#sendAddSuccessCallbacks.
I think the current fix is not sufficient, so I will fix this PR.

Fixed this PR.
f47db3d

@lhotari
Copy link
Member

lhotari commented Feb 13, 2024

Great work @shustsud . I hope @graysonzeng could also review this PR.
Perhaps not directly related to this issue, but what's your thoughts on the logic in sendAddSuccessCallbacks:

// Check if it is the next entry in the sequence.
if (pendingAddOp.entryId != 0 && pendingAddOp.entryId != pendingAddsSequenceHead + 1) {
if (LOG.isDebugEnabled()) {
LOG.debug("Head of the queue entryId: {} is not the expected value: {}", pendingAddOp.entryId,
pendingAddsSequenceHead + 1);
}
return;
}

Why does that logic exist in the first place? Are entry ids guaranteed to be continuous?

@shustsud
Copy link
Contributor Author

@lhotari

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.

ensembles={
  0=[Bookie1:<port>, Bookie2:<port>, Bookie3:<port>],
  100=[Bookie1:<port>, Bookie2:<port>, Bookie4:<port>],
  1000=[Bookie5:<port>, Bookie6:<port>, Bookie4:<port>]
}

This check may exist because entry ids must have been successfully written sequentially.

@shustsud
Copy link
Contributor Author

@jvrao @merlimat @nicoloboschi @dlg99 @hangc0276 @codelipenghui
Could you review this PR?
This PR is not directly related to the #4171, so I think it can be merged once the review is complete.

@graysonzeng
Copy link
Contributor

graysonzeng commented Feb 22, 2024

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

@shustsud
Copy link
Contributor Author

@graysonzeng

If this PR is merged, I think the repair of #4171 may be improved.

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?
f47db3d#diff-1d893bb31553b5e1f55c8301d04ae15f38e0d35f531f9dd22475128b7972ddf9R2004

@graysonzeng
Copy link
Contributor

@graysonzeng

If this PR is merged, I think the repair of #4171 may be improved.

Is the issue with #4171 that BookKeeperClientWorker-OrderedExecutor thread and pulsar-io thread called LedgerHandle#sendAddSuccessCallbacks at the same time?

Yes.

Even if this PR is merged, will BookKeeperClientWorker-OrderedExecutor thread still call LedgerHandle#sendAddSuccessCallbacks, but will #4171 improve? f47db3d#diff-1d893bb31553b5e1f55c8301d04ae15f38e0d35f531f9dd22475128b7972ddf9R2004

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?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants