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 13:13:35 UTC

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

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



##########
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);
+      }
+
+      if (mutableFavoredNodes.size() > 0) {
+        List<Node> chosenFavoredNodesInForLoop = new ArrayList<>();
+        for (DatanodeDetails favoredNode : mutableFavoredNodes) {
+          Node curRack = getRackOfDatanodeDetails(favoredNode);
+          if (toChooseRack.contains(curRack)) {
+            chosenNodes.add(favoredNode);
+            toChooseRack.remove(curRack);
+            chosenFavoredNodesInForLoop.add(favoredNode);
+            unavailableNodes.add(favoredNode);
+            nodesRequired--;
+            if (nodesRequired == 0) {
+              break;
+            }
+          }
+        }
+        mutableFavoredNodes.removeAll(chosenFavoredNodesInForLoop);
+      }
+
+      // If satisfied by favored nodes, return then.
+      if (nodesRequired == 0) {
+        break;
+      }
+
+      List<Node> chosenRacksInForLoop = new ArrayList<>();
+      for (Node rack : toChooseRack) {
+        if (((InnerNode)rack).getNodes(2).size() > 0) {
+          Node affinityNode = ((InnerNode)rack).getNodes(2).get(0);
+          Node node = chooseNode(unavailableNodes, affinityNode,
+              metadataSizeRequired, dataSizeRequired);
+          if (node != null) {
+            chosenNodes.add(node);
+            mutableFavoredNodes.remove(node);
+            unavailableNodes.add(node);
+            nodesRequired--;
+            if (nodesRequired == 0) {
+              break;
+            }
+          }
+        }
+        chosenRacksInForLoop.add(rack);
+      }
+      toChooseRack.removeAll(chosenRacksInForLoop);
+      if (chosenListSize == chosenNodes.size()) {
+        retryCount++;
+      } else {
+        retryCount = 0;
+      }
+    }
+
+    return Arrays.asList(chosenNodes.toArray(new DatanodeDetails[0]));
+  }
+
+  @Override
+  public DatanodeDetails chooseNode(List<DatanodeDetails> healthyNodes) {
+    return null;
+  }
+
+  /**
+   * Choose a datanode which meets the requirements. If there is no node which
+   * meets all the requirements, there is fallback chosen process depending on
+   * whether fallback is allowed when this class is instantiated.
+   *
+   *
+   * @param excludedNodes - list of the datanodes to excluded. Can be null.
+   * @param affinityNode - the chosen nodes should be on the same rack as
+   *                    affinityNode. Can be null.
+   * @param dataSizeRequired - size required for the container.
+   * @param metadataSizeRequired - size required for Ratis metadata.
+   * @return List of chosen datanodes.
+   * @throws SCMException  SCMException
+   */
+  private Node chooseNode(List<Node> excludedNodes, Node affinityNode,
+      long metadataSizeRequired, long dataSizeRequired) throws SCMException {
+    int ancestorGen = RACK_LEVEL;
+    int maxRetry = MAX_RETRY;
+    List<String> excludedNodesForCapacity = null;
+    while(true) {
+      metrics.incrDatanodeChooseAttemptCount();
+      Node node = networkTopology.chooseRandom(NetConstants.ROOT,
+          excludedNodesForCapacity, excludedNodes, affinityNode, ancestorGen);
+      if (node == null) {
+        // cannot find the node which meets all constrains
+        LOG.warn("Failed to find the datanode for container. excludedNodes:" +
+            (excludedNodes == null ? "" : excludedNodes.toString()) +
+            ", affinityNode:" +
+            (affinityNode == null ? "" : affinityNode.getNetworkFullPath()));
+        return null;
+      }
+
+      DatanodeDetails datanodeDetails = (DatanodeDetails)node;
+      if (isValidNode(datanodeDetails, metadataSizeRequired,
+          dataSizeRequired)) {
+        return node;
+      }
+
+      maxRetry--;
+      if (maxRetry == 0) {
+        // avoid the infinite loop
+        String errMsg = "No satisfied datanode to meet the space constrains. "
+            + "metadatadata size required: " + metadataSizeRequired +
+            " data size required: " + dataSizeRequired;
+        LOG.info(errMsg);
+        return null;
+      }
+      if (excludedNodesForCapacity == null) {
+        excludedNodesForCapacity = new ArrayList<>();
+      }
+      excludedNodesForCapacity.add(node.getNetworkFullPath());
+    }
+  }
+
+  /**
+   * This default implementation handles rack aware policies and non rack
+   * aware policies. If a future placement policy needs to check more than racks
+   * to validate the policy (eg node groups, HDFS like upgrade domain) this
+   * method should be overridden in the sub class.
+   * This method requires that subclasses which implement rack aware policies
+   * override the default method getRequiredRackCount and getNetworkTopology.
+   * @param dns List of datanodes holding a replica of the container
+   * @param replicas The expected number of replicas
+   * @return ContainerPlacementStatus indicating if the placement policy is
+   *         met or not. Not this only considers the rack count and not the
+   *         number of replicas.
+   */
+  @Override

Review comment:
       This method is almost a duplicate of the one in SCMCommonPlacementPolicy. My intention of that implementation was that the method getRequiredRackCount() should be overridden in the sub-classes, and hopefully then this method does not need to be overridden. However getRequiredRackCount() does not work for EC, as we need to pass the number of replicas to it.
   
   I think it would be better, if we changed `SCMCommonPlacementPolicy.getRequiredRackCount()` to accept a parameter `int numReplicas`, and then the sub-classes can ignore or use that. That means in this class, the implementation would be:
   
   ```
   @Override
   protected int getRequiredRackCount(int expectedReplicas) {
       return expectedReplicas;
     }
   ```
   
   Do you think that makes sense?




-- 
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