Skip to content

Commit 49135b6

Browse files
authored
Exclude defaultrack bookies when enforceMinNumRacksPerWriteQuorum is enabled
Descriptions of the changes in this PR: - enforceMinNumRacksPerWriteQuorum is meant to be used for strict placement policy. So when it is enabled, bookies which belong to default faultzone/rack (because of failure in resolving network location) should be excluded from bookie selection. - add gauge for number of bookies in default faultzone/rack. It will be helpful to create alerts based on this gauge. - add gauge for number of ledgers found not adhering to strict placement policy in Auditor's placement policy check. This gauge will be more helpful in creating alert instead of using monotonously increasing alert. Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Pasha Kuznetsov <None>, Sijie Guo <sijie@apache.org>, Venkateswararao Jujjuri (JV) <None> This closes #1941 from reddycharan/ignoredefaultzoneandfixmetrics
1 parent 31901ff commit 49135b6

File tree

13 files changed

+427
-101
lines changed

13 files changed

+427
-101
lines changed

bookkeeper-common/src/test/java/org/apache/bookkeeper/test/TestStatsProvider.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,7 @@ public Counter getCounter(String name) {
177177
}
178178

179179
public Gauge<? extends Number> getGauge(String name) {
180-
return gaugeMap.get(path);
180+
return gaugeMap.get(getSubPath(name));
181181
}
182182

