You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by eo...@apache.org on 2021/12/15 16:33:27 UTC

[pulsar] branch branch-2.9 updated: Do not reuse the Failed OpAddEntry object. (#12993)

This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch branch-2.9
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.9 by this push:
     new be2610d  Do not reuse the Failed OpAddEntry object. (#12993)
be2610d is described below

commit be2610d9445f196a4e0cb840b2608a828f521c73
Author: lipenghui <pe...@apache.org>
AuthorDate: Sat Nov 27 09:21:01 2021 +0800

    Do not reuse the Failed OpAddEntry object. (#12993)
    
    ### 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 3e3622c1d4b8b9169b4ba69081a16a539606b72d)
---
 .../main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java  | 1 -
 .../src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java     | 1 -
 2 files changed, 2 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index 957cfbb..ce897f3 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -1678,7 +1678,6 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
     void clearPendingAddEntries(ManagedLedgerException e) {
         while (!pendingAddEntries.isEmpty()) {
             OpAddEntry op = pendingAddEntries.poll();
-            op.close();
             op.failed(e);
         }
     }
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
index e08b204..4f36d56 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
@@ -137,7 +137,6 @@ public class OpAddEntry extends SafeRunnable implements AddCallback, CloseCallba
             ReferenceCountUtil.release(data);
             cb.addFailed(e, ctx);
             ml.mbean.recordAddEntryError();
-            this.recycle();
         }
     }