|
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
@@ -780,4 +781,53 @@ public class TestDecommission {
|
|
|
Thread.sleep(HEARTBEAT_INTERVAL * 1000);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=120000)
|
|
|
+ public void testDecommissionWithOpenfile() throws IOException, InterruptedException {
|
|
|
+ LOG.info("Starting test testDecommissionWithOpenfile");
|
|
|
+
|
|
|
+ //At most 4 nodes will be decommissioned
|
|
|
+ startCluster(1, 7, conf);
|
|
|
+
|
|
|
+ FileSystem fileSys = cluster.getFileSystem(0);
|
|
|
+ FSNamesystem ns = cluster.getNamesystem(0);
|
|
|
+
|
|
|
+ String openFile = "/testDecommissionWithOpenfile.dat";
|
|
|
+
|
|
|
+ writeFile(fileSys, new Path(openFile), (short)3);
|
|
|
+ // make sure the file was open for write
|
|
|
+ FSDataOutputStream fdos = fileSys.append(new Path(openFile));
|
|
|
+
|
|
|
+ LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(cluster.getNameNode(0), openFile, 0, fileSize);
|
|
|
+
|
|
|
+ DatanodeInfo[] dnInfos4LastBlock = lbs.getLastLocatedBlock().getLocations();
|
|
|
+ DatanodeInfo[] dnInfos4FirstBlock = lbs.get(0).getLocations();
|
|
|
+
|
|
|
+ ArrayList<String> nodes = new ArrayList<String>();
|
|
|
+ ArrayList<DatanodeInfo> dnInfos = new ArrayList<DatanodeInfo>();
|
|
|
+
|
|
|
+ for (DatanodeInfo datanodeInfo : dnInfos4FirstBlock) {
|
|
|
+ DatanodeInfo found = datanodeInfo;
|
|
|
+ for (DatanodeInfo dif: dnInfos4LastBlock) {
|
|
|
+ if (datanodeInfo.equals(dif)) {
|
|
|
+ found = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (found != null) {
|
|
|
+ nodes.add(found.getXferAddr());
|
|
|
+ dnInfos.add(found);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ //decommission one of the 3 nodes which have last block
|
|
|
+ nodes.add(dnInfos4LastBlock[0].getXferAddr());
|
|
|
+ dnInfos.add(dnInfos4LastBlock[0]);
|
|
|
+
|
|
|
+ writeConfigFile(excludeFile, nodes);
|
|
|
+ refreshNodes(ns, conf);
|
|
|
+ for (DatanodeInfo dn : dnInfos) {
|
|
|
+ waitNodeState(dn, AdminStates.DECOMMISSIONED);
|
|
|
+ }
|
|
|
+
|
|
|
+ fdos.close();
|
|
|
+ }
|
|
|
}
|