|
@@ -40,14 +40,16 @@ import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
-import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
@@ -55,6 +57,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -81,6 +84,9 @@ public class TestDecommissionWithStriped {
|
|
// replication interval
|
|
// replication interval
|
|
private static final int NAMENODE_REPLICATION_INTERVAL = 1;
|
|
private static final int NAMENODE_REPLICATION_INTERVAL = 1;
|
|
|
|
|
|
|
|
+ private int replicationStreamsHardLimit =
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT;
|
|
|
|
+
|
|
private Path decommissionDir;
|
|
private Path decommissionDir;
|
|
private Path hostsFile;
|
|
private Path hostsFile;
|
|
private Path excludeFile;
|
|
private Path excludeFile;
|
|
@@ -273,7 +279,6 @@ public class TestDecommissionWithStriped {
|
|
fsn.getNumDecomLiveDataNodes());
|
|
fsn.getNumDecomLiveDataNodes());
|
|
|
|
|
|
// Ensure decommissioned datanode is not automatically shutdown
|
|
// Ensure decommissioned datanode is not automatically shutdown
|
|
- DFSClient client = getDfsClient(cluster.getNameNode(0), conf);
|
|
|
|
assertEquals("All datanodes must be alive", numDNs,
|
|
assertEquals("All datanodes must be alive", numDNs,
|
|
client.datanodeReport(DatanodeReportType.LIVE).length);
|
|
client.datanodeReport(DatanodeReportType.LIVE).length);
|
|
|
|
|
|
@@ -283,6 +288,65 @@ public class TestDecommissionWithStriped {
|
|
cleanupFile(dfs, ecFile);
|
|
cleanupFile(dfs, ecFile);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * DN decommission shouldn't reconstruction busy DN block.
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
|
+ public void testDecommissionWithBusyNode() throws Exception {
|
|
|
|
+ byte busyDNIndex = 1;
|
|
|
|
+ byte decommisionDNIndex = 0;
|
|
|
|
+ //1. create EC file
|
|
|
|
+ final Path ecFile = new Path(ecDir, "testDecommissionWithBusyNode");
|
|
|
|
+ int writeBytes = cellSize * dataBlocks;
|
|
|
|
+ writeStripedFile(dfs, ecFile, writeBytes);
|
|
|
|
+ Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
|
|
|
|
+ FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
|
|
|
|
+
|
|
|
|
+ //2. make once DN busy
|
|
|
|
+ final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
|
|
|
|
+ .getINode4Write(ecFile.toString()).asFile();
|
|
|
|
+ BlockInfo firstBlock = fileNode.getBlocks()[0];
|
|
|
|
+ DatanodeStorageInfo[] dnStorageInfos = bm.getStorages(firstBlock);
|
|
|
|
+ DatanodeDescriptor busyNode =
|
|
|
|
+ dnStorageInfos[busyDNIndex].getDatanodeDescriptor();
|
|
|
|
+ for (int j = 0; j < replicationStreamsHardLimit; j++) {
|
|
|
|
+ busyNode.incrementPendingReplicationWithoutTargets();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //3. decomission one node
|
|
|
|
+ List<DatanodeInfo> decommisionNodes = new ArrayList<>();
|
|
|
|
+ decommisionNodes.add(
|
|
|
|
+ dnStorageInfos[decommisionDNIndex].getDatanodeDescriptor());
|
|
|
|
+ decommissionNode(0, decommisionNodes, AdminStates.DECOMMISSIONED);
|
|
|
|
+ assertEquals(decommisionNodes.size(), fsn.getNumDecomLiveDataNodes());
|
|
|
|
+
|
|
|
|
+ //4. wait for decommission block to replicate
|
|
|
|
+ Thread.sleep(3000);
|
|
|
|
+ DatanodeStorageInfo[] newDnStorageInfos = bm.getStorages(firstBlock);
|
|
|
|
+ Assert.assertEquals("Busy DN shouldn't be reconstructed",
|
|
|
|
+ dnStorageInfos[busyDNIndex].getStorageID(),
|
|
|
|
+ newDnStorageInfos[busyDNIndex].getStorageID());
|
|
|
|
+
|
|
|
|
+ //5. check decommission DN block index, it should be reconstructed again
|
|
|
|
+ LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations(
|
|
|
|
+ ecFile.toString(), 0, writeBytes);
|
|
|
|
+ LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0));
|
|
|
|
+ int decommissionBlockIndexCount = 0;
|
|
|
|
+ for (byte index : bg.getBlockIndices()) {
|
|
|
|
+ if (index == decommisionDNIndex) {
|
|
|
|
+ decommissionBlockIndexCount++;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Assert.assertEquals("Decommission DN block should be reconstructed", 2,
|
|
|
|
+ decommissionBlockIndexCount);
|
|
|
|
+
|
|
|
|
+ FileChecksum fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes);
|
|
|
|
+ Assert.assertTrue("Checksum mismatches!",
|
|
|
|
+ fileChecksum1.equals(fileChecksum2));
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Tests to verify that the file checksum should be able to compute after the
|
|
* Tests to verify that the file checksum should be able to compute after the
|
|
* decommission operation.
|
|
* decommission operation.
|