183183
@Override

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -93,6 +93,9 @@ public interface BookKeeperClientStats {
9393
String FAILED_CONNECTION_COUNTER = "FAILED_CONNECTION_COUNTER";
9494
String FAILED_TLS_HANDSHAKE_COUNTER = "FAILED_TLS_HANDSHAKE_COUNTER";
9595

96+
// placementpolicy stats
97+
String NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK = "NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK";
98+
9699
OpStatsLogger getCreateOpLogger();
97100
OpStatsLogger getOpenOpLogger();
98101
OpStatsLogger getDeleteOpLogger();

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -251,6 +251,9 @@ Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBooki
251251
* <p>{@code customMetadata} is the same user defined data that user provides
252252
* when {@link BookKeeper#createLedger(int, int, int, BookKeeper.DigestType, byte[], Map)}.
253253
*
254+
* <p>If 'enforceMinNumRacksPerWriteQuorum' config is enabled then the bookies belonging to default
255+
* faultzone (rack) will be excluded while selecting bookies.
256+
*
254257
* @param ensembleSize
255258
* Ensemble Size
256259
* @param writeQuorumSize
@@ -274,6 +277,9 @@ PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
274277
* Choose a new bookie to replace <i>bookieToReplace</i>. If no bookie available in the cluster,
275278
* {@link BKNotEnoughBookiesException} is thrown.
276279
*
280+
* <p>If 'enforceMinNumRacksPerWriteQuorum' config is enabled then the bookies belonging to default
281+
* faultzone (rack) will be excluded while selecting bookies.
282+
*
277283
* @param ensembleSize
278284
* the value of ensembleSize
279285
* @param writeQuorumSize

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java

Lines changed: 91 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIES_LEFT;
2323
import static org.apache.bookkeeper.bookie.BookKeeperServerStats.FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER;
2424
import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
25+
import static org.apache.bookkeeper.client.BookKeeperClientStats.NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK;
2526
import static org.apache.bookkeeper.client.BookKeeperClientStats.READ_REQUESTS_REORDERED;
2627
import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.UNKNOWN_REGION;
2728

@@ -67,6 +68,7 @@
6768
import org.apache.bookkeeper.net.ScriptBasedMapping;
6869
import org.apache.bookkeeper.net.StabilizeNetworkTopology;
6970
import org.apache.bookkeeper.stats.Counter;
71+
import org.apache.bookkeeper.stats.Gauge;
7072
import org.apache.bookkeeper.stats.OpStatsLogger;
7173
import org.apache.bookkeeper.stats.StatsLogger;
7274
import org.apache.bookkeeper.stats.annotations.StatsDoc;
@@ -114,11 +116,6 @@ static class DefaultResolver implements DNSToSwitchMapping {
114116

115117
final Supplier<String> defaultRackSupplier;
116118

117-
// for backwards compat
118-
public DefaultResolver() {
119-
this(() -> NetworkTopology.DEFAULT_REGION_AND_RACK);
120-
}
121-
122119
public DefaultResolver(Supplier<String> defaultRackSupplier) {
123120
checkNotNull(defaultRackSupplier, "defaultRackSupplier should not be null");
124121
this.defaultRackSupplier = defaultRackSupplier;
@@ -240,7 +237,16 @@ public void reloadCachedMappings() {
240237
help = "The distribution of number of bookies reordered on each read request"
241238
)
242239
protected OpStatsLogger readReorderedCounter = null;
240+
@StatsDoc(
241+
name = FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER,
242+
help = "Counter for number of times DNSResolverDecorator failed to resolve Network Location"
243+
)
243244
protected Counter failedToResolveNetworkLocationCounter = null;
245+
@StatsDoc(
246+
name = NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK,
247+
help = "Gauge for the number of writable Bookies in default rack"
248+
)
249+
protected Gauge<Integer> numWritableBookiesInDefaultRack;
244250

245251
private String defaultRack = NetworkTopology.DEFAULT_RACK;
246252

@@ -282,8 +288,24 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns
282288
this.bookiesJoinedCounter = statsLogger.getOpStatsLogger(BOOKIES_JOINED);
283289
this.bookiesLeftCounter = statsLogger.getOpStatsLogger(BOOKIES_LEFT);
284290
this.readReorderedCounter = statsLogger.getOpStatsLogger(READ_REQUESTS_REORDERED);
285-
this.failedToResolveNetworkLocationCounter = statsLogger
286-
.getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER);
291+
this.failedToResolveNetworkLocationCounter = statsLogger.getCounter(FAILED_TO_RESOLVE_NETWORK_LOCATION_COUNTER);
292+
this.numWritableBookiesInDefaultRack = new Gauge<Integer>() {
293+
@Override
294+
public Integer getDefaultValue() {
295+
return 0;
296+
}
297+
298+
@Override
299+
public Integer getSample() {
300+
rwLock.readLock().lock();
301+
try {
302+
return topology.countNumOfAvailableNodes(getDefaultRack(), Collections.emptySet());
303+
} finally {
304+
rwLock.readLock().unlock();
305+
}
306+
}
307+
};
308+
this.statsLogger.registerGauge(NUM_WRITABLE_BOOKIES_IN_DEFAULT_RACK, numWritableBookiesInDefaultRack);
287309
this.reorderReadsRandom = reorderReadsRandom;
288310
this.stabilizePeriodSeconds = stabilizePeriodSeconds;
289311
this.reorderThresholdPendingRequests = reorderThresholdPendingRequests;
@@ -417,7 +439,9 @@ public void onBookieRackChange(List<BookieSocketAddress> bookieAddressList) {
417439
if (node != null) {
418440
// refresh the rack info if its a known bookie
419441
topology.remove(node);
420-
topology.add(createBookieNode(bookieAddress));
442+
BookieNode newNode = createBookieNode(bookieAddress);
443+
topology.add(newNode);
444+
knownBookies.put(bookieAddress, newNode);
421445
}
422446
}
423447
} finally {
@@ -455,6 +479,9 @@ public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writab
455479
}
456480
}
457481

482+
/*
483+
* this method should be called in writelock scope of 'rwLock'
484+
*/
458485
@Override
459486
public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
460487
for (BookieSocketAddress addr : leftBookies) {
@@ -483,6 +510,9 @@ public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
483510
}
484511
}
485512

