|
@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
|
|
@@ -53,6 +54,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
|
|
|
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
|
|
|
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
|
|
|
import org.apache.hadoop.hdfs.tools.DiskBalancerCLI;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.test.PathUtils;
|
|
|
import org.apache.hadoop.util.Tool;
|
|
@@ -103,6 +105,50 @@ public class TestDiskBalancerCommand {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Tests if it's allowed to submit and execute plan when Datanode is in status
|
|
|
+ * other than REGULAR.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testSubmitPlanInNonRegularStatus() throws Exception {
|
|
|
+ final int numDatanodes = 1;
|
|
|
+ MiniDFSCluster miniCluster = null;
|
|
|
+ final Configuration hdfsConf = new HdfsConfiguration();
|
|
|
+
|
|
|
+ try {
|
|
|
+ /* new cluster with imbalanced capacity */
|
|
|
+ miniCluster = DiskBalancerTestUtil.newImbalancedCluster(
|
|
|
+ hdfsConf,
|
|
|
+ numDatanodes,
|
|
|
+ CAPACITIES,
|
|
|
+ DEFAULT_BLOCK_SIZE,
|
|
|
+ FILE_LEN,
|
|
|
+ StartupOption.ROLLBACK);
|
|
|
+
|
|
|
+ /* get full path of plan */
|
|
|
+ final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
|
|
|
+
|
|
|
+ try {
|
|
|
+ /* run execute command */
|
|
|
+ final String cmdLine = String.format(
|
|
|
+ "hdfs diskbalancer -%s %s",
|
|
|
+ EXECUTE,
|
|
|
+ planFileFullName);
|
|
|
+ runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+ } catch(RemoteException e) {
|
|
|
+ assertThat(e.getClassName(), containsString("DiskBalancerException"));
|
|
|
+ assertThat(e.toString(),
|
|
|
+ is(allOf(
|
|
|
+ containsString("Datanode is in special state"),
|
|
|
+ containsString("Disk balancing not permitted."))));
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (miniCluster != null) {
|
|
|
+ miniCluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Tests running multiple commands under on setup. This mainly covers
|
|
|
* {@link org.apache.hadoop.hdfs.server.diskbalancer.command.Command#close}
|
|
@@ -122,36 +168,16 @@ public class TestDiskBalancerCommand {
|
|
|
CAPACITIES,
|
|
|
DEFAULT_BLOCK_SIZE,
|
|
|
FILE_LEN);
|
|
|
- String cmdLine = "";
|
|
|
- List<String> outputs = null;
|
|
|
- final DataNode dn = miniCluster.getDataNodes().get(0);
|
|
|
-
|
|
|
- /* run plan command */
|
|
|
- cmdLine = String.format(
|
|
|
- "hdfs diskbalancer -%s %s",
|
|
|
- PLAN,
|
|
|
- dn.getDatanodeUuid());
|
|
|
- outputs = runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
|
|
|
- /* get path of plan file*/
|
|
|
- final String planFileName = dn.getDatanodeUuid();
|
|
|
-
|
|
|
- /* verify plan command */
|
|
|
- assertEquals(
|
|
|
- "There must be two lines: the 1st is writing plan to...,"
|
|
|
- + " the 2nd is actual full path of plan file.",
|
|
|
- 2, outputs.size());
|
|
|
- assertThat(outputs.get(1), containsString(planFileName));
|
|
|
-
|
|
|
- /* get full path of plan file*/
|
|
|
- final String planFileFullName = outputs.get(1);
|
|
|
+ /* get full path of plan */
|
|
|
+ final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
|
|
|
|
|
|
/* run execute command */
|
|
|
- cmdLine = String.format(
|
|
|
+ final String cmdLine = String.format(
|
|
|
"hdfs diskbalancer -%s %s",
|
|
|
EXECUTE,
|
|
|
planFileFullName);
|
|
|
- outputs = runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+ runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
} finally {
|
|
|
if (miniCluster != null) {
|
|
|
miniCluster.shutdown();
|
|
@@ -159,6 +185,35 @@ public class TestDiskBalancerCommand {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private String runAndVerifyPlan(
|
|
|
+ final MiniDFSCluster miniCluster,
|
|
|
+ final Configuration hdfsConf) throws Exception {
|
|
|
+ String cmdLine = "";
|
|
|
+ List<String> outputs = null;
|
|
|
+ final DataNode dn = miniCluster.getDataNodes().get(0);
|
|
|
+
|
|
|
+ /* run plan command */
|
|
|
+ cmdLine = String.format(
|
|
|
+ "hdfs diskbalancer -%s %s",
|
|
|
+ PLAN,
|
|
|
+ dn.getDatanodeUuid());
|
|
|
+ outputs = runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+
|
|
|
+ /* get path of plan file*/
|
|
|
+ final String planFileName = dn.getDatanodeUuid();
|
|
|
+
|
|
|
+ /* verify plan command */
|
|
|
+ assertEquals(
|
|
|
+ "There must be two lines: the 1st is writing plan to...,"
|
|
|
+ + " the 2nd is actual full path of plan file.",
|
|
|
+ 2, outputs.size());
|
|
|
+ assertThat(outputs.get(1), containsString(planFileName));
|
|
|
+
|
|
|
+ /* get full path of plan file*/
|
|
|
+ final String planFileFullName = outputs.get(1);
|
|
|
+ return planFileFullName;
|
|
|
+ }
|
|
|
+
|
|
|
/* test basic report */
|
|
|
@Test(timeout = 60000)
|
|
|
public void testReportSimple() throws Exception {
|