-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Do not reuse the Failed OpAddEntry object which lead bundle unloading timeout. #12993
Do not reuse the Failed OpAddEntry object which lead bundle unloading timeout. #12993
Conversation
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` apache#12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by apache#11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete.
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.
Nice Catch!
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.
lgtm
is this problem present in 2.9.0 as well ?
@nicoloboschi PTAL |
@eolivelli Yes |
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.
It makes sense. If we don't recycle the object we can actually skip updating the state to CLOSED
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` #12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by #11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete. (cherry picked from commit 3e3622c)
@codelipenghui what's the relationship between this and #12396 . Does this mean we don't need it anymore? |
@@ -1678,7 +1678,6 @@ public ManagedLedgerInterceptor getManagedLedgerInterceptor() { | |||
void clearPendingAddEntries(ManagedLedgerException e) { | |||
while (!pendingAddEntries.isEmpty()) { | |||
OpAddEntry op = pendingAddEntries.poll(); | |||
op.close(); |
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.
@codelipenghui - can you explain why you removed this line? Without it, when the addComplete
callback is run, the method won't exit early because the state for the OpAddEntry
will still be INITIATED
.
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.
After thinking about this a little more, I think we'll need a more nuanced solution than just re-adding op.close();
on this line. I have to sign off for now. I'll follow up with more commentary tomorrow.
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.
@michaeljmarshall we will not reuse the failed OpAddEntry, so we don't need to update the state of the OpAddEntry here.
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.
@codelipenghui - okay. My concern was that if we call clearPendingEntries
before the BK callback completes, we would want the state to be closed so that the callback doesn't run further.
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` apache#12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by apache#11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete.
NO relationship. It's OpReadEntry in #12396. |
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` apache#12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by apache#11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete. (cherry picked from commit 3e3622c) (cherry picked from commit 628212e)
@codelipenghui - when you ran into this error, did you have |
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` #12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by #11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete. (cherry picked from commit 3e3622c)
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` #12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by #11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete. (cherry picked from commit 3e3622c)
### Motivation There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is `ManagedLedgerImple.clearPendingAddEntries`. But, if the OpAddEntry be completed more than once, we will get NPE: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.lambda$handleAddFailure$0(OpAddEntry.java:291) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` Another one: ``` java.lang.NullPointerException: null at org.apache.bookkeeper.mledger.impl.OpAddEntry.addComplete(OpAddEntry.java:153) ~[org.apache.pulsar-managed-ledger-2.8.1.jar:2.8.1] at org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.client.LedgerHandle$5.safeRun(LedgerHandle.java:574) ~[org.apache.bookkeeper-bookkeeper-server-4.14.2-2.jar:4.14.2-2] at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.2-2.jar:4.14.2-2] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_302] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_302] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.68.Final.jar:4.1.68.Final] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_302] ``` apache#12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused. And when we get the add entry complete callback from the bk client, we will reach here: https://github.com/apache/pulsar/blob/5dbb7d25849f3a037aa522b5d0767801aa0a5096/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java#L147 But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case. ### Modification So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by apache#11737. We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete.
Motivation
There are 2 ways to complete the OpAddEntry with exception, one is the bk client callback and another one is
ManagedLedgerImple.clearPendingAddEntries
.But, if the OpAddEntry be completed more than once, we will get NPE:
Another one:
#12364 tries to fix the NPE by change the state of the OpAddEntry to CLOSED, but the OpAddEntry still will be recyled and be reused.
And when we get the add entry complete callback from the bk client, we will reach here:
pulsar/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
Line 147 in 5dbb7d2
But it might to recycle the OpAddEntry already be reused by other entry add operation. It might lead to lost data for this case.
Modification
So the fix is do not recycle the OpAddEntry when call OpAddEntry.failed() which is introduced by #11737.
We should contain this fix in 2.8.2, we have encounter serious problem when unloading the bundles, the topic close will be blocked and never complete
because of LedgerHandle.errorOutPendingAdds() https://github.com/apache/bookkeeper/blob/87579b0a9f18833ee41fcae37582bb68606d68e7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L574 get NPE but can't throw out and the ledger close callback will never complete.
Need to update docs?
doc-required
(If you need help on updating docs, create a doc issue)
no-need-doc
(Please explain why)
doc
(If this PR contains doc changes)