513+
/*
514+
* this method should be called in writelock scope of 'rwLock'
515+
*/
486516
@Override
487517
public void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies) {
488518
// node joined
@@ -531,26 +561,53 @@ private static Set<String> getNetworkLocations(Set<Node> bookieNodes) {
531561
return networkLocs;
532562
}
533563

564+
/*
565+
* this method should be called in readlock scope of 'rwLock'
566+
*/
567+
protected Set<BookieSocketAddress> addDefaultRackBookiesIfMinNumRacksIsEnforced(
568+
Set<BookieSocketAddress> excludeBookies) {
569+
Set<BookieSocketAddress> comprehensiveExclusionBookiesSet;
570+
if (enforceMinNumRacksPerWriteQuorum) {
571+
Set<BookieSocketAddress> bookiesInDefaultRack = null;
572+
Set<Node> defaultRackLeaves = topology.getLeaves(getDefaultRack());
573+
for (Node node : defaultRackLeaves) {
574+
if (node instanceof BookieNode) {
575+
if (bookiesInDefaultRack == null) {
576+
bookiesInDefaultRack = new HashSet<BookieSocketAddress>(excludeBookies);
577+
}
578+
bookiesInDefaultRack.add(((BookieNode) node).getAddr());
579+
} else {
580+
LOG.error("found non-BookieNode: {} as leaf of defaultrack: {}", node, getDefaultRack());
581+
}
582+
}
583+
if ((bookiesInDefaultRack == null) || bookiesInDefaultRack.isEmpty()) {
584+
comprehensiveExclusionBookiesSet = excludeBookies;
585+
} else {
586+
comprehensiveExclusionBookiesSet = new HashSet<BookieSocketAddress>(excludeBookies);
587+
comprehensiveExclusionBookiesSet.addAll(bookiesInDefaultRack);
588+
LOG.info("enforceMinNumRacksPerWriteQuorum is enabled, so Excluding bookies of defaultRack: {}",
589+
bookiesInDefaultRack);
590+
}
591+
} else {
592+
comprehensiveExclusionBookiesSet = excludeBookies;
593+
}
594+
return comprehensiveExclusionBookiesSet;
595+
}
596+
534597
@Override
535598
public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
536599
int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
537600
throws BKNotEnoughBookiesException {
538-
return newEnsembleInternal(ensembleSize, writeQuorumSize, excludeBookies, null, null);
539-
}
540-
541-
protected PlacementResult<List<BookieSocketAddress>> newEnsembleInternal(int ensembleSize,
542-
int writeQuorumSize,
543-
Set<BookieSocketAddress> excludeBookies,
544-
Ensemble<BookieNode> parentEnsemble,
545-
Predicate<BookieNode> parentPredicate)
546-
throws BKNotEnoughBookiesException {
547-
return newEnsembleInternal(
548-
ensembleSize,
549-
writeQuorumSize,
550-
writeQuorumSize,
551-
excludeBookies,
552-
parentEnsemble,
553-
parentPredicate);
601+
rwLock.readLock().lock();
602+
try {
603+
Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
604+
excludeBookies);
605+
PlacementResult<List<BookieSocketAddress>> newEnsembleResult = newEnsembleInternal(ensembleSize,
606+
writeQuorumSize, ackQuorumSize, comprehensiveExclusionBookiesSet, null, null);
607+
return newEnsembleResult;
608+
} finally {
609+
rwLock.readLock().unlock();
610+
}
554611
}
555612

