-
Notifications
You must be signed in to change notification settings - Fork 902
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
Missing ledgers when inducing network packet loss in Bookkeeper 4.15 #3466
Comments
A couple of questions:
|
another question:
|
Thanks @eolivelli, reposting the answer in Slack for better visibility: https://pravega-io.slack.com/archives/C0151LSM46L/p1662115749173509
The cluster is not in state anymore, but if the Bookkeeper Client gets a
I don't know if the fact that this request queries
In this experiment, the configuration is
The Bookkeeper service is configured to keep 4 Bookies.
Pravega rolls over ledger when they reach their max size (1GB default) or when there is some container recovery (due to fencing). So, this particular ledger was created but contained 0 entries at the moment when the issue happened. It would have contained many more in case the issue wouldn't had appeared.
Could be, according to this log, the ledger should be created using 2 Bookies: |
Not sure if it has something to do with this pr: ISSUE 3220-Autorecovery does not process underreplicated empty ledgers |
We found the same issue on the Pulsar side. apache/pulsar#17520
|
@eolivelli @zhaohaidao @hangc0276 We have done another set of failure injection tests to narrow down the issue. These are the results:
With this, I think we can narrow down the regression to be in the Bookkeeper client only in Bookkeeper 4.15. Hope this helps us to shortlist the set of PRs that can be related to this problem. |
I have done some testing on master branch. After re-reading again the logs in this issue it seems that:
|
I did a "git diff" between |
@RaulGracia do you have "auto-recovery" running into your cluster? |
@zhaohaidao your idea is interesting. The fact that the client tries to read from entry 0 even if the ledger is empty means that something CLOSED the ledger and set the LAC to 0 (and not -1). I am saying that LAC should be at least 0 because of these logs: |
it is also interesting the the Writer tried to write entry 0
but it wasn't able to even send the write request. |
@RaulGracia
There have two things I noticed:
And looks like in the Pravega, the ledger handler closes again so the error happens again. (step 2 closed and step 3 closed again)
I saw in the previous, you are using 4.14.1, I found there has a change related to this which was released in 4.14.3. Could you please also run the test against with the 4.14.1 and 4.14.3? |
@zymap thanks a lot for your analysis. Yes, you are right, we are using 4.14.2 concretely, as the last commit we have from mainline is . And we do not have that PR #2794 when Bookkeeper works well with our fault injection tests. Maybe we could try another test with latest 4.14 commit, to see if the same issue is reproduced. |
@RaulGracia is there any chance that lossy network caused some kind of split brain or data inconsistency on ZK side? |
@dlg99 thanks Andrey. While this situation can indeed happen (specially inducing failure injection at the network level), my understanding is that that would happen to any version of Bookkeeper, right? However, our observation is that when executing this kind of tests, only Pravega using Bookkeeper 4.15 client gets into this situation where ledgers supposed to be there are not; when using Bookkeeper 4.14.2, the issue does not reproduce. Note that we have done this test many times with Bookkeeper 4.14.2, while the few times we executed that with Bookkeeper 4.15 client we always got into that situation. We are currently executing the same test using Bookkeeper client 4.14.3, to validate the hypothesis of @zymap #3466 (comment) |
@RaulGracia My thought process was along the lines of updated ZK client making the problem easier to repro if that's ZK related. |
FYI @zymap @eolivelli @dlg99 we have performed the same test again with Bookkeeper 4.14.3 client as suggested by @zymap, but we have not been able to reproduce the issue this time. So, at the moment, with the evidences we have we cannot say that #2794 is the root cause of the problem. |
@RaulGracia I am trying to wrap my head around how #2794 could result in a missing ledger handle. One thing off with the PR is that it calls blocking lh.close() from the callback running on the main threadpool. AFAICT there are no deadlocks because whatever else happens runs on the ZK. Zk can be quite slow because of injected network errors. Can the lost ledger handle might be caused by the way Pravega handles errors in ledger open? |
Thanks a lot @dlg99, I can certainly review #3513. But after the latest experiments we did with Bookkeeper 4.14.3, which includes #2794, we couldn't reproduce the issue, so I'm not sure that that PR is contributing to the issue described here. In any case, it is independent from the value that #3513 brings by itself.
The point is that the very same Pravega code has no issues using Bookkeeper 4.14.2 client and injecting network failures, and we consistently get issues like the one describe here when upgrading to Bookkeeper 4.15 client. For this reason, I thought that the problem may be related to some change in the Bookkeeper client within I think that to discard that the issue is present in the whole |
I am working on this. The. docker images contain a build of Pravega compatible with BK 4.15 (thanks to @RaulGracia ) but with the BK client compiled at different points in the GIT history of 4.15.0. This way we will see which commit broke the compatibility. It will take much time because those system tests take at least 24h before failing |
To wrap up this issue: We have done fault injection tests of Pravega and several Bookkeeper client versions and we have not been able to reproduce the original issue (see details here). We have also upgraded Pravega |
Hi all, I think we could have another reproduction of this problem, this time using Bookkeeper 4.14.2 client/server (therefore, the issue reported here may be present in other versions of Bookkeeper, not only in Bookkeeper 4.15). This is the information I have been able to recollect so far (the missing ledger this time is
Once the issue was reported, I tried to dig a little more with the Pravega and Bookkeeper CLIs:
How inducing delays at the network level in the pods leads very rarely to this situation is something that I still don't know. I'm also attaching the relevant log lines from Pravega Segment Store for ledger |
BUG REPORT
Describe the bug
During our tests of Pravega (https://cncf.pravega.io/), we have detected a regression after upgrading to Bookkeeper 4.15 (pravega/pravega#6676). The regression is consistently reproducible when doing a fault injection test (30% network packet loss). Note that this kind of test consistently pass using Bookkeeper 4.14, for which we believe that is a regression or a change in the behavior of the new Bookkeeper release.
A reproduction of this issue goes as follows:
3246
:2022-07-20T14:48:11
:L3246
):In my understanding, from the Pravega point of view ledger
3246
is created and not explicitly deleted by Pravega itself, which means that there has been a data loss, most likely induced by some corner case exposed via network packet loss. In normal conditions, where there are no induced packet loss, things work fine. But it does not mean that, in a real cluster, a similar network failure could happen and the same issue could be reproduced. Again, we have been doing fault injection tests with Bookkeeper 4.14 and probably previous versions as well (e.g., Bookkeeper 4.11), and this is the first time we encounter this behavior.To Reproduce
Steps to reproduce the behavior:
No Such Ledger Exception
in some containers using Bookkeeper 4.15 (with Bookkeeper 4.14, the system is able to work under such conditions)Expected behavior
Bookkeeper 4.15 should offer a similar behavior in how ledgers are managed compared to Bookkeeper 4.14 (and previous versions).
Screenshots
n/a
Additional context
At the moment, there are 2 changes that could be related to this issue (but I cannot confirm that they actually cause the problem):
The text was updated successfully, but these errors were encountered: