Skip to content

Commit c99a121

Browse files
Stephen O'Donnelljojochuang
authored andcommitted
HDFS-14637. Namenode may not replicate blocks to meet the policy after enabling upgradeDomain. Contributed by Stephen O'Donnell.
Reviewed-by: Ayush Saxena <ayushsaxena@apache.org> Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
1 parent 844b766 commit c99a121

File tree

11 files changed

+362
-35
lines changed

11 files changed

+362
-35
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

Lines changed: 48 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2009,6 +2009,7 @@ boolean hasEnoughEffectiveReplicas(BlockInfo block,
20092009
(pendingReplicaNum > 0 || isPlacementPolicySatisfied(block));
20102010
}
20112011

2012+
@VisibleForTesting
20122013
BlockReconstructionWork scheduleReconstruction(BlockInfo block,
20132014
int priority) {
20142015
// skip abandoned block or block reopened for append
@@ -2053,7 +2054,9 @@ BlockReconstructionWork scheduleReconstruction(BlockInfo block,
20532054
additionalReplRequired = requiredRedundancy - numReplicas.liveReplicas()
20542055
- pendingNum;
20552056
} else {
2056-
additionalReplRequired = 1; // Needed on a new rack
2057+
// Violates placement policy. Needed on a new rack or domain etc.
2058+
BlockPlacementStatus placementStatus = getBlockPlacementStatus(block);
2059+
additionalReplRequired = placementStatus.getAdditionalReplicasRequired();
20572060
}
20582061

20592062
final BlockCollection bc = getBlockCollection(block);
@@ -2086,20 +2089,6 @@ BlockReconstructionWork scheduleReconstruction(BlockInfo block,
20862089
}
20872090
}
20882091

