HDFS-11499. Decommissioning stuck because of failing recovery. Contributed by Lukas Majercak and Manoj Govindassamy.

(cherry picked from commit 385d2cb777)

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

(cherry picked from commit 60be2e5d8a)

 Conflicts:
    hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
This commit is contained in:
Wei-Chiu Chuang 2017-03-13 13:41:13 -07:00
parent 72fc7e0520
commit 851ba7d9d1
2 changed files with 54 additions and 1 deletions

View File

@ -736,7 +736,12 @@ public class BlockManager implements BlockStatsMXBean {
return false; // already completed (e.g. by syncBlock)
final boolean b = commitBlock(lastBlock, commitBlock);
if (countNodes(lastBlock).liveReplicas() >= minReplication) {
// Count replicas on decommissioning nodes, as these will not be
// decommissioned unless recovery/completing last block has finished
NumberReplicas numReplicas = countNodes(lastBlock);
if (numReplicas.liveReplicas() + numReplicas.decommissioning() >=
minReplication) {
if (b) {
addExpectedReplicasToPending(lastBlock, bc);
}

View File

@ -873,6 +873,54 @@ public class TestDecommission {
fdos.close();
}
@Test(timeout = 360000)
public void testDecommissionWithOpenFileAndBlockRecovery()
throws IOException, InterruptedException {
startCluster(1, 6, conf);
cluster.waitActive();
Path file = new Path("/testRecoveryDecommission");
// Create a file and never close the output stream to trigger recovery
DistributedFileSystem dfs = cluster.getFileSystem();
FSNamesystem ns = cluster.getNamesystem(0);
FSDataOutputStream out = dfs.create(file, true,
conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
(short) 3, blockSize);
// Write data to the file
long writtenBytes = 0;
while (writtenBytes < fileSize) {
out.writeLong(writtenBytes);
writtenBytes += 8;
}
out.hsync();
DatanodeInfo[] lastBlockLocations = NameNodeAdapter.getBlockLocations(
cluster.getNameNode(), "/testRecoveryDecommission", 0, fileSize)
.getLastLocatedBlock().getLocations();
// Decommission all nodes of the last block
ArrayList<String> toDecom = new ArrayList<>();
for (DatanodeInfo dnDecom : lastBlockLocations) {
toDecom.add(dnDecom.getXferAddr());
}
writeConfigFile(excludeFile, toDecom);
refreshNodes(ns, conf);
// Make sure hard lease expires to trigger replica recovery
cluster.setLeasePeriod(300L, 300L);
Thread.sleep(2 * BLOCKREPORT_INTERVAL_MSEC);
for (DatanodeInfo dnDecom : lastBlockLocations) {
DatanodeInfo datanode = NameNodeAdapter.getDatanode(
cluster.getNamesystem(), dnDecom);
waitNodeState(datanode, AdminStates.DECOMMISSIONED);
}
assertEquals(dfs.getFileStatus(file).getLen(), writtenBytes);
}
/**
* Tests restart of namenode while datanode hosts are added to exclude file