|
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
@@ -75,6 +76,7 @@ public class TestDecommissioningStatus {
|
|
|
private static FileSystem localFileSys;
|
|
|
private static Configuration conf;
|
|
|
private static Path dir;
|
|
|
+ private static Logger LOG;
|
|
|
|
|
|
final ArrayList<String> decommissionedNodes = new ArrayList<String>(numDatanodes);
|
|
|
|
|
@@ -98,7 +100,7 @@ public class TestDecommissioningStatus {
|
|
|
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
|
|
|
4);
|
|
|
- conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
|
|
|
conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1);
|
|
|
|
|
@@ -111,6 +113,7 @@ public class TestDecommissioningStatus {
|
|
|
cluster.getNamesystem().getBlockManager().getDatanodeManager()
|
|
|
.setHeartbeatExpireInterval(3000);
|
|
|
Logger.getLogger(DecommissionManager.class).setLevel(Level.DEBUG);
|
|
|
+ LOG = Logger.getLogger(TestDecommissioningStatus.class);
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -431,4 +434,110 @@ public class TestDecommissioningStatus {
|
|
|
writeConfigFile(localFileSys, excludeFile, null);
|
|
|
dm.refreshNodes(conf);
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=120000)
|
|
|
+ public void testDecommissionLosingData() throws Exception {
|
|
|
+ ArrayList<String> nodes = new ArrayList<String>(2);
|
|
|
+ FSNamesystem fsn = cluster.getNamesystem();
|
|
|
+ BlockManager bm = fsn.getBlockManager();
|
|
|
+ DatanodeManager dm = bm.getDatanodeManager();
|
|
|
+ Path file1 = new Path("decommissionLosingData.dat");
|
|
|
+ writeFile(fileSys, file1, (short)numDatanodes);
|
|
|
+ Thread.sleep(1000);
|
|
|
+
|
|
|
+ // Shutdown dn1
|
|
|
+ LOG.info("Shutdown dn1");
|
|
|
+ DatanodeID dnID = cluster.getDataNodes().get(1).getDatanodeId();
|
|
|
+ String dnName = dnID.getXferAddr();
|
|
|
+ DatanodeDescriptor dnDescriptor1 = dm.getDatanode(dnID);
|
|
|
+ nodes.add(dnName);
|
|
|
+ DataNodeProperties stoppedDN1 = cluster.stopDataNode(1);
|
|
|
+ DFSTestUtil.waitForDatanodeState(cluster, dnID.getDatanodeUuid(),
|
|
|
+ false, 30000);
|
|
|
+
|
|
|
+ // Shutdown dn0
|
|
|
+ LOG.info("Shutdown dn0");
|
|
|
+ dnID = cluster.getDataNodes().get(0).getDatanodeId();
|
|
|
+ dnName = dnID.getXferAddr();
|
|
|
+ DatanodeDescriptor dnDescriptor0 = dm.getDatanode(dnID);
|
|
|
+ nodes.add(dnName);
|
|
|
+ DataNodeProperties stoppedDN0 = cluster.stopDataNode(0);
|
|
|
+ DFSTestUtil.waitForDatanodeState(cluster, dnID.getDatanodeUuid(),
|
|
|
+ false, 30000);
|
|
|
+
|
|
|
+ // Decommission the nodes.
|
|
|
+ LOG.info("Decommissioning nodes");
|
|
|
+ writeConfigFile(localFileSys, excludeFile, nodes);
|
|
|
+ dm.refreshNodes(conf);
|
|
|
+ BlockManagerTestUtil.recheckDecommissionState(dm);
|
|
|
+ assertTrue(dnDescriptor0.isDecommissioned());
|
|
|
+ assertTrue(dnDescriptor1.isDecommissioned());
|
|
|
+
|
|
|
+ // All nodes are dead and decommed. Blocks should be missing.
|
|
|
+ long missingBlocks = bm.getMissingBlocksCount();
|
|
|
+ long underreplicated = bm.getUnderReplicatedBlocksCount();
|
|
|
+ assertTrue(missingBlocks > 0);
|
|
|
+ assertTrue(underreplicated > 0);
|
|
|
+
|
|
|
+ // Bring back dn0
|
|
|
+ LOG.info("Bring back dn0");
|
|
|
+ cluster.restartDataNode(stoppedDN0, true);
|
|
|
+ do {
|
|
|
+ dnID = cluster.getDataNodes().get(0).getDatanodeId();
|
|
|
+ } while (dnID == null);
|
|
|
+ dnDescriptor0 = dm.getDatanode(dnID);
|
|
|
+ // Wait until it sends a block report.
|
|
|
+ while (dnDescriptor0.numBlocks() == 0) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Bring back dn1
|
|
|
+ LOG.info("Bring back dn1");
|
|
|
+ cluster.restartDataNode(stoppedDN1, true);
|
|
|
+ do {
|
|
|
+ dnID = cluster.getDataNodes().get(1).getDatanodeId();
|
|
|
+ } while (dnID == null);
|
|
|
+ dnDescriptor1 = dm.getDatanode(dnID);
|
|
|
+ // Wait until it sends a block report.
|
|
|
+ while (dnDescriptor1.numBlocks() == 0) {
|
|
|
+ Thread.sleep(100);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Blocks should be still be under-replicated
|
|
|
+ Thread.sleep(2000); // Let replication monitor run
|
|
|
+ assertEquals(underreplicated, bm.getUnderReplicatedBlocksCount());
|
|
|
+
|
|
|
+ // Start up a node.
|
|
|
+ LOG.info("Starting two more nodes");
|
|
|
+ cluster.startDataNodes(conf, 2, true, null, null);
|
|
|
+ cluster.waitActive();
|
|
|
+ // Replication should fix it.
|
|
|
+ int count = 0;
|
|
|
+ while((bm.getUnderReplicatedBlocksCount() > 0 ||
|
|
|
+ bm.getPendingReplicationBlocksCount() > 0) &&
|
|
|
+ count++ < 10) {
|
|
|
+ Thread.sleep(1000);
|
|
|
+ }
|
|
|
+
|
|
|
+ assertEquals(0, bm.getUnderReplicatedBlocksCount());
|
|
|
+ assertEquals(0, bm.getPendingReplicationBlocksCount());
|
|
|
+ assertEquals(0, bm.getMissingBlocksCount());
|
|
|
+
|
|
|
+ // Shutdown the extra nodes.
|
|
|
+ dnID = cluster.getDataNodes().get(3).getDatanodeId();
|
|
|
+ cluster.stopDataNode(3);
|
|
|
+ DFSTestUtil.waitForDatanodeState(cluster, dnID.getDatanodeUuid(),
|
|
|
+ false, 30000);
|
|
|
+
|
|
|
+ dnID = cluster.getDataNodes().get(2).getDatanodeId();
|
|
|
+ cluster.stopDataNode(2);
|
|
|
+ DFSTestUtil.waitForDatanodeState(cluster, dnID.getDatanodeUuid(),
|
|
|
+ false, 30000);
|
|
|
+
|
|
|
+ // Call refreshNodes on FSNamesystem with empty exclude file to remove the
|
|
|
+ // datanode from decommissioning list and make it available again.
|
|
|
+ writeConfigFile(localFileSys, excludeFile, null);
|
|
|
+ dm.refreshNodes(conf);
|
|
|
+ fileSys.delete(file1, false);
|
|
|
+ }
|
|
|
}
|