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

Do not reuse the Failed OpAddEntry object which lead bundle unloading timeout. #12993

Conversation

codelipenghui
Copy link
Contributor

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:

if (!STATE_UPDATER.compareAndSet(OpAddEntry.this, State.INITIATED, State.COMPLETED)) {

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)

### 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.
@codelipenghui codelipenghui self-assigned this Nov 26, 2021
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Nov 26, 2021
@codelipenghui codelipenghui changed the title Do not reuse the Failed OpAddEntry object. Do not reuse the Failed OpAddEntry object which lead bundle unload timeout. Nov 26, 2021
@codelipenghui codelipenghui changed the title Do not reuse the Failed OpAddEntry object which lead bundle unload timeout. Do not reuse the Failed OpAddEntry object which lead bundle unloading timeout. Nov 26, 2021
@codelipenghui codelipenghui added this to the 2.10.0 milestone Nov 26, 2021
Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice Catch!

Copy link
Contributor

@eolivelli eolivelli left a 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 ?

@eolivelli
Copy link
Contributor

@nicoloboschi PTAL

@codelipenghui
Copy link
Contributor Author

is this problem present in 2.9.0 as well ?

@eolivelli Yes

Copy link
Contributor

@nicoloboschi nicoloboschi left a 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

@codelipenghui codelipenghui merged commit 3e3622c into apache:master Nov 27, 2021
@codelipenghui codelipenghui deleted the penghui/fix-race-condition-for-OpAddEntry branch November 27, 2021 01:21
codelipenghui added a commit that referenced this pull request Nov 27, 2021
### 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 codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Nov 27, 2021
@shoothzj
Copy link
Member

@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();
Copy link
Member

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.

Copy link
Member

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.

Copy link
Contributor Author

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.

Copy link
Member

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.

eolivelli pushed a commit to eolivelli/pulsar that referenced this pull request Nov 29, 2021
### 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.
@Jason918
Copy link
Contributor

Jason918 commented Dec 1, 2021

@codelipenghui what's the relationship between this and #12396 . Does this mean we don't need it anymore?

NO relationship. It's OpReadEntry in #12396.

lhotari pushed a commit to datastax/pulsar that referenced this pull request Dec 3, 2021
### 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)
@michaeljmarshall
Copy link
Member

@codelipenghui - when you ran into this error, did you have AddEntryTimeoutSeconds configured to a non zero value?

codelipenghui added a commit that referenced this pull request Dec 11, 2021
### 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 codelipenghui added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Dec 11, 2021
eolivelli pushed a commit that referenced this pull request Dec 15, 2021
### 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)
@eolivelli eolivelli added cherry-picked/branch-2.9 Archived: 2.9 is end of life release/2.9.1 and removed release/2.9.2 labels Dec 15, 2021
fxbing pushed a commit to fxbing/pulsar that referenced this pull request Dec 19, 2021
### 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.
@codelipenghui codelipenghui restored the penghui/fix-race-condition-for-OpAddEntry branch May 17, 2022 01:18
@codelipenghui codelipenghui deleted the penghui/fix-race-condition-for-OpAddEntry branch May 17, 2022 01:29
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker cherry-picked/branch-2.7 Archived: 2.7 is end of life cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.9 Archived: 2.9 is end of life doc-not-needed Your PR changes do not impact docs release/2.7.4 release/2.8.2 release/2.9.1
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

8 participants