You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/08/16 15:03:21 UTC

[GitHub] [ozone] symious commented on a change in pull request #2497: HDDS-5326. EC: Create a new as many racks as possible placement policy for EC

symious commented on a change in pull request #2497:
URL: https://github.com/apache/ozone/pull/2497#discussion_r689617353



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/SCMContainerPlacementRackScatter.java
##########
@@ -0,0 +1,343 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * 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.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+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;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.InnerNode;
+import org.apache.hadoop.hdds.scm.net.NetConstants;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Container placement policy that scatter datanodes on different racks
+ * , together with the space to satisfy the size constraints.
+ * <p>
+ * This placement policy will try to distribute datanodes on as many racks as
+ * possible.
+ * <p>
+ * This implementation applies to network topology like "/rack/node". Don't
+ * recommend to use this if the network topology has more layers.
+ * <p>
+ */
+public final class SCMContainerPlacementRackScatter
+    extends SCMCommonPlacementPolicy {
+  @VisibleForTesting
+  public static final Logger LOG =
+      LoggerFactory.getLogger(SCMContainerPlacementRackScatter.class);
+  private final NetworkTopology networkTopology;
+  private static final int RACK_LEVEL = 1;
+  private static final int MAX_RETRY= 3;
+  private final SCMContainerPlacementMetrics metrics;
+
+  /**
+   * Constructs a Container Placement with rack awareness.
+   *
+   * @param nodeManager Node Manager
+   * @param conf Configuration
+   */
+  public SCMContainerPlacementRackScatter(final NodeManager nodeManager,
+      final ConfigurationSource conf, final NetworkTopology networkTopology,
+      boolean fallback, final SCMContainerPlacementMetrics metrics) {
+    super(nodeManager, conf);
+    this.networkTopology = networkTopology;
+    this.metrics = metrics;
+  }
+
+  /**
+   * Called by SCM to choose datanodes.
+   *
+   *
+   * @param excludedNodes - list of the datanodes to exclude.
+   * @param favoredNodes - list of nodes preferred. This is a hint to the
+   *                     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 dataSizeRequired - size required for the container.
+   * @param metadataSizeRequired - size required for Ratis metadata.
+   * @return List of datanodes.
+   * @throws SCMException  SCMException
+   */
+  @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);
+    int datanodeCount = networkTopology.getNumOfLeafNode(NetConstants.ROOT);
+    int excludedNodesCount = excludedNodes == null ? 0 : excludedNodes.size();
+    if (datanodeCount < nodesRequired + excludedNodesCount) {
+      throw new SCMException("No enough datanodes to choose. " +
+          "TotalNode = " + datanodeCount +
+          " RequiredNode = " + nodesRequired +
+          " ExcludedNode = " + excludedNodesCount, null);
+    }
+
+    List<DatanodeDetails> mutableFavoredNodes = new ArrayList<>();
+    if (favoredNodes != null) {
+      // Generate mutableFavoredNodes, only stores valid favoredNodes
+      for (DatanodeDetails datanodeDetails : favoredNodes) {
+        if (isValidNode(datanodeDetails, metadataSizeRequired,
+            dataSizeRequired)) {
+          mutableFavoredNodes.add(datanodeDetails);
+        }
+      }
+    }
+    if (excludedNodes != null) {
+      mutableFavoredNodes.removeAll(excludedNodes);
+    }
+
+    // For excluded nodes, we sort their racks at rear
+    List<Node> racks = getAllRacks();
+    if (excludedNodes != null && excludedNodes.size() > 0) {
+      racks = sortRackWithExcludedNodes(racks, excludedNodes);
+    }
+
+    List<Node> toChooseRack = new LinkedList<>(racks);
+    List<Node> chosenNodes = new ArrayList<>();
+    List<Node> unavailableNodes = new ArrayList<>();
+    if (excludedNodes != null) {
+      unavailableNodes.addAll(excludedNodes);
+    }
+
+    // If the result doesn't change after retryCount, we return with exception
+    int retryCount = 0;
+    while (nodesRequired > 0) {
+      if (retryCount > MAX_RETRY) {
+        throw new SCMException("No satisfied datanode to meet the" +
+            " excludedNodes and affinityNode constrains.", null);
+      }
+      int chosenListSize = chosenNodes.size();
+
+      if (toChooseRack.size() == 0) {
+        toChooseRack.addAll(racks);

Review comment:
       > Do we need to remove all the racks in chosenNodes here?
   
   I think we don't need to remove the content of chosenNodes. The size of chosenNodes could be used the check if it's a retry or not.
   
   > When do we expect the loop to run again for a retry? Is this related to fallback, or do we just retry a few times if we don't find enough nodes on unique racks?
   
   If the size of chosenNodes doesn't change, we treat it as a retry, and if the size doesn't change for RETRY_MAX times, then we return with exception.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org