|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.diskbalancer.command;
|
|
|
|
|
|
|
|
|
+import static java.lang.Thread.sleep;
|
|
|
import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.CANCEL;
|
|
|
import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.EXECUTE;
|
|
|
import static org.apache.hadoop.hdfs.tools.DiskBalancerCLI.HELP;
|
|
@@ -58,6 +59,7 @@ 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.LambdaTestUtils;
|
|
|
import org.apache.hadoop.test.PathUtils;
|
|
|
import org.apache.hadoop.util.Tool;
|
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
@@ -187,6 +189,132 @@ public class TestDiskBalancerCommand {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+
|
|
|
+ @Test(timeout = 600000)
|
|
|
+ public void testDiskBalancerExecuteOptionPlanValidityWithException() throws
|
|
|
+ Exception {
|
|
|
+ final int numDatanodes = 1;
|
|
|
+
|
|
|
+ final Configuration hdfsConf = new HdfsConfiguration();
|
|
|
+ hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
|
|
|
+ hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "0d");
|
|
|
+
|
|
|
+ /* new cluster with imbalanced capacity */
|
|
|
+ final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
|
|
|
+ newImbalancedCluster(
|
|
|
+ hdfsConf,
|
|
|
+ numDatanodes,
|
|
|
+ CAPACITIES,
|
|
|
+ DEFAULT_BLOCK_SIZE,
|
|
|
+ FILE_LEN);
|
|
|
+
|
|
|
+ try {
|
|
|
+ /* get full path of plan */
|
|
|
+ final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
|
|
|
+
|
|
|
+ /* run execute command */
|
|
|
+ final String cmdLine = String.format(
|
|
|
+ "hdfs diskbalancer -%s %s",
|
|
|
+ EXECUTE,
|
|
|
+ planFileFullName);
|
|
|
+
|
|
|
+ LambdaTestUtils.intercept(
|
|
|
+ RemoteException.class,
|
|
|
+ "DiskBalancerException",
|
|
|
+ "Plan was generated more than 0d ago",
|
|
|
+ () -> {
|
|
|
+ runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+ });
|
|
|
+ } finally{
|
|
|
+ if (miniCluster != null) {
|
|
|
+ miniCluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout = 600000)
|
|
|
+ public void testDiskBalancerExecutePlanValidityWithOutUnitException()
|
|
|
+ throws
|
|
|
+ Exception {
|
|
|
+ final int numDatanodes = 1;
|
|
|
+
|
|
|
+ final Configuration hdfsConf = new HdfsConfiguration();
|
|
|
+ hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
|
|
|
+ hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "0");
|
|
|
+
|
|
|
+ /* new cluster with imbalanced capacity */
|
|
|
+ final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
|
|
|
+ newImbalancedCluster(
|
|
|
+ hdfsConf,
|
|
|
+ numDatanodes,
|
|
|
+ CAPACITIES,
|
|
|
+ DEFAULT_BLOCK_SIZE,
|
|
|
+ FILE_LEN);
|
|
|
+
|
|
|
+ try {
|
|
|
+ /* get full path of plan */
|
|
|
+ final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
|
|
|
+
|
|
|
+ /* run execute command */
|
|
|
+ final String cmdLine = String.format(
|
|
|
+ "hdfs diskbalancer -%s %s",
|
|
|
+ EXECUTE,
|
|
|
+ planFileFullName);
|
|
|
+
|
|
|
+ LambdaTestUtils.intercept(
|
|
|
+ RemoteException.class,
|
|
|
+ "DiskBalancerException",
|
|
|
+ "Plan was generated more than 0ms ago",
|
|
|
+ () -> {
|
|
|
+ runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+ });
|
|
|
+ } finally{
|
|
|
+ if (miniCluster != null) {
|
|
|
+ miniCluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ @Test(timeout = 600000)
|
|
|
+ public void testDiskBalancerExecuteOptionPlanValidity() throws Exception {
|
|
|
+ final int numDatanodes = 1;
|
|
|
+
|
|
|
+ final Configuration hdfsConf = new HdfsConfiguration();
|
|
|
+ hdfsConf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
|
|
|
+ hdfsConf.set(DFSConfigKeys.DFS_DISK_BALANCER_PLAN_VALID_INTERVAL, "600s");
|
|
|
+
|
|
|
+ /* new cluster with imbalanced capacity */
|
|
|
+ final MiniDFSCluster miniCluster = DiskBalancerTestUtil.
|
|
|
+ newImbalancedCluster(
|
|
|
+ hdfsConf,
|
|
|
+ numDatanodes,
|
|
|
+ CAPACITIES,
|
|
|
+ DEFAULT_BLOCK_SIZE,
|
|
|
+ FILE_LEN);
|
|
|
+
|
|
|
+ try {
|
|
|
+ /* get full path of plan */
|
|
|
+ final String planFileFullName = runAndVerifyPlan(miniCluster, hdfsConf);
|
|
|
+
|
|
|
+ /* run execute command */
|
|
|
+ final String cmdLine = String.format(
|
|
|
+ "hdfs diskbalancer -%s %s",
|
|
|
+ EXECUTE,
|
|
|
+ planFileFullName);
|
|
|
+
|
|
|
+ // Plan is valid for 600 seconds, sleeping for 10seconds, so now
|
|
|
+ // diskbalancer should execute the plan
|
|
|
+ sleep(10000);
|
|
|
+ runCommand(cmdLine, hdfsConf, miniCluster);
|
|
|
+ } finally{
|
|
|
+ if (miniCluster != null) {
|
|
|
+ miniCluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private String runAndVerifyPlan(
|
|
|
final MiniDFSCluster miniCluster,
|
|
|
final Configuration hdfsConf) throws Exception {
|