556613
@Override
@@ -643,6 +700,7 @@ public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int
643700
throws BKNotEnoughBookiesException {
644701
rwLock.readLock().lock();
645702
try {
703+
excludeBookies = addDefaultRackBookiesIfMinNumRacksIsEnforced(excludeBookies);
646704
excludeBookies.addAll(currentEnsemble);
647705
BookieNode bn = knownBookies.get(bookieToReplace);
648706
if (null == bn) {
@@ -1253,28 +1311,31 @@ static void shuffleWithMask(DistributionSchedule.WriteSet writeSet,
12531311
}
12541312
}
12551313

1314+
// this method should be called in readlock scope of 'rwlock'
12561315
@Override
12571316
public boolean isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList, int writeQuorumSize,
12581317
int ackQuorumSize) {
12591318
int ensembleSize = ensembleList.size();
12601319
int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum);
1261-
HashSet<String> racksOrRegionsInQuorum = new HashSet<String>();
1320+
HashSet<String> racksInQuorum = new HashSet<String>();
12621321
BookieSocketAddress bookie;
12631322
for (int i = 0; i < ensembleList.size(); i++) {
1264-
racksOrRegionsInQuorum.clear();
1323+
racksInQuorum.clear();
12651324
for (int j = 0; j < writeQuorumSize; j++) {
12661325
bookie = ensembleList.get((i + j) % ensembleSize);
12671326
try {
1268-
racksOrRegionsInQuorum.add(knownBookies.get(bookie).getNetworkLocation());
1327+
racksInQuorum.add(knownBookies.get(bookie).getNetworkLocation());
12691328
} catch (Exception e) {
12701329
/*
1271-
* any issue/exception in analyzing whether ensemble is strictly adhering to
1272-
* placement policy should be swallowed.
1330+
* any issue/exception in analyzing whether ensemble is
1331+
* strictly adhering to placement policy should be
1332+
* swallowed.
12731333
*/
12741334
LOG.warn("Received exception while trying to get network location of bookie: {}", bookie, e);
12751335
}
12761336
}
1277-
if (racksOrRegionsInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble) {
1337+
if ((racksInQuorum.size() < minNumRacksPerWriteQuorumForThisEnsemble)
1338+
|| (enforceMinNumRacksPerWriteQuorum && racksInQuorum.contains(getDefaultRack()))) {
12781339
return false;
12791340
}
12801341
}

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java

Lines changed: 15 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -228,7 +228,7 @@ protected List<BookieNode> selectRandomFromRegions(Set<String> availableRegions,
228228

229229
@Override
230230
public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
231-
int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
231+
int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludedBookies)
232232
throws BKException.BKNotEnoughBookiesException {
233233

234234
int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability;
@@ -257,7 +257,9 @@ public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
257257

258258
rwLock.readLock().lock();
259259
try {
260-
Set<Node> excludeNodes = convertBookiesToNodes(excludeBookies);
260+
Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
261+
excludedBookies);
262+
Set<Node> excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet);
261263
Set<String> availableRegions = new HashSet<String>();
262264
for (String region: perRegionPlacement.keySet()) {
263265
if ((null == disallowBookiePlacementInRegionFeatureName)
@@ -294,8 +296,8 @@ public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
294296
effectiveMinRegionsForDurability, minNumRacksPerWriteQuorum);
295297
TopologyAwareEnsemblePlacementPolicy nextPolicy = perRegionPlacement.get(
296298
availableRegions.iterator().next());
297-
return nextPolicy.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize, excludeBookies, ensemble,
298-
ensemble);
299+
return nextPolicy.newEnsemble(ensembleSize, writeQuorumSize, writeQuorumSize,
300+
comprehensiveExclusionBookiesSet, ensemble, ensemble);
299301
}
300302

301303
int remainingEnsemble = ensembleSize;
@@ -349,9 +351,10 @@ public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
349351
int newEnsembleSize = currentAllocation.getLeft() + addToEnsembleSize;
350352
int newWriteQuorumSize = currentAllocation.getRight() + addToWriteQuorum;
351353
try {
352-
List<BookieSocketAddress> allocated = policyWithinRegion.newEnsemble(newEnsembleSize,
353-
newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble,
354-
tempEnsemble).getResult();
354+
List<BookieSocketAddress> allocated = policyWithinRegion
355+
.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize,
356+
comprehensiveExclusionBookiesSet, tempEnsemble, tempEnsemble)
357+
.getResult();
355358
ensemble = tempEnsemble;
356359
remainingEnsemble -= addToEnsembleSize;
357360
remainingWriteQuorum -= addToWriteQuorum;
@@ -379,12 +382,12 @@ public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
379382
if (regionsReachedMaxAllocation.contains(region)) {
380383
if (currentAllocation.getLeft() > 0) {
381384
LOG.info("Allocating {} bookies in region {} : ensemble {} exclude {}",
382-
currentAllocation.getLeft(), region, excludeBookies, ensemble);
385+
currentAllocation.getLeft(), region, comprehensiveExclusionBookiesSet, ensemble);
383386
policyWithinRegion.newEnsemble(
384387
currentAllocation.getLeft(),
385388
currentAllocation.getRight(),
386389
currentAllocation.getRight(),
387-
excludeBookies,
390+
comprehensiveExclusionBookiesSet,
388391
ensemble,
389392
ensemble);
390393
LOG.info("Allocated {} bookies in region {} : {}",
@@ -428,7 +431,9 @@ public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int
428431
try {
429432
boolean enforceDurability = enforceDurabilityInReplace && !disableDurabilityFeature.isAvailable();
430433
int effectiveMinRegionsForDurability = enforceDurability ? minRegionsForDurability : 1;
431-
Set<Node> excludeNodes = convertBookiesToNodes(excludeBookies);
434+
Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
435+
excludeBookies);
436+
Set<Node> excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet);
432437
RRTopologyAwareCoverageEnsemble ensemble = new RRTopologyAwareCoverageEnsemble(ensembleSize,
433438
writeQuorumSize,
434439
ackQuorumSize,

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,6 @@ public boolean validate() {
7878
}
7979

8080
protected static class BookieNode extends NodeBase {
81-
8281
private final BookieSocketAddress addr; // identifier of a bookie node.
8382

8483
BookieNode(BookieSocketAddress addr, String networkLoc) {
@@ -108,7 +107,6 @@ public boolean equals(Object obj) {
108107
public String toString() {
109108
return String.format("<Bookie:%s>", name);
110109
}
111-
112110
}
113111

114112
/**

bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopology.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.bookkeeper.net;
1919

20+
import java.util.Collection;
2021
import java.util.Set;
2122

2223
/**
@@ -76,4 +77,14 @@ public interface NetworkTopology {
7677
*/
7778
Set<Node> getLeaves(String loc);
7879

80+
/**
81+
* Return the number of leaves in <i>scope</i> but not in <i>excludedNodes</i>.
82+
*
83+
* <p>If scope starts with ~, return the number of nodes that are not
84+
* in <i>scope</i> and <i>excludedNodes</i>;
85+
* @param scope a path string that may start with ~
86+
* @param excludedNodes a list of nodes
87+
* @return number of available nodes
88+
*/
89+
int countNumOfAvailableNodes(String scope, Collection<Node> excludedNodes);
7990
}

bookkeeper-server/src/main/java/org/apache/bookkeeper/net/NetworkTopologyImpl.java

Lines changed: 2 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -789,15 +789,7 @@ public Set<Node> getLeaves(String scope) {
789789
}
790790
}
791791

792-
/**
793-
* Return the number of leaves in <i>scope</i> but not in <i>excludedNodes</i>.
794-
*
795-
* <p>If scope starts with ~, return the number of nodes that are not
796-
* in <i>scope</i> and <i>excludedNodes</i>;
797-
* @param scope a path string that may start with ~
798-
* @param excludedNodes a list of nodes
799-
* @return number of available nodes
800-
*/
792+
@Override
801793
public int countNumOfAvailableNodes(String scope, Collection<Node> excludedNodes) {
802794
boolean isExcluded = false;
803795
if (scope.startsWith("~")) {
@@ -815,7 +807,7 @@ public int countNumOfAvailableNodes(String scope, Collection<Node> excludedNodes
815807
}
816808
}
817809
Node n = getNode(scope);
818-
int scopeNodeCount = 1;
810+
int scopeNodeCount = 0;
819811
if (n instanceof InnerNode) {
820812
scopeNodeCount = ((InnerNode) n).getNumOfLeaves();
821813
}

0 commit comments

Comments
 (0)