-
Notifications
You must be signed in to change notification settings - Fork 905
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
fix dead lock issue in ReadOnlyLedgerHandle#handleBookieFailure #4467
base: master
Are you sure you want to change the base?
fix dead lock issue in ReadOnlyLedgerHandle#handleBookieFailure #4467
Conversation
since this is a fatal issue. Could you take a look of this analyse and fix? @shoothzj @dlg99 @merlimat @hangc0276 |
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.
First of all, this is a great catch and definitely looks like a regression.
There are couple of things that require additional attention:
First, the fix seems to be on the the right track but I think it should be done in BookieClientImpl completeAdd(..)
.
bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java
Line 281 in 4ca020a
private void completeAdd(final int rc, |
Second, the fix reverts part of the change #3784 (and follow up fix #3806 )
I'd love to have @merlimat look at this issue - is there a better fix or should we simply revert these perf improvements (that don't have any perf numbers shared on what they improve) for the sake of correctness?
Third, it would be nice to have a test reproing the problem to prevent future regression.
From #4457 stack info.
thread thread I think we should fix the Timepoint4 behavior, it will handle all the pendindAddOps, and try to send the data to new replaced bookie. I think this behavior shouldn't use the same thread as before. |
3639c50
to
bbccd3f
Compare
|
@horizonzy I think your solution is try to fix this dead lock issue itself, but I am still concerned that whether #3784 would bring out other dead lock risk, because it add so many synchronized in PendingAddOp. Actually, I hold 3 kinds of solution idea.
Besides, I think your solution is able to fix current dead lock, but maybe result in other problem. As I observe in both early version bookkeeper and pulsar. In the same ledger, PendingAddOp#sendWriteRequest is always in same thread. If we use different "pulsar-io" thread to send data of the same ledger, It seems break the bookkeeper's design. |
We also faced this bug (version:4.16.5), where two just like org.apache.bookkeeper.client.LedgerHandle#ensembleChangeLoop the stack snippet is below:
|
Fix #4457, apache/pulsar#22986
Motivation
There is a dead lock issue in pulsar-3.0, which is easily to reproduce when some bookies become unavailable. Then it would cause broker unavailable and topic unavailable
The issue occurred when openLedger and do recoverAdd in ReadOnlyLedgerHandle.
We have analysed the stack in issue. The deadlock process can be simplify as:
Therefore, the deadlock occur. And I guess it occur after this pr #3784, because the pr add synchronized in pendingAddOp.
As our known for bookkeeper, handleBookieFailure and the other operations of entry should always run in specific thread for ledgerId. However, we saw that the two pendingAddOps of the same ledger is run in different "pulsar-io" thread. This is not reasonable.
After diving into code, we can find that "pulsar-io" threadPool is used for PerChannelBookieClient connect, and "BookKeeperClientWorker" threadPool is used for handling write or read entry. But if connect fail and throw exception, bookieClient.completeAdd still use "pulsar-io" to execute. That is the root reason of the dead lock issue.
If we can make sure all the pendingAddOps of the same ledger run in the specific executor, the dead lock will not occur.
Changes
If channel is not ready for addEntry, execute completeAddOperation in specific orderedExecutor of ledger, instead of in eventLoopGroup.