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

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

commit 7a797c7e61f7cacac6d70fd5a27a70a65a7c6910
Author: lipenghui <peng...@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 003a40f..1e8e7fd 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
@@ -1511,7 +1511,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 e42ce25..c58f2b9 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
@@ -119,7 +119,6 @@ class OpAddEntry extends SafeRunnable implements 
AddCallback, CloseCallback {
             data.release();
             cb.addFailed(e, ctx);
             ml.mbean.recordAddEntryError();
-            this.recycle();
         }
     }
 

Reply via email to