[fix][bk] Fix NPE in IsolatedBookieEnsemblePlacementPolicy when policy class does not match#25825
Conversation
…y class does not match MutablePair was initialized without default values, leaving left/right as null when the ensemble placement policy class name did not match. This caused a NullPointerException when getLeft().contains() was called. Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
| } | ||
| for (String group : allGroups) { | ||
| Set<String> bookiesInGroup = allGroupsBookieMapping.get(group).keySet(); | ||
| if (!primaryIsolationGroup.contains(group)) { |
There was a problem hiding this comment.
This is the problematic behavior for the non-matching policyClass case. After this PR, getIsolationGroup() returns empty primary/secondary groups when the metadata policyClass is not
IsolatedBookieEnsemblePlacementPolicy. With primaryIsolationGroup empty, this condition is true for every configured group, so all grouped bookies are added to excludedBookies below. That avoids the NPE but can
still make replaceBookie fail with BKNotEnoughBookiesException. A non-matching policyClass should be ignored or fall back to defaultIsolationGroups, not be treated as an empty isolation group.
There was a problem hiding this comment.
@Shawyeok Please check this. I wonder if there's a test case that would cover the explained scenario?
Fixes NPE in
IsolatedBookieEnsemblePlacementPolicyMotivation
A
NullPointerExceptionoccurs ingetExcludedBookiesWithIsolationGroupswhengetIsolationGroup()returns aPairwhoseleftvalue isnull. This happens when theEnsemblePlacementPolicyConfigcarries a policy class name that does not equalIsolatedBookieEnsemblePlacementPolicy— in that case theifblock that callspair.setLeft()/pair.setRight()is skipped entirely, leaving both sides of theMutablePairasnull.This bug was observed during the upgrade from the 2.8 branch to the 3.0 branch. The same defect may also exist in master.
Full stack trace from production:
This error causes topics to time out during loading. Since there is no automatic recovery path, the only way to restore service is by restarting the broker.
Modifications
Two complementary fixes:
getIsolationGroup(): InitializeMutablePairwith empty sets as defaults so the returnedPairalways has non-null values, and remove the now-redundantelsebranches that were explicitly setting the same empty sets.getExcludedBookiesWithIsolationGroups()(defensive programming): ResolveprimaryIsolationGroupandsecondaryIsolationGroupat the top of the method usingObjectUtils.getIfNull, before any use. This eliminates both the original NPE at the early-returncontainscheck and a second latent NPE at lines 218–219 where the pair values were previously assigned and dereferenced without null guards.Verifying this change
This change added tests and can be verified as follows:
testReplaceBookieWithNonMatchingPolicyClassShouldNotThrowNPEto reproduce the exact production failure: callingreplaceBookiewith custom metadata whoseEnsemblePlacementPolicyConfigpolicy class does not matchIsolatedBookieEnsemblePlacementPolicy, which previously caused the NPE.Does this pull request potentially affect one of the following parts: