NPE in RackwareEnsemblePlacementPolicyImpl.
Describe the bug
java.lang.NullPointerException: null
at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.isEnsembleAdheringToPlacementPolicy(RackawareEnsemblePlacementPolicyImpl.java:1261) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.replaceBookie(RackawareEnsemblePlacementPolicyImpl.java:681) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.replaceBookie(RackawareEnsemblePlacementPolicy.java:114) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.BookieWatcherImpl.replaceBookie(BookieWatcherImpl.java:278) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.EnsembleUtils.replaceBookiesInEnsemble(EnsembleUtils.java:71) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.LedgerHandle.lambda$ensembleChangeLoop$2(LedgerHandle.java:1908) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.MetadataUpdateLoop.writeLoop(MetadataUpdateLoop.java:122) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.MetadataUpdateLoop.run(MetadataUpdateLoop.java:111) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.LedgerHandle.ensembleChangeLoop(LedgerHandle.java:1927) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.LedgerHandle.handleBookieFailure(LedgerHandle.java:1876) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:360) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.proto.BookieClientImpl$1.safeRun(BookieClientImpl.java:280) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.9.2.jar:4.9.2]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_171]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_171]
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.43.Final.jar:4.1.43.Final]
at java.lang.Thread.run(Thread.java:748) [?:1.8.0_171]
To Reproduce Have no idea to reproduce it.
Additional context in master branch
I'm seeing this bug too. @codelipenghui - did you find a solution to this issue?
Adding some details here as I investigate this NPE.
The NPE is coming from the following code block (the lines have changed a bit since this issue was reported): https://github.com/apache/bookkeeper/blob/e9e9bf92f7c124fb11d8de57fdfec860441ac8a8/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L1026-L1038.
It looks like the underlying issue must be related to a change in the knownBookies map. Bookie nodes are removed from that list when bookies are no longer writable. Here is the only logic that is responsible for removing bookies from the knownBookies map:
https://github.com/apache/bookkeeper/blob/e9e9bf92f7c124fb11d8de57fdfec860441ac8a8/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java#L648-L675
A naive solution is to simply update the isEnsembleAdheringToPlacementPolicy method as follows:
BookieNode bn = knownBookies.get(bookie);
if (bn == null) {
continue;
}
racksInQuorum.add(bn.getNetworkLocation());
I don't know if that is the right solution yet, though.
I think the race comes here:
https://github.com/apache/bookkeeper/blob/e9e9bf92f7c124fb11d8de57fdfec860441ac8a8/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L1934-L1936
Essentially, we get this "currentEnsemble" set, and assume that the set contains only knownBookies. However, we don't acquire the rwLock until later, which means that while the "currentEnsemble" could have known bookies at its creation, it might not by the time we iterate over the set.
@eolivelli - I see in the git history that you have modified some of the code in this section of the code base. Do you have any insight here?
I'm happy to help contribute a fix, but I am new to this section of the code base.
I think this will fix the issue: https://github.com/apache/bookkeeper/pull/3350
I encountered this in the logs of the customer's cluster.
closed by #3350