2089-
private boolean isInNewRack(DatanodeDescriptor[] srcs,
2090-
DatanodeDescriptor target) {
2091-
LOG.debug("check if target {} increases racks, srcs={}", target,
2092-
Arrays.asList(srcs));
2093-
for (DatanodeDescriptor src : srcs) {
2094-
if (!src.isDecommissionInProgress() &&
2095-
src.getNetworkLocation().equals(target.getNetworkLocation())) {
2096-
LOG.debug("the target {} is in the same rack with src {}", target, src);
2097-
return false;
2098-
}
2099-
}
2100-
return true;
2101-
}
2102-
21032092
private boolean validateReconstructionWork(BlockReconstructionWork rw) {
21042093
BlockInfo block = rw.getBlock();
21052094
int priority = rw.getPriority();
@@ -2125,10 +2114,16 @@ private boolean validateReconstructionWork(BlockReconstructionWork rw) {
21252114
}
21262115

21272116
DatanodeStorageInfo[] targets = rw.getTargets();
2117+
BlockPlacementStatus placementStatus = getBlockPlacementStatus(block);
21282118
if ((numReplicas.liveReplicas() >= requiredRedundancy) &&
2129-
(!isPlacementPolicySatisfied(block)) ) {
2130-
if (!isInNewRack(rw.getSrcNodes(), targets[0].getDatanodeDescriptor())) {
2131-
// No use continuing, unless a new rack in this case
2119+
(!placementStatus.isPlacementPolicySatisfied())) {
2120+
BlockPlacementStatus newPlacementStatus =
2121+
getBlockPlacementStatus(block, targets);
2122+
if (!newPlacementStatus.isPlacementPolicySatisfied() &&
2123+
(newPlacementStatus.getAdditionalReplicasRequired() >=
2124+
placementStatus.getAdditionalReplicasRequired())) {
2125+
// If the new targets do not meet the placement policy, or at least
2126+
// reduce the number of replicas needed, then no use continuing.
21322127
return false;
21332128
}
21342129
// mark that the reconstruction work is to replicate internal block to a
@@ -4562,15 +4557,48 @@ public boolean containsInvalidateBlock(final DatanodeInfo dn,
45624557
}
45634558

45644559
boolean isPlacementPolicySatisfied(BlockInfo storedBlock) {
4560+
return getBlockPlacementStatus(storedBlock, null)
4561+
.isPlacementPolicySatisfied();
4562+
}
4563+
4564+
BlockPlacementStatus getBlockPlacementStatus(BlockInfo storedBlock) {
4565+
return getBlockPlacementStatus(storedBlock, null);
4566+
}
4567+
4568+
BlockPlacementStatus getBlockPlacementStatus(BlockInfo storedBlock,
4569+
DatanodeStorageInfo[] additionalStorage) {
45654570
List<DatanodeDescriptor> liveNodes = new ArrayList<>();
4571+
if (additionalStorage != null) {
4572+
// additionalNodes, are potential new targets for the block. If there are
4573+
// any passed, include them when checking the placement policy to see if
4574+
// the policy is met, when it may not have been met without these nodes.
4575+
for (DatanodeStorageInfo s : additionalStorage) {
4576+
liveNodes.add(getDatanodeDescriptorFromStorage(s));
4577+
}
4578+
}
45664579
Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
45674580
.getNodes(storedBlock);
45684581
for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
45694582
if (storage.getStorageType() == StorageType.PROVIDED
45704583
&& storage.getState() == State.NORMAL) {
45714584
// assume the policy is satisfied for blocks on PROVIDED storage
45724585
// as long as the storage is in normal state.
4573-
return true;
4586+
return new BlockPlacementStatus() {
4587+
@Override
4588+
public boolean isPlacementPolicySatisfied() {
4589+
return true;
4590+
}
4591+
4592+
@Override
4593+
public String getErrorDescription() {
4594+
return null;
4595+
}
4596+
4597+
@Override
4598+
public int getAdditionalReplicasRequired() {
4599+
return 0;
4600+
}
4601+
};
45744602
}
45754603
final DatanodeDescriptor cur = getDatanodeDescriptorFromStorage(storage);
45764604
// Nodes under maintenance should be counted as valid replicas from
@@ -4586,8 +4614,7 @@ boolean isPlacementPolicySatisfied(BlockInfo storedBlock) {
45864614
.getPolicy(blockType);
45874615
int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock)
45884616
.getRealTotalBlockNum() : storedBlock.getReplication();
4589-
return placementPolicy.verifyBlockPlacement(locs, numReplicas)
4590-
.isPlacementPolicySatisfied();
4617+
return placementPolicy.verifyBlockPlacement(locs, numReplicas);
45914618
}
45924619

45934620
boolean isNeededReconstructionForMaintenance(BlockInfo storedBlock,

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,4 +39,12 @@ public interface BlockPlacementStatus {
3939
*/
4040
public String getErrorDescription();
4141

42+
/**
43+
* Return the number of additional replicas needed to ensure the block
44+
* placement policy is satisfied.
45+
* @return The number of new replicas needed to satisify the placement policy
46+
* or zero if no extra are needed
47+
*/
48+
int getAdditionalReplicasRequired();
49+
4250
}

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -45,4 +45,12 @@ public String getErrorDescription() {
4545
" more rack(s). Total number of racks in the cluster: " + totalRacks;
4646
}
4747

48+
@Override
49+
public int getAdditionalReplicasRequired() {
50+
if (isPlacementPolicySatisfied()) {
51+
return 0;
52+
} else {
53+
return requiredRacks - currentRacks;
54+
}
55+
}
4856
}

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithNodeGroup.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -78,4 +78,15 @@ public String getErrorDescription() {
7878
}
7979
return errorDescription.toString();
8080
}
81+
82+
@Override
83+
public int getAdditionalReplicasRequired() {
84+
if (isPlacementPolicySatisfied()) {
85+
return 0;
86+
} else {
87+
int parent = parentBlockPlacementStatus.getAdditionalReplicasRequired();
88+
int child = requiredNodeGroups - currentNodeGroups.size();
89+
return Math.max(parent, child);
90+
}
91+
}
8192
}

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusWithUpgradeDomain.java

Lines changed: 21 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -85,4 +85,24 @@ public String getErrorDescription() {
8585
}
8686
return errorDescription.toString();
8787
}
88-
}
88+
89+
@Override
90+
public int getAdditionalReplicasRequired() {
91+
if (isPlacementPolicySatisfied()) {
92+
return 0;
93+
} else {
94+
// It is possible for a block to have the correct number of upgrade
95+
// domains, but only a single rack, or be on multiple racks, but only in
96+
// one upgrade domain.
97+
int parent = parentBlockPlacementStatus.getAdditionalReplicasRequired();
98+
int child;
99+
100+
if (numberOfReplicas <= upgradeDomainFactor) {
101+
child = numberOfReplicas - upgradeDomains.size();
102+
} else {
103+
child = upgradeDomainFactor - upgradeDomains.size();
104+
}
105+
return Math.max(parent, child);
106+
}
107+
}
108+
}

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -556,17 +556,24 @@ public static boolean allBlockReplicasCorrupt(MiniDFSCluster cluster,
556556
}
557557
}
558558

559+
public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b,
560+
int racks, int replicas, int neededReplicas)
561+
throws TimeoutException, InterruptedException {
562+
waitForReplication(cluster, b, racks, replicas, neededReplicas, 0);
563+
}
564+
559565
/*
560566
* Wait up to 20s for the given block to be replicated across
561567
* the requested number of racks, with the requested number of
562568
* replicas, and the requested number of replicas still needed.
563569
*/
564570
public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b,
565-
int racks, int replicas, int neededReplicas)
571+
int racks, int replicas, int neededReplicas, int neededDomains)
566572
throws TimeoutException, InterruptedException {
567573
int curRacks = 0;
568574
int curReplicas = 0;
569575
int curNeededReplicas = 0;
576+
int curDomains = 0;
570577
int count = 0;
571578
final int ATTEMPTS = 20;
572579

@@ -577,17 +584,21 @@ public static void waitForReplication(MiniDFSCluster cluster, ExtendedBlock b,
577584
curRacks = r[0];
578585
curReplicas = r[1];
579586
curNeededReplicas = r[2];
587+
curDomains = r[3];
580588
count++;
581589
} while ((curRacks != racks ||
582590
curReplicas != replicas ||
583-
curNeededReplicas != neededReplicas) && count < ATTEMPTS);
591+
curNeededReplicas != neededReplicas ||
592+
(neededDomains != 0 && curDomains != neededDomains))
593+
&& count < ATTEMPTS);
584594

