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 2022/09/19 06:49:36 UTC

[GitHub] [ozone] symious opened a new pull request, #3760: HDDS-7233. Add DiskBalancerService on Datanode

symious opened a new pull request, #3760:
URL: https://github.com/apache/ozone/pull/3760

   ## What changes were proposed in this pull request?
   
   This ticket is to add the DiskBalancerService on Datanode to do the real balancer work.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7233
   
   ## How was this patch tested?
   
   unit test.
   


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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r992188689


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {

Review Comment:
   Could we define the algorithm in the document around how the balancing would happen?
   I think the limits in the algorithm are defined by the bandwidth and parallel threads.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991864347


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/DiskBalancerConfiguration.java:
##########
@@ -38,6 +38,10 @@ public final class DiskBalancerConfiguration {
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancerConfiguration.class);
 
+  // The path where datanode diskBalancer's conf is to be written to.
+  public static final String HDDS_DATANODE_DISK_BALANCER_INFO_DIR =
+      "hdds.datanode.disk.balancer.info.dir";
+

Review Comment:
   Can we use the same format as configs below?



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


[GitHub] [ozone] symious commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1617109501

   This PR has been split to https://github.com/apache/ozone/pull/4887 and https://github.com/apache/ozone/pull/3874, so I'm closing this one.


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


[GitHub] [ozone] ferhui commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r983291016


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private long nextAvailableTime = Time.monotonicNow();
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bandwidth = bandwidthInMB * megaByte / 1000f;

Review Comment:
   my fault, you are correct.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r990921893


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   I think in that case we do not need the requeue functionality. Requeue itself is a complex mechanism because it will keep occurring in a loop.
   Why aren't we blocking the operations on container lock instead of requeue mechanism? Is it because the handler thread is getting blocked in that case?



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r992190429


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+          break;
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime.get()) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bytesPerMillisec = bandwidthInMB * megaByte / 1000f;
+    nextAvailableTime.set(Time.monotonicNow() +
+        ((long) (bytesBalanced / bytesPerMillisec)));
+    return false;
+  }
+
+  private class DiskBalancerTask implements BackgroundTask {
+
+    private HddsVolume sourceVolume;
+    private HddsVolume destVolume;
+    private ContainerData containerData;
+    private long containerId;
+
+    DiskBalancerTask(ContainerData containerData,
+        HddsVolume sourceVolume, HddsVolume destVolume) {
+      this.containerData = containerData;
+      this.sourceVolume = sourceVolume;
+      this.destVolume = destVolume;
+      this.containerId = containerData.getContainerID();
+    }
+
+    @Override
+    public BackgroundTaskResult call() {

Review Comment:
   I would try looking for an appropriate class for this. I think the move logic should not be present in the disk balancer and there should be separate implementation for the different container versions.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r992188335


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   I mean we should have only one task involving a particular volume at a time. Multiple tasks could also lead to consistency issues since deltaSizes would be updated by two tasks. Also deltaSizes parameter would not be needed in that case.
   
   Regarding splitting the PR, I think we could have multiple PRs -:
   1. Adding policies on containers and volumes
   2. Supporting cross volume move in Datanode
   3. Configuration and start/stop balancer
   4. Algorithm for balancer
   
   It would be easier to review and also to add tests for every component this way.



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


[GitHub] [ozone] kerneltime commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "kerneltime (via GitHub)" <gi...@apache.org>.
kerneltime commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1418557384

   @symious can you please rebase, your branch is 400+ commits behind master.


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


[GitHub] [ozone] ferhui commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1257529230

   @xichen01 could you please review this PR?


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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991895254


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   I will change the PR later to abandon the requests.
   
   > Can we also update the docs once the PR is merged?
   
   Sure.
   



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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991982144


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/DiskBalancerConfiguration.java:
##########
@@ -38,6 +38,10 @@ public final class DiskBalancerConfiguration {
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancerConfiguration.class);
 
+  // The path where datanode diskBalancer's conf is to be written to.
+  public static final String HDDS_DATANODE_DISK_BALANCER_INFO_DIR =
+      "hdds.datanode.disk.balancer.info.dir";
+

Review Comment:
   It will be a path like "ozone.metadata.dirs", the default path will not be a common value.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r984232130


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java:
##########
@@ -280,4 +280,8 @@ private HddsConfigKeys() {
 
   public static final String OZONE_AUDIT_LOG_DEBUG_CMD_LIST_DNAUDIT =
       "ozone.audit.log.debug.cmd.list.dnaudit";
+
+  // The path where datanode diskBalancer's conf is to be written to.
+  public static final String HDDS_DATANODE_DISK_BALANCER_INFO_DIR =
+      "hdds.datanode.disk.balancer.info.dir";

Review Comment:
   Can this config be moved to DiskBalancerConfiguration?



##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -174,6 +174,13 @@
       The value should be between 0-1. Such as 0.1 which means 10% of volume space will be reserved.
     </description>
   </property>
+  <property>
+    <name>hdds.datanode.disk.balancer.info.dir</name>
+    <value/>
+    <tag>STORAGE, DISK_BALANCER</tag>
+    <description>The file to persist latest DiskBalancer information.
+    </description>
+  </property>

Review Comment:
   We can probably remove this change if config is moved to DiskBalancerConfiguration



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+          break;
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime.get()) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bytesPerMillisec = bandwidthInMB * megaByte / 1000f;
+    nextAvailableTime.set(Time.monotonicNow() +
+        ((long) (bytesBalanced / bytesPerMillisec)));
+    return false;
+  }
+
+  private class DiskBalancerTask implements BackgroundTask {
+
+    private HddsVolume sourceVolume;
+    private HddsVolume destVolume;
+    private ContainerData containerData;
+    private long containerId;
+
+    DiskBalancerTask(ContainerData containerData,
+        HddsVolume sourceVolume, HddsVolume destVolume) {
+      this.containerData = containerData;
+      this.sourceVolume = sourceVolume;
+      this.destVolume = destVolume;
+      this.containerId = containerData.getContainerID();
+    }
+
+    @Override
+    public BackgroundTaskResult call() {
+      boolean destVolumeIncreased = false;
+      Path diskBalancerTmpDir = null, diskBalancerDestDir = null;
+      long containerSize = containerData.getBytesUsed();
+      try {
+        diskBalancerTmpDir = Paths.get(destVolume.getTmpDir().getPath())
+            .resolve(DISK_BALANCER_DIR).resolve(String.valueOf(containerId));
+
+        // Copy container to new Volume's tmp Dir
+        ozoneContainer.getController().copyContainer(
+            containerData.getContainerType(),
+            containerData.getContainerID(), diskBalancerTmpDir);
+
+        // Move container directory to final place on new volume
+        String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
+            destVolume, destVolume.getClusterID());
+        diskBalancerDestDir =
+            Paths.get(KeyValueContainerLocationUtil.getBaseContainerLocation(
+                destVolume.getHddsRootDir().toString(), idDir,
+                containerData.getContainerID()));
+        Path destDirParent = diskBalancerDestDir.getParent();
+        if (destDirParent != null) {
+          Files.createDirectories(destDirParent);
+        }
+        Files.move(diskBalancerTmpDir, diskBalancerDestDir,
+            StandardCopyOption.ATOMIC_MOVE,
+            StandardCopyOption.REPLACE_EXISTING);
+
+        // Generate a new Container based on destDir
+        File containerFile = ContainerUtils.getContainerFile(
+            diskBalancerDestDir.toFile());
+        if (!containerFile.exists()) {
+          throw new IOException("ContainerFile for container " + containerId
+              + " doesn't exists.");
+        }
+        ContainerData originalContainerData = ContainerDataYaml
+            .readContainerFile(containerFile);
+        Container newContainer = ozoneContainer.getController()
+            .importContainer(originalContainerData, diskBalancerDestDir);
+        newContainer.getContainerData().getVolume()
+            .incrementUsedSpace(containerSize);

Review Comment:
   Do we need this? If we are copying the entire container contents, these should be copied as well?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java:
##########
@@ -112,6 +112,29 @@ public boolean addContainer(Container<?> container) throws
     }
   }
 
+  /**
+   * Update Container to container map.
+   * @param container container to be added
+   * @return If container is added to containerMap returns true, otherwise
+   * false
+   */
+  public Container updateContainer(Container<?> container) throws
+      StorageContainerException {
+    Preconditions.checkNotNull(container, "container cannot be null");
+
+    long containerId = container.getContainerData().getContainerID();
+    if (!containerMap.containsKey(containerId)) {
+      LOG.error("Container doesn't exists with container Id {}", containerId);
+      throw new StorageContainerException("Container doesn't exist with " +
+          "container Id " + containerId,
+          ContainerProtos.Result.CONTAINER_NOT_FOUND);
+    } else {
+      LOG.debug("Container with container Id {} is updated to containerMap",
+            containerId);

Review Comment:
   NIT: enclose in if (LOG.isDebugEnabled()) check.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java:
##########
@@ -112,6 +112,10 @@ public void handle(SCMCommand command, OzoneContainer container,
           SCMCommandProto.Type.deleteBlocksCommand, command.getType());
       return;
     }
+    if (shouldRequeue(command, context, container)) {
+      context.requeueCommand(command);
+      return;
+    }

Review Comment:
   For DeleteBlocks we retry the command from SCM. I am not sure if we need a requeue here.
   I think we can probably send failure to SCM in this case and add a failure message in a later PR?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeConfiguration.java:
##########
@@ -297,6 +297,78 @@ public void setWaitOnAllFollowers(boolean val) {
   )
   private String containerSchemaV3KeySeparator = "|";
 
+  @Config(key = "disk.balancer.should.run.default",
+      defaultValue = "false",
+      type = ConfigType.BOOLEAN,
+      tags = { DATANODE, ConfigTag.DISKBALANCER},
+      description =
+          "Refresh Time interval of the Datanode DiskBalancer service. " +
+              "The Datanode will check the service periodically and update " +
+              "the config and running status for DiskBalancer service. " +
+              "Unit could be defined with postfix (ns,ms,s,m,h,d). "

Review Comment:
   Need to change the message.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/HddsVolumeUtil.java:
##########
@@ -122,4 +129,28 @@ private static void mapDbVolumesToDataVolumesIfNeeded(
         hddsVolume.setDbVolume(globalDbVolumeMap.getOrDefault(
             hddsVolume.getStorageID(), null)));
   }
+
+  /**
+   * Get the HddsVolume according to the path.
+   * @param volumes volume list to match from
+   * @param pathStr path to match
+   */
+  public static HddsVolume matchHddsVolume(List<HddsVolume> volumes,
+      String pathStr) {
+    assert pathStr != null;
+    List<HddsVolume> resList = new ArrayList<>();
+    for (HddsVolume hddsVolume: volumes) {
+      if (pathStr.startsWith(hddsVolume.getVolumeRootDir())) {
+        resList.add(hddsVolume);
+      }
+    }
+    if (resList.size() == 1) {
+      return resList.get(0);
+    } else if (resList.size() > 1) {
+      LOG.warn("Get multi volumes {} matching path {}",
+          resList.stream().map(StorageVolume::getVolumeRootDir).collect(
+              Collectors.joining(",")), pathStr);

Review Comment:
   I think it is better to throw an exception in this case.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   How are we making sure same volume is not part of different tasks?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   I think new tasks should not be added until the older tasks finish for better consistency.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   Also can we add policy for choosing source, target volumes? Policy for choosing which container to move between those volumes? We can probably do it in a separate PR.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   Can we check if DeleteContainer request is retried from SCM? I think it is retried.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {

Review Comment:
   It would be better to have one task per pair of volumes.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {

Review Comment:
   There should be a limit on how many containers/data we want to move per volume in a single interval.
   Also we do not want parallel tasks trying to move containers between same pair of volumes.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+          break;
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime.get()) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bytesPerMillisec = bandwidthInMB * megaByte / 1000f;
+    nextAvailableTime.set(Time.monotonicNow() +
+        ((long) (bytesBalanced / bytesPerMillisec)));
+    return false;
+  }
+
+  private class DiskBalancerTask implements BackgroundTask {
+
+    private HddsVolume sourceVolume;
+    private HddsVolume destVolume;
+    private ContainerData containerData;
+    private long containerId;
+
+    DiskBalancerTask(ContainerData containerData,
+        HddsVolume sourceVolume, HddsVolume destVolume) {
+      this.containerData = containerData;
+      this.sourceVolume = sourceVolume;
+      this.destVolume = destVolume;
+      this.containerId = containerData.getContainerID();
+    }
+
+    @Override
+    public BackgroundTaskResult call() {

Review Comment:
   Can we simplify this function and probably move it to utils or make it part of KeyValueContainer.. class itself? I think we should just have an API call here.



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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r1250030153


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java:
##########
@@ -282,6 +281,10 @@ private void processCmd(DeleteCmdInfo cmd) {
       List<Future> futures = new ArrayList<>();
       for (int i = 0; i < containerBlocks.size(); i++) {
         DeletedBlocksTransaction tx = containerBlocks.get(i);
+        // Container is being balanced, ignore this transaction
+        if (shouldAbandon(tx.getContainerID(), cmd.getContainer())) {
+          continue;

Review Comment:
   @vtutrinov Thank you for the review. This PR has been separated to small PRs, could you help to review https://github.com/apache/ozone/pull/4887?
   
   I will close this PR.



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


[GitHub] [ozone] ferhui commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1285074631

   @lokeshj1703 Thanks for your detailed review! do you have other comments?


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


[GitHub] [ozone] DaveTeng0 commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "DaveTeng0 (via GitHub)" <gi...@apache.org>.
DaveTeng0 commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1408098074

   > @ChenSammi @lokeshj1703 @siddhantsangwan @sodonnel @neils-dev @JacksonYao287 Could you help to review this PR?
   
   hey guys~ let's start to gradually digest this pr together ~ lol


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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991911625


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   > I think we can still have multiple tasks trying to move the containers from the same volume. I feel we should have only task per volume at a time.
   
   Sorry I didn't get the idea from this line, seems a little controversial.
   
   > Also it would be good to have policies for choosing the volumes to balance. It can be a simple policy for now but it would avoid refactoring in future.
   
   Sure, I will add a policy class here.
   
   > We should have policies for choosing the container to move. Some containers could be undergoing replication to other DNs. We could avoid them.
   
   Sure, I will also add a policy class for containers.
   
   > 4. We shouldn't allow new tasks to be added until older tasks finish.
   
   I think this is currently controlled by the parallelThread. There will only be new tasks when a thread is avaialbe.
   
   > I would suggest breaking this PR into smaller PRs to reduce the scope of single PR.
   
   Yes, it's a big PR, could you give some advice on splitting the PR, since I found this service quite assembled?
   
   



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


[GitHub] [ozone] symious commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1251122834

   @ChenSammi @ferhui @lokeshj1703 @siddhantsangwan @sodonnel @neils-dev @JacksonYao287 Could you help to review this PR?


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


[GitHub] [ozone] symious commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1255924755

   @siddhantsangwan Sure, added a summary in PR's description, please have a look.


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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r1002981036


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   @lokeshj1703 Raised a new ticket HDDS-7383 for better review.
   Please have a look. The other PRs will be on the way.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991868746


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   What is the approach we are going with here then? Can we also update the docs once the PR is merged?
   If we fail the operation in DN, then SCM would at least know that disk balancer is running. Later we can add a change to avoid block deletions for that specific container in SCM.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991868746


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   What is the approach we are going with here then? Can we also update the docs once the PR is merged?
   If we fail the operation in DN, then SCM would at least know that disk balancer is running. Later we can add a change to avoid block deletions for that specific container.



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


[GitHub] [ozone] ferhui commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r981944129


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private long nextAvailableTime = Time.monotonicNow();
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bandwidth = bandwidthInMB * megaByte / 1000f;

Review Comment:
   seems not bytes/millisec, but bytes/kilo sec?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private long nextAvailableTime = Time.monotonicNow();
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {

Review Comment:
   One container per task, right?
   It will add many tasks here before ending this while loop. There is no condition for exiting this loop.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991876846


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   1. I think we can still have multiple tasks trying to move the containers from the same volume. I feel we should have only task per volume at a time. Since this is a disk heavy operation multiple tasks will not help.
   2. Also it would be good to have policies for choosing the volumes to balance. It can be a simple policy for now but it would avoid refactoring in future.
   3. We should have policies for choosing the container to move. Some containers could be undergoing replication to other DNs. We could avoid them.
   4. We shouldn't allow new tasks to be added until older tasks finish.
   
   I would suggest breaking this PR into smaller PRs to reduce the scope of single PR.



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


[GitHub] [ozone] symious commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1409613631

   > Can you add to the PR's description or as a comment, the new tickets that were raised from this PR?
   
   @xBis7 Sure updated in the description.


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


[GitHub] [ozone] xBis7 commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1409037047

   > Raised a new ticket [HDDS-7383](https://issues.apache.org/jira/browse/HDDS-7383) for better review.
   Please have a look. The other PRs will be on the way.
   
   @symious Can you add to the PR's description or as a comment, the new tickets that raised from this PR?


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


[GitHub] [ozone] ferhui commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r974857708


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java:
##########
@@ -112,6 +112,31 @@ public boolean addContainer(Container<?> container) throws
     }
   }
 
+  /**
+   * Update Container to container map.
+   * @param container container to be added
+   * @return If container is added to containerMap returns true, otherwise
+   * false
+   */
+  public Container updateContainer(Container<?> container) throws
+      StorageContainerException {
+    Preconditions.checkNotNull(container, "container cannot be null");
+
+    long containerId = container.getContainerData().getContainerID();
+    if (!containerMap.containsKey(containerId)) {
+      LOG.error("Container doesn't exists with container Id {}", containerId);
+      throw new StorageContainerException("Container doesn't exist with " +
+          "container Id " + containerId,
+          ContainerProtos.Result.CONTAINER_NOT_FOUND);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Container with container Id {} is updated to containerMap",

Review Comment:
   can remove if clause, use LOG.debug directly. 
   There is no performance issue for this case if removing the 'if' clause. refer to https://www.slf4j.org/faq.html#logging_performance



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java:
##########
@@ -617,6 +689,57 @@ public void exportContainerData(OutputStream destination,
     }
   }
 
+  @Override
+  public void copyContainerData(Path destination) throws IOException {
+    writeLock();
+    try {
+      // Closed/ Quasi closed containers are considered for replication by
+      // replication manager if they are under-replicated.
+      ContainerProtos.ContainerDataProto.State state =
+          getContainerData().getState();
+      if (!(state == ContainerProtos.ContainerDataProto.State.CLOSED ||
+          state == ContainerDataProto.State.QUASI_CLOSED)) {
+        throw new IllegalStateException(
+            "Only (quasi)closed containers can be exported, but " +
+                "ContainerId=" + getContainerData().getContainerID() +
+                " is in state " + state);
+      }
+
+      try {
+        if (!containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+          compactDB();
+          // Close DB (and remove from cache) to avoid concurrent modification
+          // while copying it.
+          BlockUtils.removeDB(containerData, config);
+        }
+      } finally {
+        readLock();
+        writeUnlock();
+      }
+
+      if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+        // Synchronize the dump and copy operation,
+        // so concurrent copy don't get dump files overwritten.
+        // We seldom got concurrent exports for a container,
+        // so it should not influence performance much.
+        synchronized (dumpLock) {
+          BlockUtils.dumpKVContainerDataToFiles(containerData, config);
+          copyContainerToDestination(destination);
+        }
+      } else {
+        copyContainerToDestination(destination);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();

Review Comment:
   Use LOG here?



##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/diskbalancer/TestDiskBalancerService.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.TestBlockDeletingService;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
+import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion;
+import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
+import org.apache.hadoop.ozone.container.keyvalue.ContainerTestVersionInfo;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.Collections.singletonMap;
+import static org.apache.hadoop.ozone.container.common.ContainerTestUtils.createDbInstancesForTestIfNeeded;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests to test diskBalancer service.
+ */
+@RunWith(Parameterized.class)
+public class TestDiskBalancerService {
+  private File testRoot;
+  private String scmId;
+  private String datanodeUuid;
+  private OzoneConfiguration conf;
+
+  private final ContainerLayoutVersion layout;
+  private final String schemaVersion;
+  private MutableVolumeSet volumeSet;
+
+  public TestDiskBalancerService(ContainerTestVersionInfo versionInfo) {
+    this.layout = versionInfo.getLayout();
+    this.schemaVersion = versionInfo.getSchemaVersion();
+    conf = new OzoneConfiguration();
+    ContainerTestVersionInfo.setTestSchemaVersion(schemaVersion, conf);
+  }
+
+  @Parameterized.Parameters
+  public static Iterable<Object[]> parameters() {
+    return ContainerTestVersionInfo.versionParameters();
+  }
+
+  @Before
+  public void init() throws IOException {
+    testRoot = GenericTestUtils
+        .getTestDir(TestBlockDeletingService.class.getSimpleName());
+    if (testRoot.exists()) {
+      FileUtils.cleanDirectory(testRoot);
+    }
+    scmId = UUID.randomUUID().toString();
+    conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY,
+        generateVolumeLocation(testRoot.getAbsolutePath(), 2));
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testRoot.getAbsolutePath());
+    conf.set("hdds.datanode.du.factory.classname",
+        "org.apache.hadoop.hdds.fs.MockSpaceUsageCheckFactory$HalfTera");
+    datanodeUuid = UUID.randomUUID().toString();
+    volumeSet = new MutableVolumeSet(datanodeUuid, scmId, conf, null,
+        StorageVolume.VolumeType.DATA_VOLUME, null);
+    createDbInstancesForTestIfNeeded(volumeSet, scmId, scmId, conf);
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    BlockUtils.shutdownCache(conf);
+    FileUtils.deleteDirectory(testRoot);
+  }
+
+  @Test
+  public void testBandwidthInMB() throws Exception {

Review Comment:
   do we need to add timeout for cases?



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


[GitHub] [ozone] ferhui commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
ferhui commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r975965260


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java:
##########
@@ -617,6 +689,57 @@ public void exportContainerData(OutputStream destination,
     }
   }
 
+  @Override
+  public void copyContainerData(Path destination) throws IOException {
+    writeLock();
+    try {
+      // Closed/ Quasi closed containers are considered for replication by
+      // replication manager if they are under-replicated.
+      ContainerProtos.ContainerDataProto.State state =
+          getContainerData().getState();
+      if (!(state == ContainerProtos.ContainerDataProto.State.CLOSED ||
+          state == ContainerDataProto.State.QUASI_CLOSED)) {
+        throw new IllegalStateException(
+            "Only (quasi)closed containers can be exported, but " +
+                "ContainerId=" + getContainerData().getContainerID() +
+                " is in state " + state);
+      }
+
+      try {
+        if (!containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+          compactDB();
+          // Close DB (and remove from cache) to avoid concurrent modification
+          // while copying it.
+          BlockUtils.removeDB(containerData, config);
+        }
+      } finally {
+        readLock();
+        writeUnlock();
+      }
+
+      if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+        // Synchronize the dump and copy operation,
+        // so concurrent copy don't get dump files overwritten.
+        // We seldom got concurrent exports for a container,
+        // so it should not influence performance much.
+        synchronized (dumpLock) {
+          BlockUtils.dumpKVContainerDataToFiles(containerData, config);
+          copyContainerToDestination(destination);
+        }
+      } else {
+        copyContainerToDestination(destination);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();

Review Comment:
   Sorry, I meant LOG instead of printStackTrace



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


[GitHub] [ozone] siddhantsangwan commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
siddhantsangwan commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1254945312

   This is a pretty big PR. @symious would it be possible for you to give a short summary of the changes so it's easier to review?


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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r983295328


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,486 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private long nextAvailableTime = Time.monotonicNow();
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {

Review Comment:
   @ferhui Thank you for the review, updated the patch, please have a look.



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


[GitHub] [ozone] DaveTeng0 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "DaveTeng0 (via GitHub)" <gi...@apache.org>.
DaveTeng0 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r1098250796


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration.HDDS_DATANODE_DISK_BALANCER_CONTAINER_CHOOSING_POLICY;
+import static org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration.HDDS_DATANODE_DISK_BALANCER_INFO_DIR;
+import static org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration.HDDS_DATANODE_DISK_BALANCER_VOLUME_CHOOSING_POLICY;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private DiskBalancerVolumeChoosingPolicy volumeChoosingPolicy;
+  private DiskBalancerContainerChoosingPolicy containerChoosingPolicy;
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    try {
+      volumeChoosingPolicy = conf.getClass(
+          HDDS_DATANODE_DISK_BALANCER_VOLUME_CHOOSING_POLICY,
+          SimpleDiskBalancerVolumeChoosingPolicy.class,
+          DiskBalancerVolumeChoosingPolicy.class).newInstance();
+      containerChoosingPolicy = conf.getClass(
+          HDDS_DATANODE_DISK_BALANCER_CONTAINER_CHOOSING_POLICY,
+          SimpleDiskBalancerContainerChoosingPolicy.class,
+          DiskBalancerContainerChoosingPolicy.class).newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair = volumeChoosingPolicy
+          .chooseVolume(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      ContainerData toBalanceContainer = containerChoosingPolicy
+          .chooseContainer(ozoneContainer, sourceVolume, inProgressContainers);
+      if (toBalanceContainer != null) {
+        queue.add(new DiskBalancerTask(toBalanceContainer, sourceVolume,
+            destVolume));
+        inProgressContainers.add(toBalanceContainer.getContainerID());
+        deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+            - toBalanceContainer.getBytesUsed());
+        deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+            + toBalanceContainer.getBytesUsed());
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime.get()) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bytesPerMillisec = bandwidthInMB * megaByte / 1000f;
+    nextAvailableTime.set(Time.monotonicNow() +
+        ((long) (bytesBalanced / bytesPerMillisec)));
+    return false;
+  }
+
+  private class DiskBalancerTask implements BackgroundTask {
+
+    private HddsVolume sourceVolume;
+    private HddsVolume destVolume;
+    private ContainerData containerData;
+    private long containerId;
+
+    DiskBalancerTask(ContainerData containerData,
+        HddsVolume sourceVolume, HddsVolume destVolume) {
+      this.containerData = containerData;
+      this.sourceVolume = sourceVolume;
+      this.destVolume = destVolume;
+      this.containerId = containerData.getContainerID();
+    }
+
+    @Override
+    public BackgroundTaskResult call() {
+      boolean destVolumeIncreased = false;
+      Path diskBalancerTmpDir = null, diskBalancerDestDir = null;
+      long containerSize = containerData.getBytesUsed();
+      try {
+        diskBalancerTmpDir = Paths.get(destVolume.getTmpDir().getPath())
+            .resolve(DISK_BALANCER_DIR).resolve(String.valueOf(containerId));
+
+        // Copy container to new Volume's tmp Dir
+        ozoneContainer.getController().copyContainer(
+            containerData.getContainerType(),
+            containerData.getContainerID(), diskBalancerTmpDir);
+
+        // Move container directory to final place on new volume
+        String idDir = VersionedDatanodeFeatures.ScmHA.chooseContainerPathID(
+            destVolume, destVolume.getClusterID());
+        diskBalancerDestDir =
+            Paths.get(KeyValueContainerLocationUtil.getBaseContainerLocation(
+                destVolume.getHddsRootDir().toString(), idDir,
+                containerData.getContainerID()));
+        Path destDirParent = diskBalancerDestDir.getParent();
+        if (destDirParent != null) {

Review Comment:
   should this be '== null' ?



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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r992073628


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);

Review Comment:
   @lokeshj1703 Updated the PR, please have a look.



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


[GitHub] [ozone] vtutrinov commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "vtutrinov (via GitHub)" <gi...@apache.org>.
vtutrinov commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r1247734992


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java:
##########
@@ -282,6 +281,10 @@ private void processCmd(DeleteCmdInfo cmd) {
       List<Future> futures = new ArrayList<>();
       for (int i = 0; i < containerBlocks.size(); i++) {
         DeletedBlocksTransaction tx = containerBlocks.get(i);
+        // Container is being balanced, ignore this transaction
+        if (shouldAbandon(tx.getContainerID(), cmd.getContainer())) {
+          continue;

Review Comment:
   Shouldn't we be throwing some kind of exception here to tell the user that the operation can't be performed right now (including a detailed reason message)? (the same notice is for DeleteContainerCommandHandler)



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r993071419


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/diskbalancer/DiskBalancerService.java:
##########
@@ -0,0 +1,487 @@
+/*
+ * 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.ozone.container.diskbalancer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DiskBalancerReportProto;
+import org.apache.hadoop.hdds.scm.storage.DiskBalancerConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A per-datanode disk balancing service takes in charge
+ * of moving contains among disks.
+ */
+public class DiskBalancerService extends BackgroundService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerService.class);
+
+  public static final String DISK_BALANCER_DIR = "diskBalancer";
+
+  private OzoneContainer ozoneContainer;
+  private final ConfigurationSource conf;
+
+  private boolean shouldRun = false;
+  private double threshold;
+  private long bandwidthInMB;
+  private int parallelThread;
+
+  private AtomicLong totalBalancedBytes = new AtomicLong(0L);
+  private AtomicLong balancedBytesInLastWindow = new AtomicLong(0L);
+  private AtomicLong nextAvailableTime = new AtomicLong(Time.monotonicNow());
+
+  private Map<DiskBalancerTask, Integer> inProgressTasks;
+  private Set<Long> inProgressContainers;
+  private Map<HddsVolume, Long> deltaSizes;
+  private MutableVolumeSet volumeSet;
+
+  private final File diskBalancerInfoFile;
+
+  private DiskBalancerServiceMetrics metrics;
+
+  public DiskBalancerService(OzoneContainer ozoneContainer,
+      long serviceCheckInterval, long serviceCheckTimeout, TimeUnit timeUnit,
+      int workerSize, ConfigurationSource conf) throws IOException {
+    super("DiskBalancerService", serviceCheckInterval, timeUnit, workerSize,
+        serviceCheckTimeout);
+    this.ozoneContainer = ozoneContainer;
+    this.conf = conf;
+
+    String diskBalancerInfoPath = getDiskBalancerInfoPath();
+    Preconditions.checkNotNull(diskBalancerInfoPath);
+    diskBalancerInfoFile = new File(diskBalancerInfoPath);
+
+    inProgressTasks = new ConcurrentHashMap<>();
+    inProgressContainers = ConcurrentHashMap.newKeySet();
+    deltaSizes = new ConcurrentHashMap<>();
+    volumeSet = ozoneContainer.getVolumeSet();
+
+    metrics = DiskBalancerServiceMetrics.create();
+
+    loadDiskBalancerInfo();
+
+    constructTmpDir();
+  }
+
+  /**
+   * Update DiskBalancerService based on new DiskBalancerInfo.
+   * @param diskBalancerInfo
+   * @throws IOException
+   */
+  public void refresh(DiskBalancerInfo diskBalancerInfo) throws IOException {
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void constructTmpDir() {
+    for (HddsVolume volume:
+        StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())) {
+      Path tmpDir = getDiskBalancerTmpDir(volume);
+      try {
+        FileUtils.deleteFully(tmpDir);
+        FileUtils.createDirectories(tmpDir);
+      } catch (IOException ex) {
+        LOG.warn("Can not reconstruct tmp directory under volume {}", volume,
+            ex);
+      }
+    }
+  }
+
+  /**
+   * If the diskBalancer.info file exists, load the file. If not exists,
+   * return the default config.
+   * @throws IOException
+   */
+  private void loadDiskBalancerInfo() throws IOException {
+    DiskBalancerInfo diskBalancerInfo = null;
+    try {
+      if (diskBalancerInfoFile.exists()) {
+        diskBalancerInfo = readDiskBalancerInfoFile(diskBalancerInfoFile);
+      }
+    } catch (IOException e) {
+      LOG.warn("Can not load diskBalancerInfo from diskBalancer.info file. " +
+          "Falling back to default configs", e);
+    } finally {
+      if (diskBalancerInfo == null) {
+        boolean shouldRunDefault = conf.getObject(DatanodeConfiguration.class)
+            .getDiskBalancerShouldRun();
+        diskBalancerInfo = new DiskBalancerInfo(shouldRunDefault,
+            new DiskBalancerConfiguration());
+      }
+    }
+
+    applyDiskBalancerInfo(diskBalancerInfo);
+  }
+
+  private void applyDiskBalancerInfo(DiskBalancerInfo diskBalancerInfo)
+      throws IOException {
+    // First store in local file, then update in memory variables
+    writeDiskBalancerInfoTo(diskBalancerInfo, diskBalancerInfoFile);
+
+    setShouldRun(diskBalancerInfo.isShouldRun());
+    setThreshold(diskBalancerInfo.getThreshold());
+    setBandwidthInMB(diskBalancerInfo.getBandwidthInMB());
+    setParallelThread(diskBalancerInfo.getParallelThread());
+
+    // Default executorService is ScheduledThreadPoolExecutor, so we can
+    // update the poll size by setting corePoolSize.
+    if ((getExecutorService() instanceof ScheduledThreadPoolExecutor)) {
+      ((ScheduledThreadPoolExecutor) getExecutorService())
+          .setCorePoolSize(parallelThread);
+    }
+  }
+
+  private String getDiskBalancerInfoPath() {
+    String diskBalancerInfoDir =
+        conf.getTrimmed(HddsConfigKeys.HDDS_DATANODE_DISK_BALANCER_INFO_DIR);
+    if (Strings.isNullOrEmpty(diskBalancerInfoDir)) {
+      File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
+      if (metaDirPath == null) {
+        // this means meta data is not found, in theory should not happen at
+        // this point because should've failed earlier.
+        throw new IllegalArgumentException("Unable to locate meta data" +
+            "directory when getting datanode disk balancer file path");
+      }
+      diskBalancerInfoDir = metaDirPath.toString();
+    }
+    // Use default datanode disk balancer file name for file path
+    return new File(diskBalancerInfoDir,
+        OzoneConsts.OZONE_SCM_DATANODE_DISK_BALANCER_INFO_DEFAULT).toString();
+  }
+
+  /**
+   * Read {@link DiskBalancerInfo} from a local info file.
+   *
+   * @param path DiskBalancerInfo file local path
+   * @return {@link DatanodeDetails}
+   * @throws IOException If the conf file is malformed or other I/O exceptions
+   */
+  private synchronized DiskBalancerInfo readDiskBalancerInfoFile(
+      File path) throws IOException {
+    if (!path.exists()) {
+      throw new IOException("DiskBalancerConf file not found.");
+    }
+    try {
+      return DiskBalancerYaml.readDiskBalancerInfoFile(path);
+    } catch (IOException e) {
+      LOG.warn("Error loading DiskBalancerInfo yaml from {}",
+          path.getAbsolutePath(), e);
+      throw new IOException("Failed to parse DiskBalancerInfo from "
+          + path.getAbsolutePath(), e);
+    }
+  }
+
+  /**
+   * Persistent a {@link DiskBalancerInfo} to a local file.
+   *
+   * @throws IOException when read/write error occurs
+   */
+  private synchronized void writeDiskBalancerInfoTo(
+      DiskBalancerInfo diskBalancerInfo, File path)
+      throws IOException {
+    if (path.exists()) {
+      if (!path.delete() || !path.createNewFile()) {
+        throw new IOException("Unable to overwrite the DiskBalancerInfo file.");
+      }
+    } else {
+      if (!path.getParentFile().exists() &&
+          !path.getParentFile().mkdirs()) {
+        throw new IOException("Unable to create DiskBalancerInfo directories.");
+      }
+    }
+    DiskBalancerYaml.createDiskBalancerInfoFile(diskBalancerInfo, path);
+  }
+
+
+
+  public void setShouldRun(boolean shouldRun) {
+    this.shouldRun = shouldRun;
+  }
+
+  public void setThreshold(double threshold) {
+    this.threshold = threshold;
+  }
+
+  public void setBandwidthInMB(long bandwidthInMB) {
+    this.bandwidthInMB = bandwidthInMB;
+  }
+
+  public void setParallelThread(int parallelThread) {
+    this.parallelThread = parallelThread;
+  }
+
+  public DiskBalancerInfo getDiskBalancerInfo() {
+    return new DiskBalancerInfo(shouldRun, threshold, bandwidthInMB,
+        parallelThread);
+  }
+
+  public DiskBalancerReportProto getDiskBalancerReportProto() {
+    DiskBalancerReportProto.Builder builder =
+        DiskBalancerReportProto.newBuilder();
+    return builder.setIsRunning(shouldRun)
+        .setBalancedBytes(totalBalancedBytes.get())
+        .setDiskBalancerConf(
+            HddsProtos.DiskBalancerConfigurationProto.newBuilder()
+                .setThreshold(threshold)
+                .setDiskBandwidthInMB(bandwidthInMB)
+                .setParallelThread(parallelThread)
+                .build())
+        .build();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+
+    if (!shouldRun) {
+      return queue;
+    }
+    metrics.incrRunningLoopCount();
+
+    if (shouldDelay()) {
+      metrics.incrIdleLoopExceedsBandwidthCount();
+      return queue;
+    }
+
+    int availableTaskCount = parallelThread - inProgressTasks.size();
+    if (availableTaskCount <= 0) {
+      LOG.info("No available thread for disk balancer service. " +
+          "Current thread count is {}.", parallelThread);
+      return queue;
+    }
+
+    for (int i = 0; i < availableTaskCount; i++) {
+      Pair<HddsVolume, HddsVolume> pair =
+          DiskBalancerUtils.getVolumePair(volumeSet, threshold, deltaSizes);
+      if (pair == null) {
+        continue;
+      }
+      HddsVolume sourceVolume = pair.getLeft(), destVolume = pair.getRight();
+      Iterator<Container<?>> itr = ozoneContainer.getController()
+          .getContainers(sourceVolume);
+      while (itr.hasNext()) {
+        ContainerData containerData = itr.next().getContainerData();
+        if (!inProgressContainers.contains(
+            containerData.getContainerID()) && containerData.isClosed()) {
+          queue.add(new DiskBalancerTask(containerData, sourceVolume,
+              destVolume));
+          inProgressContainers.add(containerData.getContainerID());
+          deltaSizes.put(sourceVolume, deltaSizes.getOrDefault(sourceVolume, 0L)
+              - containerData.getBytesUsed());
+          deltaSizes.put(destVolume, deltaSizes.getOrDefault(destVolume, 0L)
+              + containerData.getBytesUsed());
+          break;
+        }
+      }
+    }
+    if (queue.isEmpty()) {
+      metrics.incrIdleLoopNoAvailableVolumePairCount();
+    }
+    return queue;
+  }
+
+  private boolean shouldDelay() {
+    // We should wait for next AvailableTime.
+    if (Time.monotonicNow() <= nextAvailableTime.get()) {
+      return true;
+    }
+    // Calculate the next AvailableTime based on bandwidth
+    long bytesBalanced = balancedBytesInLastWindow.getAndSet(0L);
+
+    final int megaByte = 1024 * 1024;
+
+    // converting disk bandwidth in byte/millisec
+    float bytesPerMillisec = bandwidthInMB * megaByte / 1000f;
+    nextAvailableTime.set(Time.monotonicNow() +
+        ((long) (bytesBalanced / bytesPerMillisec)));
+    return false;
+  }
+
+  private class DiskBalancerTask implements BackgroundTask {
+
+    private HddsVolume sourceVolume;
+    private HddsVolume destVolume;
+    private ContainerData containerData;
+    private long containerId;
+
+    DiskBalancerTask(ContainerData containerData,
+        HddsVolume sourceVolume, HddsVolume destVolume) {
+      this.containerData = containerData;
+      this.sourceVolume = sourceVolume;
+      this.destVolume = destVolume;
+      this.containerId = containerData.getContainerID();
+    }
+
+    @Override
+    public BackgroundTaskResult call() {

Review Comment:
   Can you please take a look at KeyValueContainerUtil?



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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r990922689


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   Yes. Blocking is the original idea. But it might causing the thread pool of handler being blocked.



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


[GitHub] [ozone] lokeshj1703 commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
lokeshj1703 commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r991868746


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteContainerCommandHandler.java:
##########
@@ -67,6 +67,9 @@ public void handle(final SCMCommand command,
                      final SCMConnectionManager connectionManager) {
     final DeleteContainerCommand deleteContainerCommand =
         (DeleteContainerCommand) command;
+    if (shouldRequeue(command, context, ozoneContainer)) {
+      context.requeueCommand(command);
+    }

Review Comment:
   What is the approach we are going with here then? Can we also update the docs once the PR is merged?
   If we fail the operation in DN, then SCM would at least know that disk balancer is running. Later we can add a change to avoid block deletions for that specific container in SCM (Since SCM has a retry count for every transaction,  we do not want transaction retry count to be exhausted).



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


[GitHub] [ozone] symious commented on a diff in pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by GitBox <gi...@apache.org>.
symious commented on code in PR #3760:
URL: https://github.com/apache/ozone/pull/3760#discussion_r975965756


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java:
##########
@@ -617,6 +689,57 @@ public void exportContainerData(OutputStream destination,
     }
   }
 
+  @Override
+  public void copyContainerData(Path destination) throws IOException {
+    writeLock();
+    try {
+      // Closed/ Quasi closed containers are considered for replication by
+      // replication manager if they are under-replicated.
+      ContainerProtos.ContainerDataProto.State state =
+          getContainerData().getState();
+      if (!(state == ContainerProtos.ContainerDataProto.State.CLOSED ||
+          state == ContainerDataProto.State.QUASI_CLOSED)) {
+        throw new IllegalStateException(
+            "Only (quasi)closed containers can be exported, but " +
+                "ContainerId=" + getContainerData().getContainerID() +
+                " is in state " + state);
+      }
+
+      try {
+        if (!containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+          compactDB();
+          // Close DB (and remove from cache) to avoid concurrent modification
+          // while copying it.
+          BlockUtils.removeDB(containerData, config);
+        }
+      } finally {
+        readLock();
+        writeUnlock();
+      }
+
+      if (containerData.getSchemaVersion().equals(OzoneConsts.SCHEMA_V3)) {
+        // Synchronize the dump and copy operation,
+        // so concurrent copy don't get dump files overwritten.
+        // We seldom got concurrent exports for a container,
+        // so it should not influence performance much.
+        synchronized (dumpLock) {
+          BlockUtils.dumpKVContainerDataToFiles(containerData, config);
+          copyContainerToDestination(destination);
+        }
+      } else {
+        copyContainerToDestination(destination);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();

Review Comment:
   Sorry, I forget to remove the original line.



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


[GitHub] [ozone] symious closed pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious closed pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode
URL: https://github.com/apache/ozone/pull/3760


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


[GitHub] [ozone] symious commented on pull request #3760: HDDS-7233. Add DiskBalancerService on Datanode

Posted by "symious (via GitHub)" <gi...@apache.org>.
symious commented on PR #3760:
URL: https://github.com/apache/ozone/pull/3760#issuecomment-1420309007

   > can you please rebase, your branch is 400+ commits behind master.
   
   Sure, will try to do the rebase lately.


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