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/11/17 10:32:01 UTC

[GitHub] [ozone] lokeshj1703 commented on a change in pull request #2808: HDDS-5517. Support multiple container moves from a source datanode in one balance iteration

lokeshj1703 commented on a change in pull request #2808:
URL: https://github.com/apache/ozone/pull/2808#discussion_r751104203



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.balancer;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The selection criteria for selecting source datanodes , the containers of
+ * which will be moved out.
+ */
+public class FindSourceGreedy implements FindSourceStrategy{
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FindSourceGreedy.class);
+  private Map<DatanodeDetails, Long> sizeLeavingNode;
+  private List<DatanodeUsageInfo> potentialSources;
+  private NodeManager nodeManager;
+  private ContainerBalancerConfiguration config;
+  private Double lowerLimit;
+
+  FindSourceGreedy(NodeManager nodeManager) {
+    sizeLeavingNode = new HashMap<>();
+    this.nodeManager = nodeManager;
+  }
+
+  private void setLowerLimit(Double lowerLimit) {
+    this.lowerLimit = lowerLimit;
+  }
+
+  private void setPotentialSources(
+      List<DatanodeUsageInfo> potentialSources) {
+    this.potentialSources = potentialSources;
+    sizeLeavingNode.clear();
+    potentialSources.forEach(
+        c -> sizeLeavingNode.put(c.getDatanodeDetails(), 0L));
+  }
+
+  private void setConfiguration(ContainerBalancerConfiguration conf) {
+    this.config = conf;
+  }
+
+  /**
+   * increase the Leaving size of a candidate source data node.
+   */
+  @Override
+  public void increaseSizeLeaving(DatanodeDetails dui, long size) {
+    Long currentSize = sizeLeavingNode.get(dui);
+    if(currentSize != null) {
+      sizeLeavingNode.put(dui, currentSize + size);
+      return;
+    }
+    LOG.warn("Cannot find datanode {} in candidate source datanodes",
+        dui.getUuid());
+  }
+
+  /**
+   * get the next candidate source data node according to
+   * the strategy.
+   *
+   * @return the nex candidate source data node.
+   */
+  @Override
+  public DatanodeDetails getNextCandidateSourceDataNode() {
+    if (potentialSources.isEmpty()) {
+      LOG.info("no more candidate source data node");
+      return null;
+    }
+    //TODOļ¼šuse a more quick data structure, which will hava a
+    // better performance when changing or deleting one element at once
+    potentialSources.sort((a, b) -> {

Review comment:
       We can probably use a priority queue here or a TreeSet. TreeSet might be better.

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
##########
@@ -518,14 +475,31 @@ private ContainerMoveSelection matchSourceWithTarget(
       }
       return null;
     }
+
+    //if the utilization of the source data node becomes lower than lowerLimit
+    //after the container is moved out , then the container can not be
+    // a candidate one, and we should remove it from the candidateContainers.
+    candidateContainers.removeIf(c -> {
+      ContainerInfo cInfo;
+      try {
+        cInfo = containerManager.getContainer(c);
+      } catch (ContainerNotFoundException e) {
+        LOG.warn("Could not find container {} when " +
+            "be matched with a move target", c);
+        //remove this not found container
+        return true;
+      }
+      return !findSourceStrategy.canSizeLeaveSource(
+          source, cInfo.getUsedBytes());
+    });

Review comment:
       Can we move this to selection criteria#getCandidateContainers? Maybe selection criteria can have reference to find source strategy.

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
##########
@@ -341,99 +341,59 @@ private boolean initializeIteration() {
         nodeManager, replicationManager, containerManager);
     sourceToTargetMap = new HashMap<>(overUtilizedNodes.size() +
         withinThresholdUtilizedNodes.size());
-
-    // initialize maps to track how much size is leaving and entering datanodes
-    sizeLeavingNode = new HashMap<>(overUtilizedNodes.size() +
-        withinThresholdUtilizedNodes.size());
-    overUtilizedNodes.forEach(datanodeUsageInfo -> sizeLeavingNode
-        .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
-    withinThresholdUtilizedNodes.forEach(datanodeUsageInfo -> sizeLeavingNode
-        .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
-
-    sizeEnteringNode = new HashMap<>(underUtilizedNodes.size() +
-        withinThresholdUtilizedNodes.size());
-    underUtilizedNodes.forEach(datanodeUsageInfo -> sizeEnteringNode
-        .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
-    withinThresholdUtilizedNodes.forEach(datanodeUsageInfo -> sizeEnteringNode
-        .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
-
     return true;
   }
 
   private IterationResult doIteration() {
     // note that potential and selected targets are updated in the following
     // loop
-    List<DatanodeDetails> potentialTargets = getPotentialTargets();
+    //TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both
+    // source and target
+    findSourceStrategy.reInitialize(getPotentialSources(), config, lowerLimit);
+    List<DatanodeUsageInfo> potentialTargets = getPotentialTargets();
+    findTargetStrategy.reInitialize(potentialTargets, config, upperLimit);

Review comment:
       Can we avoid calling reinitialize in every iteration?




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