-
Notifications
You must be signed in to change notification settings - Fork 892
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
Deferred failure handling can cause data loss #1591
Comments
Also, we should only allow a single ensemble change at a time per ledger. The case where you need more than one is rare enough that people can live with the latency hit. It would solve the scenario in the test case posted, though it wouldn't solve the general case (2:2:1 will still have this type of failure). |
Excellent catch Ivan. Thanks.
I think we should restrict one outstanding ensemble change in all cases.
Which is clean.
…On Thu, Aug 9, 2018 at 11:31 AM, Ivan Kelly ***@***.***> wrote:
Also, we should only allow a single ensemble change at a time per ledger.
The case where you need more than one is rare enough that people can live
with the latency hit. It would solve the scenario in the test case posted,
though it wouldn't solve the general case (2:2:1 will still have this type
of failure).
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#1591 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AAChrvhC4p-7Cs46Z9EhkXZ7CbGyZAsdks5uPIAYgaJpZM4V11eR>
.
--
Jvrao
---
First they ignore you, then they laugh at you, then they fight you, then
you win. - Mahatma Gandhi
|
I think a quick fix is handleDelayedWriteBookieFailure should increment |
I will send a patch asap. |
@sijie exactly, the blockAddCompletions stuff should fix this. |
@jvrao just to reemphasize, this restriction, while simplifying things won't solve this problem if adds can complete while the change is happening. The 2:2:1 case I was talking about is.
So, we should absolutely make it so a single ensemble change occurs at a time (it should be easier now with CompletableFuture and stuff. This wasn't available last time this had a major revisit). But the fix should be blocking add completions while ensemble change is in progress. If, by the time I start tomorrow you haven't a started on a patch, i'll submit something. |
For now the fix is to call handleBookieFailure()
Until @ivankelly 's immutable local metadata comes in. The original intent of this change |
For completeness, the same scenario with immutable metadata would be. Two bookie ensemble, b1,b2, ensemble is 2:2:1
So e1 is only on b2, which if the ensemble is 1: b1, b3, can never be read back by a client. Again the source of the issue is allowing add acknowledgement while changing ensemble. |
With immutable this should work quite well. There's 2 cases, a) there's more bookies available, b) there's no more bookies available. With a) we block completions while replacing. |
@jvrao fix looks good btw 👍 |
How does that protect the scenario you mentioned with "Write e1 to b1 (fail for some reason), b2 (success for some reason)" Unless you block writes while the proposed change to the metadata is outstanding? In the current code, local metadata is already changed hence the e1 would have gone on b1,b3 so blocking on the completion is good enough. With local metadata becoming immutable we need to block on the write. |
Descriptions of the changes in this PR: The Original intent of this change is to do a best-effort ensemble change. But this is not possible until the local metadata is completely immutable. Until the feature "Make LedgerMetadata Immutable #610" Is complete we will use handleBookieFailure() to handle delayed writes as regular bookie failures. Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com> Master Issue: #1591 Relate Issue: #1395 Author: JV Jujjuri <vjujjuri@salesforce.com> Author: Ivan Kelly <ivank@apache.org> Reviewers: Ivan Kelly <ivank@apache.org>, Sijie Guo <sijie@apache.org> This closes #1592 from jvrao/datalossbug
Descriptions of the changes in this PR: The Original intent of this change is to do a best-effort ensemble change. But this is not possible until the local metadata is completely immutable. Until the feature "Make LedgerMetadata Immutable #610" Is complete we will use handleBookieFailure() to handle delayed writes as regular bookie failures. Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com> Master Issue: #1591 Relate Issue: #1395 Author: JV Jujjuri <vjujjuri@salesforce.com> Author: Ivan Kelly <ivank@apache.org> Reviewers: Ivan Kelly <ivank@apache.org>, Sijie Guo <sijie@apache.org> This closes #1592 from jvrao/datalossbug (cherry picked from commit 3ab6e92) Signed-off-by: Ivan Kelly <ivank@apache.org>
@jvrao I think the idea is that once you make an ensemble change beginning at entry e, you must defer responding to the client on any entry e' >= e until the metadata update is durable in zookeeper. |
* Avoid releasing sent buffer to early in BookieClient mock If the buffer was sent to more than one bookie with the mock, it would be released after being sent to the first one. Each write should retain a refCount themselves, and then release when done. Author: Ivan Kelly <ivank@apache.org> Reviewers: Sijie Guo <sijie@apache.org> This closes apache#1598 from ivankelly/double-rel-mock * (@bug W-5344681@) Delayed write ensemble change may cause dataloss Descriptions of the changes in this PR: The Original intent of this change is to do a best-effort ensemble change. But this is not possible until the local metadata is completely immutable. Until the feature "Make LedgerMetadata Immutable apache#610" Is complete we will use handleBookieFailure() to handle delayed writes as regular bookie failures. Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com> Master Issue: apache#1591 Relate Issue: apache#1395 Author: JV Jujjuri <vjujjuri@salesforce.com> Author: Ivan Kelly <ivank@apache.org> Reviewers: Ivan Kelly <ivank@apache.org>, Sijie Guo <sijie@apache.org> @Rev Sam Just@ This closes apache#1592 from jvrao/datalossbug
This is fixed by #1592 |
The bookkeeper client has a feature where if you have a ledger with a write quorum(Qw) larger than an ack quorum(Qa), such as 3:3:2, if (Qw-Qa) bookies return an error, after the entry add has completed, the erroring bookie will be replaced in the ensemble in the background.
This can cause data loss.
Consider a 3:3:2 ledger. Assume zookeeper is not accepting writes.
Start ensemble is b1,b2,b3
As each bookie fails, it will be replaced and writes will be acknowledged. Eventually the ensembles will look something like
How ever, this is only local, zookeeper still only has the initial ensemble. So, even though all entries from 4 onwards are acknowledged successfully to the client, if another client comes to read the ledger, they will not see them. If the other client recovers the ledger, the data is lost. TOAB violation.
Here's a test case which triggers the issue:
https://github.com/ivankelly/bookkeeper/blob/15bc251d46d5cd5fcceef130c0046eeacbe446cc/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDeferredFailure.java
The text was updated successfully, but these errors were encountered: