sumitagrawl commented on code in PR #3782:
URL: https://github.com/apache/ozone/pull/3782#discussion_r988751441


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java:
##########
@@ -0,0 +1,1009 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import 
org.apache.hadoop.hdds.scm.container.replication.LegacyReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.ha.SCMContext;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL;
+import static 
org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT;
+
+/**
+ * Container balancer is a service in SCM to move containers between over- and
+ * under-utilized datanodes.
+ */
+public class ContainerBalancerTask implements Runnable {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ContainerBalancerTask.class);
+
+  private NodeManager nodeManager;
+  private ContainerManager containerManager;
+  private ReplicationManager replicationManager;
+  private OzoneConfiguration ozoneConfiguration;
+  private ContainerBalancer containerBalancer;
+  private final SCMContext scmContext;
+  private double threshold;
+  private int totalNodesInCluster;
+  private double maxDatanodesRatioToInvolvePerIteration;
+  private long maxSizeToMovePerIteration;
+  private int countDatanodesInvolvedPerIteration;
+  private long sizeScheduledForMoveInLatestIteration;
+  // count actual size moved in bytes
+  private long sizeActuallyMovedInLatestIteration;
+  private int iterations;
+  private List<DatanodeUsageInfo> unBalancedNodes;
+  private List<DatanodeUsageInfo> overUtilizedNodes;
+  private List<DatanodeUsageInfo> underUtilizedNodes;
+  private List<DatanodeUsageInfo> withinThresholdUtilizedNodes;
+  private Set<String> excludeNodes;
+  private Set<String> includeNodes;
+  private ContainerBalancerConfiguration config;
+  private ContainerBalancerMetrics metrics;
+  private long clusterCapacity;
+  private long clusterRemaining;
+  private double clusterAvgUtilisation;
+  private PlacementPolicyValidateProxy placementPolicyValidateProxy;
+  private NetworkTopology networkTopology;
+  private double upperLimit;
+  private double lowerLimit;
+  private ContainerBalancerSelectionCriteria selectionCriteria;
+  private volatile Status taskStatus = Status.RUNNING;
+
+  /*
+  Since a container can be selected only once during an iteration, these maps
+   use it as a primary key to track source to target pairings.
+  */
+  private final Map<ContainerID, DatanodeDetails> containerToSourceMap;
+  private final Map<ContainerID, DatanodeDetails> containerToTargetMap;
+
+  private Set<DatanodeDetails> selectedTargets;
+  private Set<DatanodeDetails> selectedSources;
+  private FindTargetStrategy findTargetStrategy;
+  private FindSourceStrategy findSourceStrategy;
+  private Map<ContainerMoveSelection,
+      CompletableFuture<LegacyReplicationManager.MoveResult>>
+      moveSelectionToFutureMap;
+  private IterationResult iterationResult;
+  private int nextIterationIndex;
+
+  /**
+   * Constructs ContainerBalancer with the specified arguments. Initializes
+   * ContainerBalancerMetrics. Container Balancer does not start on
+   * construction.
+   *
+   * @param scm the storage container manager
+   */
+  public ContainerBalancerTask(StorageContainerManager scm,
+                               int nextIterationIndex,
+                               ContainerBalancer containerBalancer,
+                               ContainerBalancerMetrics metrics,
+                               ContainerBalancerConfiguration config) {
+    this.nodeManager = scm.getScmNodeManager();
+    this.containerManager = scm.getContainerManager();
+    this.replicationManager = scm.getReplicationManager();
+    this.ozoneConfiguration = scm.getConfiguration();
+    this.containerBalancer = containerBalancer;
+    this.config = config;
+    this.metrics = metrics;
+    this.scmContext = scm.getScmContext();
+    this.overUtilizedNodes = new ArrayList<>();
+    this.underUtilizedNodes = new ArrayList<>();
+    this.withinThresholdUtilizedNodes = new ArrayList<>();
+    this.unBalancedNodes = new ArrayList<>();
+    this.placementPolicyValidateProxy = scm.getPlacementPolicyValidateProxy();
+    this.networkTopology = scm.getClusterMap();
+    this.nextIterationIndex = nextIterationIndex;
+    this.containerToSourceMap = new HashMap<>();
+    this.containerToTargetMap = new HashMap<>();
+    this.selectedSources = new HashSet<>();
+    this.selectedTargets = new HashSet<>();
+    findSourceStrategy = new FindSourceGreedy(nodeManager);
+  }
+
+  /**
+   * Balances the cluster in iterations. Regularly checks if balancing has
+   * been stopped.
+   */
+  public void run() {
+    try {
+      balancer();
+
+      // mark balancer completion, if ha and stopped as not a leader, it will
+      // be handled inside for not saving configuration with leader check
+      saveConfiguration(config, false, 0);
+    } catch (Throwable e) {
+      LOG.error("Container Balancer is stopped abnormally, ", e);
+    }
+    taskStatus = Status.STOPPED;
+  }
+
+  /**
+   * Tries to stop ContainerBalancer changing status to stopping. Calls
+   * {@link ContainerBalancerTask#stop()}.
+   */
+  public void stop() {
+    taskStatus = Status.STOPPING;
+  }
+
+  private void balancer() {
+    this.iterations = config.getIterations();
+    if (this.iterations == -1) {
+      //run balancer infinitely
+      this.iterations = Integer.MAX_VALUE;
+    }
+
+    // nextIterationIndex is the iteration that balancer should start from on
+    // leader change or restart
+    int i = nextIterationIndex;
+    for (; i < iterations && isBalancerRunning(); i++) {
+      // reset some variables and metrics for this iteration
+      resetState();
+
+      if (!isBalancerRunning()) {
+        return;
+      }
+
+      if (config.getTriggerDuEnable()) {
+        // before starting a new iteration, we trigger all the datanode
+        // to run `du`. this is an aggressive action, with which we can
+        // get more precise usage info of all datanodes before moving.
+        // this is helpful for container balancer to make more appropriate
+        // decisions. this will increase the disk io load of data nodes, so
+        // please enable it with caution.
+        nodeManager.refreshAllHealthyDnUsageInfo();
+        try {
+          long nodeReportInterval =
+              ozoneConfiguration.getTimeDuration(HDDS_NODE_REPORT_INTERVAL,
+                  HDDS_NODE_REPORT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
+          // one for sending command , one for running du, and one for
+          // reporting back make it like this for now, a more suitable
+          // value. can be set in the future if needed
+          wait(3 * nodeReportInterval);

Review Comment:
   > I think it is a requirement to have wait inside synchronized block.
   
   Not finding any usecase to be in synchronized block, since this is single 
run only. Searched all usecase in existing code.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to