|
@@ -5,9 +5,9 @@
|
|
* licenses this file to you under the Apache License, Version 2.0 (the
|
|
* licenses this file to you under the Apache License, Version 2.0 (the
|
|
* "License"); you may not use this file except in compliance with the License.
|
|
* "License"); you may not use this file except in compliance with the License.
|
|
* You may obtain a copy of the License at
|
|
* You may obtain a copy of the License at
|
|
- * <p/>
|
|
|
|
|
|
+ *
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
- * <p/>
|
|
|
|
|
|
+ *
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
|
@@ -37,6 +37,7 @@ import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.junit.rules.ExpectedException;
|
|
import org.junit.rules.ExpectedException;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
|
|
import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
|
|
import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
|
|
import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
|
|
import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
|
|
|
|
|
|
@@ -63,103 +64,163 @@ public class TestDiskBalancerRPC {
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
- public void testSubmitTestRpc() throws Exception {
|
|
|
|
- final int dnIndex = 0;
|
|
|
|
- cluster.restartDataNode(dnIndex);
|
|
|
|
- cluster.waitActive();
|
|
|
|
- ClusterConnector nameNodeConnector =
|
|
|
|
- ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
|
|
|
|
|
|
+ public void testSubmitPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+ dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
+ }
|
|
|
|
|
|
- DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
|
|
|
|
- diskBalancerCluster.readClusterInfo();
|
|
|
|
- Assert.assertEquals(cluster.getDataNodes().size(),
|
|
|
|
- diskBalancerCluster.getNodes().size());
|
|
|
|
- diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testSubmitPlanWithInvalidHash() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ char hashArray[] = planHash.toCharArray();
|
|
|
|
+ hashArray[0]++;
|
|
|
|
+ planHash = String.valueOf(hashArray);
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+ thrown.expect(DiskBalancerException.class);
|
|
|
|
+ dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
+ }
|
|
|
|
|
|
- DataNode dataNode = cluster.getDataNodes().get(dnIndex);
|
|
|
|
- DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
|
|
|
|
- dataNode.getDatanodeUuid());
|
|
|
|
- GreedyPlanner planner = new GreedyPlanner(10.0f, node);
|
|
|
|
- NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
|
|
|
|
- ());
|
|
|
|
- planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
|
|
|
|
- final int planVersion = 1; // So far we support only one version.
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testSubmitPlanWithInvalidVersion() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ planVersion++;
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+ thrown.expect(DiskBalancerException.class);
|
|
|
|
+ dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
+ }
|
|
|
|
|
|
- String planHash = DigestUtils.sha512Hex(plan.toJson());
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testSubmitPlanWithInvalidPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+ thrown.expect(DiskBalancerException.class);
|
|
|
|
+ dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, "");
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testCancelPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
+ dataNode.cancelDiskBalancePlan(planHash);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
- public void testCancelTestRpc() throws Exception {
|
|
|
|
- final int dnIndex = 0;
|
|
|
|
- cluster.restartDataNode(dnIndex);
|
|
|
|
- cluster.waitActive();
|
|
|
|
- ClusterConnector nameNodeConnector =
|
|
|
|
- ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
|
|
|
|
-
|
|
|
|
- DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
|
|
|
|
- diskBalancerCluster.readClusterInfo();
|
|
|
|
- Assert.assertEquals(cluster.getDataNodes().size(),
|
|
|
|
- diskBalancerCluster.getNodes().size());
|
|
|
|
- diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
|
|
|
|
- DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
|
|
|
|
- GreedyPlanner planner = new GreedyPlanner(10.0f, node);
|
|
|
|
- NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
|
|
|
|
- ());
|
|
|
|
- planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
|
|
|
|
-
|
|
|
|
- final int planVersion = 0; // So far we support only one version.
|
|
|
|
- DataNode dataNode = cluster.getDataNodes().get(dnIndex);
|
|
|
|
- String planHash = DigestUtils.sha512Hex(plan.toJson());
|
|
|
|
-
|
|
|
|
- // Since submitDiskBalancerPlan is not implemented yet, it throws an
|
|
|
|
- // Exception, this will be modified with the actual implementation.
|
|
|
|
- try {
|
|
|
|
- dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
- } catch (DiskBalancerException ex) {
|
|
|
|
- // Let us ignore this for time being.
|
|
|
|
- }
|
|
|
|
|
|
+ public void testCancelNonExistentPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ char hashArray[] = planHash.toCharArray();
|
|
|
|
+ hashArray[0]++;
|
|
|
|
+ planHash = String.valueOf(hashArray);
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
thrown.expect(DiskBalancerException.class);
|
|
thrown.expect(DiskBalancerException.class);
|
|
dataNode.cancelDiskBalancePlan(planHash);
|
|
dataNode.cancelDiskBalancePlan(planHash);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
- public void testQueryTestRpc() throws Exception {
|
|
|
|
- final int dnIndex = 0;
|
|
|
|
- cluster.restartDataNode(dnIndex);
|
|
|
|
- cluster.waitActive();
|
|
|
|
- ClusterConnector nameNodeConnector =
|
|
|
|
- ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
|
|
|
|
-
|
|
|
|
- DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
|
|
|
|
- (nameNodeConnector);
|
|
|
|
- diskBalancerCluster.readClusterInfo();
|
|
|
|
- Assert.assertEquals(cluster.getDataNodes().size(),
|
|
|
|
- diskBalancerCluster.getNodes().size());
|
|
|
|
- diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
|
|
|
|
- DataNode dataNode = cluster.getDataNodes().get(dnIndex);
|
|
|
|
- DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
|
|
|
|
- dataNode.getDatanodeUuid());
|
|
|
|
- GreedyPlanner planner = new GreedyPlanner(10.0f, node);
|
|
|
|
- NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
|
|
|
|
- ());
|
|
|
|
- planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
|
|
|
|
-
|
|
|
|
- final int planVersion = 1; // So far we support only one version.
|
|
|
|
- String planHash = DigestUtils.sha512Hex(plan.toJson());
|
|
|
|
- dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
|
|
|
|
+ public void testCancelEmptyPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = "";
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+ thrown.expect(DiskBalancerException.class);
|
|
|
|
+ dataNode.cancelDiskBalancePlan(planHash);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testQueryPlan() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+ String planHash = rpcTestHelper.getPlanHash();
|
|
|
|
+ int planVersion = rpcTestHelper.getPlanVersion();
|
|
|
|
+ NodePlan plan = rpcTestHelper.getPlan();
|
|
|
|
+
|
|
|
|
+ dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
|
|
DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
|
|
DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
|
|
Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
|
|
Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
|
|
status.getResult() == PLAN_DONE);
|
|
status.getResult() == PLAN_DONE);
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
- public void testgetDiskBalancerSetting() throws Exception {
|
|
|
|
|
|
+ public void testQueryPlanWithoutSubmit() throws Exception {
|
|
|
|
+ RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
|
|
|
|
+ DataNode dataNode = rpcTestHelper.getDataNode();
|
|
|
|
+
|
|
|
|
+ DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
|
|
|
|
+ Assert.assertTrue(status.getResult() == NO_PLAN);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetDiskBalancerSetting() throws Exception {
|
|
final int dnIndex = 0;
|
|
final int dnIndex = 0;
|
|
DataNode dataNode = cluster.getDataNodes().get(dnIndex);
|
|
DataNode dataNode = cluster.getDataNodes().get(dnIndex);
|
|
thrown.expect(DiskBalancerException.class);
|
|
thrown.expect(DiskBalancerException.class);
|
|
dataNode.getDiskBalancerSetting(
|
|
dataNode.getDiskBalancerSetting(
|
|
DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
|
|
DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private class RpcTestHelper {
|
|
|
|
+ private NodePlan plan;
|
|
|
|
+ private int planVersion;
|
|
|
|
+ private DataNode dataNode;
|
|
|
|
+ private String planHash;
|
|
|
|
+
|
|
|
|
+ public NodePlan getPlan() {
|
|
|
|
+ return plan;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public int getPlanVersion() {
|
|
|
|
+ return planVersion;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public DataNode getDataNode() {
|
|
|
|
+ return dataNode;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public String getPlanHash() {
|
|
|
|
+ return planHash;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public RpcTestHelper invoke() throws Exception {
|
|
|
|
+ final int dnIndex = 0;
|
|
|
|
+ cluster.restartDataNode(dnIndex);
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ ClusterConnector nameNodeConnector =
|
|
|
|
+ ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
|
|
|
|
+
|
|
|
|
+ DiskBalancerCluster diskBalancerCluster =
|
|
|
|
+ new DiskBalancerCluster(nameNodeConnector);
|
|
|
|
+ diskBalancerCluster.readClusterInfo();
|
|
|
|
+ Assert.assertEquals(cluster.getDataNodes().size(),
|
|
|
|
+ diskBalancerCluster.getNodes().size());
|
|
|
|
+ diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
|
|
|
|
+ dataNode = cluster.getDataNodes().get(dnIndex);
|
|
|
|
+ DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
|
|
|
|
+ dataNode.getDatanodeUuid());
|
|
|
|
+ GreedyPlanner planner = new GreedyPlanner(10.0f, node);
|
|
|
|
+ plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort());
|
|
|
|
+ planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
|
|
|
|
+ planVersion = 1;
|
|
|
|
+ planHash = DigestUtils.sha512Hex(plan.toJson());
|
|
|
|
+ return this;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|