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

ayegorov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 7ae721d  Fix NPE while reordering read-sequence for local-bookie 
ensemble policy
7ae721d is described below

commit 7ae721dd9fa3aac3cfb511616e23747b09875594
Author: Rajan Dhabalia <[email protected]>
AuthorDate: Mon Mar 28 15:54:35 2022 -0700

    Fix NPE while reordering read-sequence for local-bookie ensemble policy
    
    ### Motivation
    
    When Bookie sanity and autoreovery use the same conf file which has flag 
`reorderReadSequenceEnabled=true` then bookie-sanity command throws NPE as 
`LocalBookieEnsemblePlacementPolicy::reorderReadLACSequence` returns null 
writesets which causes the sanity failure.
    
    ```
    00:46:46.202 [BookKeeperClientWorker-OrderedExecutor-11-0] ERROR 
o.a.b.common.util.SafeRunnable       - Unexpected throwable caught
    java.lang.NullPointerException: null
        at 
org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.sendNextRead(PendingReadOp.java:399)
        at 
org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.read(PendingReadOp.java:385)
        at 
org.apache.bookkeeper.client.PendingReadOp.initiate(PendingReadOp.java:529)
        at 
org.apache.bookkeeper.client.LedgerRecoveryOp.doRecoveryRead(LedgerRecoveryOp.java:148)
        at 
org.apache.bookkeeper.client.LedgerRecoveryOp.access$000(LedgerRecoveryOp.java:37)
        at 
org.apache.bookkeeper.client.LedgerRecoveryOp$1.readLastConfirmedDataComplete(LedgerRecoveryOp.java:109)
        at 
org.apache.bookkeeper.client.ReadLastConfirmedOp.readEntryComplete(ReadLastConfirmedOp.java:135)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1829)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1910)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV3Response(PerChannelBookieClient.java:1885)
        at 
org.apache.bookkeeper.proto.PerChannelBookieClient$3.safeRun(PerChannelBookieClient.java:1446)
        at 
org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
    
    ```
    
    ### Modification
    Fix NPE for local ensemble policy while reading entry with 
`reorderReadSequenceEnabled` flag enabled.
    
    Reviewers: Andrey Yegorov <None>, Enrico Olivelli <[email protected]>, 
Nicolò Boschi <[email protected]>
    
    This closes #3127 from rdhabalia/repl_seq
---
 .../bookie/LocalBookieEnsemblePlacementPolicy.java     |  4 ++--
 .../apache/bookkeeper/client/TestSpeculativeRead.java  | 18 ++++++++++++++++++
 2 files changed, 20 insertions(+), 2 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
index 5440497..7c949cb 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
@@ -99,7 +99,7 @@ public class LocalBookieEnsemblePlacementPolicy implements 
EnsemblePlacementPoli
             List<BookieId> ensemble,
             BookiesHealthInfo bookiesHealthInfo,
             DistributionSchedule.WriteSet writeSet) {
-        return null;
+        return writeSet;
     }
 
     @Override
@@ -107,7 +107,7 @@ public class LocalBookieEnsemblePlacementPolicy implements 
EnsemblePlacementPoli
             List<BookieId> ensemble,
             BookiesHealthInfo bookiesHealthInfo,
             DistributionSchedule.WriteSet writeSet) {
-        return null;
+        return writeSet;
     }
 
     @Override
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
index c0b8e2a..174fe44 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java
@@ -24,6 +24,7 @@ import static 
org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
 import static 
org.apache.bookkeeper.client.BookKeeperClientStats.SPECULATIVE_READ_COUNT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
@@ -35,6 +36,7 @@ import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.bookkeeper.bookie.LocalBookieEnsemblePlacementPolicy;
 import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -381,4 +383,20 @@ public class TestSpeculativeRead extends 
BookKeeperClusterTestCase {
             bkspec.close();
         }
     }
+
+    @Test
+    public void testSequenceReadLocalEnsemble() throws Exception {
+        ClientConfiguration conf = new ClientConfiguration()
+                .setSpeculativeReadTimeout(1000)
+                
.setEnsemblePlacementPolicy(LocalBookieEnsemblePlacementPolicy.class)
+                .setReorderReadSequenceEnabled(true)
+                .setEnsemblePlacementPolicySlowBookies(true)
+                .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+        BookKeeper bkspec = new BookKeeperTestClient(conf, new 
TestStatsProvider());
+        LedgerHandle l = bkspec.createLedger(1, 1, digestType, passwd);
+        List<BookieId> ensemble = 
l.getLedgerMetadata().getAllEnsembles().get(0L);
+        PendingReadOp op = new PendingReadOp(l, bkspec.getClientCtx(), 0, 5, 
false);
+        PendingReadOp.LedgerEntryRequest req0 = op.new 
SequenceReadRequest(ensemble, l.getId(), 0);
+        assertNotNull(req0.writeSet);
+    }
 }

Reply via email to