You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by um...@apache.org on 2022/05/29 21:18:34 UTC
[ozone] branch master updated: HDDS-6816. SCMContainerPlacementRackScatter should use original required node num to validate placement policy (#3462)
This is an automated email from the ASF dual-hosted git repository.
umamahesh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new b3ed6f403e HDDS-6816. SCMContainerPlacementRackScatter should use original required node num to validate placement policy (#3462)
b3ed6f403e is described below
commit b3ed6f403ef55c3bd3a150e347da2c1068060049
Author: Jie Yao <ja...@tencent.com>
AuthorDate: Mon May 30 05:18:30 2022 +0800
HDDS-6816. SCMContainerPlacementRackScatter should use original required node num to validate placement policy (#3462)
---
.../SCMContainerPlacementRackScatter.java | 28 +++++++++++--------
.../TestSCMContainerPlacementRackScatter.java | 32 ++++++++++++++++++----
2 files changed, 44 insertions(+), 16 deletions(-)
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
index 1902d51a9e..6c5fea1969 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.hdds.scm.container.placement.algorithms;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
@@ -85,7 +84,7 @@ public final class SCMContainerPlacementRackScatter
* allocator, whether the favored nodes will be used
* depends on whether the nodes meets the allocator's
* requirement.
- * @param nodesRequired - number of datanodes required.
+ * @param nodesRequiredToChoose - number of datanodes required.
* @param dataSizeRequired - size required for the container.
* @param metadataSizeRequired - size required for Ratis metadata.
* @return List of datanodes.
@@ -93,11 +92,17 @@ public final class SCMContainerPlacementRackScatter
*/
@Override
public List<DatanodeDetails> chooseDatanodes(
- List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes,
- int nodesRequired, long metadataSizeRequired, long dataSizeRequired)
- throws SCMException {
- Preconditions.checkArgument(nodesRequired > 0);
- metrics.incrDatanodeRequestCount(nodesRequired);
+ final List<DatanodeDetails> excludedNodes,
+ final List<DatanodeDetails> favoredNodes,
+ final int nodesRequiredToChoose, final long metadataSizeRequired,
+ final long dataSizeRequired) throws SCMException {
+ if (nodesRequiredToChoose <= 0) {
+ String errorMsg = "num of nodes required to choose should bigger" +
+ "than 0, but the given num is " + nodesRequiredToChoose;
+ throw new SCMException(errorMsg, null);
+ }
+ metrics.incrDatanodeRequestCount(nodesRequiredToChoose);
+ int nodesRequired = nodesRequiredToChoose;
int excludedNodesCount = excludedNodes == null ? 0 : excludedNodes.size();
List<Node> availableNodes = networkTopology.getNodes(
networkTopology.getMaxLevel());
@@ -215,11 +220,12 @@ public final class SCMContainerPlacementRackScatter
}
}
ContainerPlacementStatus placementStatus =
- validateContainerPlacement(chosenNodes, nodesRequired);
+ validateContainerPlacement(chosenNodes, nodesRequiredToChoose);
if (!placementStatus.isPolicySatisfied()) {
- LOG.warn("ContainerPlacementPolicy not met, currentRacks is {}," +
- " desired racks is {}.", placementStatus.actualPlacementCount(),
- placementStatus.expectedPlacementCount());
+ String errorMsg = "ContainerPlacementPolicy not met, currentRacks is" +
+ placementStatus.actualPlacementCount() + "desired racks is" +
+ placementStatus.expectedPlacementCount();
+ throw new SCMException(errorMsg, null);
}
return chosenNodes;
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
index 1b45a75920..dbb5f06b9e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackScatter.java
@@ -61,6 +61,7 @@ import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.Assume.assumeTrue;
@@ -308,11 +309,32 @@ public class TestSCMContainerPlacementRackScatter {
excludedNodes.clear();
excludedNodes.add(datanodes.get(0));
excludedNodes.add(datanodes.get(5));
- datanodeDetails = policy.chooseDatanodes(
- excludedNodes, null, nodeNum, 0, 15);
- Assert.assertEquals(nodeNum, datanodeDetails.size());
- Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
- Math.min(totalNum, rackNum));
+ if (datanodeCount == 6) {
+ /*
+ * when datanodeCount is 6, the clusterMap will be
+ * /rack0/node0
+ * /rack0/node1
+ * /rack0/node2
+ * /rack0/node3
+ * /rack0/node4
+ * /rack1/node5
+ * if we select node0 and node5 as the excluded datanode,
+ * only datanode in rack0 will be chosen when calling
+ * `policy.chooseDatanodes` and the placement will not be
+ * met since there are two racks exist, but only one
+ * of them is chosen
+ * */
+ SCMException e = assertThrows(SCMException.class,
+ () -> policy.chooseDatanodes(excludedNodes, null, 3, 0, 15));
+ String message = e.getMessage();
+ assumeTrue(message.contains("ContainerPlacementPolicy not met"));
+ } else {
+ datanodeDetails = policy.chooseDatanodes(
+ excludedNodes, null, nodeNum, 0, 15);
+ Assert.assertEquals(nodeNum, datanodeDetails.size());
+ Assert.assertEquals(getRackSize(datanodeDetails, excludedNodes),
+ Math.min(totalNum, rackNum));
+ }
}
@Test
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org