Fix NPE while reordering read-sequence for local-bookie ensemble policy
authorRajan Dhabalia <rdhabalia@apache.org>
Mon, 28 Mar 2022 22:54:35 +0000 (15:54 -0700)
committerGitHub <noreply@github.com>
Mon, 28 Mar 2022 22:54:35 +0000 (15:54 -0700)
commit7ae721dd9fa3aac3cfb511616e23747b09875594
tree17a54f954c0284ab7665888e1046ff3fa7a0ab4c
parentb4d98e8c5afe4b9d6bcf6100dd943c003bd1ca89
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 <eolivelli@gmail.com>, Nicolò Boschi <boschi1997@gmail.com>

This closes #3127 from rdhabalia/repl_seq
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java
bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java