585595
if (count == ATTEMPTS) {
586596
throw new TimeoutException("Timed out waiting for replication."
587597
+ " Needed replicas = "+neededReplicas
588598
+ " Cur needed replicas = "+curNeededReplicas
589599
+ " Replicas = "+replicas+" Cur replicas = "+curReplicas
590-
+ " Racks = "+racks+" Cur racks = "+curRacks);
600+
+ " Racks = "+racks+" Cur racks = "+curRacks
601+
+ " Domains = "+neededDomains+" Cur domains = "+curDomains);
591602
}
592603
}
593604

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

Lines changed: 28 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,8 @@ public static void updateState(final BlockManager blockManager) {
8181

8282
/**
8383
* @return a tuple of the replica state (number racks, number live
84-
* replicas, and number needed replicas) for the given block.
84+
* replicas, number needed replicas and number of UpgradeDomains) for the
85+
* given block.
8586
*/
8687
public static int[] getReplicaInfo(final FSNamesystem namesystem, final Block b) {
8788
final BlockManager bm = namesystem.getBlockManager();
@@ -90,7 +91,8 @@ public static int[] getReplicaInfo(final FSNamesystem namesystem, final Block b)
9091
final BlockInfo storedBlock = bm.getStoredBlock(b);
9192
return new int[]{getNumberOfRacks(bm, b),
9293
bm.countNodes(storedBlock).liveReplicas(),
93-
bm.neededReconstruction.contains(storedBlock) ? 1 : 0};
94+
bm.neededReconstruction.contains(storedBlock) ? 1 : 0,
95+
getNumberOfDomains(bm, b)};
9496
} finally {
9597
namesystem.readUnlock();
9698
}
@@ -120,6 +122,30 @@ private static int getNumberOfRacks(final BlockManager blockManager,
120122
return rackSet.size();
121123
}
122124

125+
/**
126+
* @return the number of UpgradeDomains over which a given block is replicated
127+
* decommissioning/decommissioned nodes are not counted. corrupt replicas
128+
* are also ignored.
129+
*/
130+
private static int getNumberOfDomains(final BlockManager blockManager,
131+
final Block b) {
132+
final Set<String> domSet = new HashSet<String>(0);
133+
final Collection<DatanodeDescriptor> corruptNodes =
134+
getCorruptReplicas(blockManager).getNodes(b);
135+
for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) {
136+
final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
137+
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
138+
if ((corruptNodes == null) || !corruptNodes.contains(cur)) {
139+
String domain = cur.getUpgradeDomain();
140+
if (domain != null && !domSet.contains(domain)) {
141+
domSet.add(domain);
142+
}
143+
}
144+
}
145+
}
146+
return domSet.size();
147+
}
148+
123149
/**
124150
* @return redundancy monitor thread instance from block manager.
125151
*/
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,57 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hdfs.server.blockmanagement;
19+
20+
import static org.junit.Assert.assertEquals;
21+
import static org.junit.Assert.assertTrue;
22+
import static org.junit.Assert.assertFalse;
23+
import org.junit.Test;
24+
25+
/**
26+
* Unit tests to validate the BlockPlacementStatusDefault policy, focusing on
27+
* the getAdditionAlReplicasRequired method.
28+
*/
29+
public class TestBlockPlacementStatusDefault {
30+
31+
@Test
32+
public void testIsPolicySatisfiedCorrectly() {
33+
// 2 current racks and 2 expected
34+
BlockPlacementStatusDefault bps =
35+
new BlockPlacementStatusDefault(2, 2, 5);
36+
assertTrue(bps.isPlacementPolicySatisfied());
37+
assertEquals(0, bps.getAdditionalReplicasRequired());
38+
39+
// 1 current rack and 2 expected
40+
bps =
41+
new BlockPlacementStatusDefault(1, 2, 5);
42+
assertFalse(bps.isPlacementPolicySatisfied());
43+
assertEquals(1, bps.getAdditionalReplicasRequired());
44+
45+
// 3 current racks and 2 expected
46+
bps =
47+
new BlockPlacementStatusDefault(3, 2, 5);
48+
assertTrue(bps.isPlacementPolicySatisfied());
49+
assertEquals(0, bps.getAdditionalReplicasRequired());
50+
51+
// 1 current rack and 2 expected, but only 1 rack on the cluster
52+
bps =
53+
new BlockPlacementStatusDefault(1, 2, 1);
54+
assertTrue(bps.isPlacementPolicySatisfied());
55+
assertEquals(0, bps.getAdditionalReplicasRequired());
56+
}
57+
}

0 commit comments

Comments
 (0)