You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2016/06/24 06:05:32 UTC

[01/49] hadoop git commit: HDFS-9720. DiskBalancer : Add configuration parameters. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 0b9edf6e0 -> 631484388


HDFS-9720. DiskBalancer : Add configuration parameters. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/05067707
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05067707
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05067707

Branch: refs/heads/trunk
Commit: 050677077beaf42255b3936952b8e816a9201203
Parents: 6c606bf
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Apr 5 12:23:35 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |  4 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  8 +-
 .../server/datanode/DiskBalancerWorkItem.java   | 77 +++++++++++++++++++
 .../src/main/proto/ClientDatanodeProtocol.proto |  2 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   | 15 ++++
 ...tDatanodeProtocolServerSideTranslatorPB.java |  6 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  4 +-
 .../hdfs/server/datanode/DiskBalancer.java      | 81 +++++++++++++++-----
 .../server/diskbalancer/planner/MoveStep.java   | 75 ++++++++++++++++++
 .../hdfs/server/diskbalancer/planner/Step.java  | 23 ++++++
 .../diskbalancer/TestDiskBalancerRPC.java       | 31 ++++----
 .../TestDiskBalancerWithMockMover.java          | 37 ++++++++-
 .../hdfs/server/diskbalancer/TestPlanner.java   | 29 ++++---
 13 files changed, 328 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index d8df7fb..3993ce5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -168,8 +168,8 @@ public interface ClientDatanodeProtocol {
   /**
    * Submit a disk balancer plan for execution.
    */
-  void submitDiskBalancerPlan(String planID, long planVersion, long bandwidth,
-                              String plan) throws IOException;
+  void submitDiskBalancerPlan(String planID, long planVersion, String plan,
+                              boolean skipDateCheck) throws IOException;
 
   /**
    * Cancel an executing plan.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 7076026..4f314e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -342,22 +342,20 @@ public class ClientDatanodeProtocolTranslatorPB implements
    *               local copies of these plans.
    * @param planVersion - The data format of the plans - for future , not
    *                    used now.
-   * @param bandwidth - Maximum disk bandwidth to consume, setting this value
-   *                  to zero allows datanode to use the value defined in
-   *                  configration.
    * @param plan - Actual plan.
+   * @param skipDateCheck - Skips the date check.
    * @throws IOException
    */
   @Override
   public void submitDiskBalancerPlan(String planID, long planVersion,
-      long bandwidth, String plan) throws IOException {
+      String plan, boolean skipDateCheck) throws IOException {
     try {
       SubmitDiskBalancerPlanRequestProto request =
           SubmitDiskBalancerPlanRequestProto.newBuilder()
               .setPlanID(planID)
               .setPlanVersion(planVersion)
-              .setMaxDiskBandwidth(bandwidth)
               .setPlan(plan)
+              .setIgnoreDateCheck(skipDateCheck)
               .build();
       rpcProxy.submitDiskBalancerPlan(NULL_CONTROLLER, request);
     } catch (ServiceException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
index 11730e2..7381499 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonInclude;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;
@@ -31,6 +32,7 @@ import java.io.IOException;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
 public class DiskBalancerWorkItem {
   private final long bytesToCopy;
   private long bytesCopied;
@@ -38,6 +40,10 @@ public class DiskBalancerWorkItem {
   private String errMsg;
   private long blocksCopied;
 
+  private long maxDiskErrors;
+  private long tolerancePercent;
+  private long bandwidth;
+
   /**
    * Constructs a DiskBalancerWorkItem.
    *
@@ -157,4 +163,75 @@ public class DiskBalancerWorkItem {
     return mapper.writeValueAsString(this);
   }
 
+  /**
+   * Sets the Error counts for this step.
+   *
+   * @param errorCount long.
+   */
+  public void setErrorCount(long errorCount) {
+    this.errorCount = errorCount;
+  }
+
+  /**
+   * Number of blocks copied so far.
+   *
+   * @param blocksCopied Blocks copied.
+   */
+  public void setBlocksCopied(long blocksCopied) {
+    this.blocksCopied = blocksCopied;
+  }
+
+  /**
+   * Gets maximum disk errors to tolerate before we fail this copy step.
+   *
+   * @return long.
+   */
+  public long getMaxDiskErrors() {
+    return maxDiskErrors;
+  }
+
+  /**
+   * Sets maximum disk errors to tolerate before we fail this copy step.
+   *
+   * @param maxDiskErrors long
+   */
+  public void setMaxDiskErrors(long maxDiskErrors) {
+    this.maxDiskErrors = maxDiskErrors;
+  }
+
+  /**
+   * Allowed deviation from ideal storage in percentage.
+   *
+   * @return long
+   */
+  public long getTolerancePercent() {
+    return tolerancePercent;
+  }
+
+  /**
+   * Sets the tolerance percentage.
+   *
+   * @param tolerancePercent - tolerance.
+   */
+  public void setTolerancePercent(long tolerancePercent) {
+    this.tolerancePercent = tolerancePercent;
+  }
+
+  /**
+   * Max disk bandwidth to use. MB per second.
+   *
+   * @return - long.
+   */
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  /**
+   * Sets max disk bandwidth to use, in MBs per second.
+   *
+   * @param bandwidth - long.
+   */
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index c7cd4fb..e0a7fd5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -157,7 +157,7 @@ message SubmitDiskBalancerPlanRequestProto {
   required string planID = 1; // A hash of the plan like SHA512
   required string plan = 2; // Json String that describes the plan
   optional uint64 planVersion = 3; // Plan version number
-  optional uint64 maxDiskBandwidth = 4; // optional bandwidth control.
+  optional bool ignoreDateCheck = 4; // Ignore date checks on this plan.
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 224ab3d..6640ec6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -935,6 +935,21 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.disk.balancer.enabled";
   public static final boolean DFS_DISK_BALANCER_ENABLED_DEFAULT = false;
 
+  public static final String DFS_DISK_BALANCER_MAX_DISK_THRUPUT =
+      "dfs.disk.balancer.max.disk.throughputInMBperSec";
+  public static final int DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT =
+      10;
+
+  public static final String DFS_DISK_BALANCER_MAX_DISK_ERRORS =
+      "dfs.disk.balancer.max.disk.errors";
+  public static final int DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT = 5;
+
+
+  public static final String DFS_DISK_BALANCER_BLOCK_TOLERANCE =
+      "dfs.disk.balancer.block.tolerance.percent";
+  public static final int DFS_DISK_BALANCER_BLOCK_TOLERANCE_DEFAULT = 5;
+
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index d72a060..482e86f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -254,9 +254,9 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throws ServiceException {
     try {
       impl.submitDiskBalancerPlan(request.getPlanID(),
-          request.hasPlanVersion() ? request.getPlanVersion() : 0,
-          request.hasMaxDiskBandwidth() ? request.getMaxDiskBandwidth() : 0,
-          request.getPlan());
+          request.hasPlanVersion() ? request.getPlanVersion() : 1,
+          request.getPlan(),
+          request.hasIgnoreDateCheck() ? request.getIgnoreDateCheck() : false);
       SubmitDiskBalancerPlanResponseProto response =
           SubmitDiskBalancerPlanResponseProto.newBuilder()
               .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 8a61291..776da3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3331,11 +3331,11 @@ public class DataNode extends ReconfigurableBase
    */
   @Override
   public void submitDiskBalancerPlan(String planID,
-      long planVersion, long bandwidth, String plan) throws IOException {
+      long planVersion, String plan, boolean skipDateCheck) throws IOException {
 
     checkSuperuserPrivilege();
     // TODO : Support force option
-    this.diskBalancer.submitPlan(planID, planVersion, plan, bandwidth, false);
+    this.diskBalancer.submitPlan(planID, planVersion, plan, skipDateCheck);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 972f0fc..b62a4fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -35,6 +33,8 @@ import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.util.Time;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -45,6 +45,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 
 /**
@@ -99,6 +100,9 @@ public class DiskBalancer {
     this.isDiskBalancerEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_DISK_BALANCER_ENABLED,
         DFSConfigKeys.DFS_DISK_BALANCER_ENABLED_DEFAULT);
+    this.bandwidth = conf.getInt(
+        DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT,
+        DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT);
   }
 
   /**
@@ -144,13 +148,11 @@ public class DiskBalancer {
    * @param planID      - A SHA512 of the plan string
    * @param planVersion - version of the plan string - for future use.
    * @param plan        - Actual Plan
-   * @param bandwidth   - BytesPerSec to copy
    * @param force       - Skip some validations and execute the plan file.
    * @throws DiskBalancerException
    */
   public void submitPlan(String planID, long planVersion, String plan,
-                         long bandwidth, boolean force)
-      throws DiskBalancerException {
+                         boolean force) throws DiskBalancerException {
 
     lock.lock();
     try {
@@ -160,12 +162,10 @@ public class DiskBalancer {
         throw new DiskBalancerException("Executing another plan",
             DiskBalancerException.Result.PLAN_ALREADY_IN_PROGRESS);
       }
-      NodePlan nodePlan =
-          verifyPlan(planID, planVersion, plan, bandwidth, force);
+      NodePlan nodePlan = verifyPlan(planID, planVersion, plan, force);
       createWorkPlan(nodePlan);
       this.planID = planID;
       this.currentResult = Result.PLAN_UNDER_PROGRESS;
-      this.bandwidth = bandwidth;
       executePlan();
     } finally {
       lock.unlock();
@@ -292,14 +292,12 @@ public class DiskBalancer {
    * @param planID      - SHA 512 of the plan.
    * @param planVersion - Version of the plan, for future use.
    * @param plan        - Plan String in Json.
-   * @param bandwidth   - Max disk bandwidth to use per second.
    * @param force       - Skip verifying when the plan was generated.
    * @return a NodePlan Object.
    * @throws DiskBalancerException
    */
   private NodePlan verifyPlan(String planID, long planVersion, String plan,
-                              long bandwidth, boolean force)
-      throws DiskBalancerException {
+                               boolean force) throws DiskBalancerException {
 
     Preconditions.checkState(lock.isHeldByCurrentThread());
     verifyPlanVersion(planVersion);
@@ -428,7 +426,7 @@ public class DiskBalancer {
         throw new DiskBalancerException("Unable to find destination volume.",
             DiskBalancerException.Result.INVALID_VOLUME);
       }
-      createWorkPlan(sourceVol, destVol, step.getBytesToMove());
+      createWorkPlan(sourceVol, destVol, step);
     }
   }
 
@@ -488,17 +486,18 @@ public class DiskBalancer {
    *
    * @param source      - Source vol
    * @param dest        - destination volume
-   * @param bytesToMove - number of bytes to move
+   * @param step        - Move Step
    */
   private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest,
-                              long bytesToMove) throws DiskBalancerException {
+                              Step step) throws DiskBalancerException {
 
     if(source.getStorageID().equals(dest.getStorageID())) {
-      throw new DiskBalancerException("Same source and destination",
-          DiskBalancerException.Result.INVALID_MOVE);
+      LOG.info("Disk Balancer - source & destination volumes are same.");
+      throw new DiskBalancerException("source and destination volumes are " +
+          "same.", DiskBalancerException.Result.INVALID_MOVE);
     }
     VolumePair pair = new VolumePair(source, dest);
-
+    long bytesToMove = step.getBytesToMove();
     // In case we have a plan with more than
     // one line of same <source, dest>
     // we compress that into one work order.
@@ -507,6 +506,12 @@ public class DiskBalancer {
     }
 
     DiskBalancerWorkItem work = new DiskBalancerWorkItem(bytesToMove, 0);
+
+    // all these values can be zero, if so we will use
+    // values from configuration.
+    work.setBandwidth(step.getBandwidth());
+    work.setTolerancePercent(step.getTolerancePercent());
+    work.setMaxDiskErrors(step.getMaxDiskErrors());
     workMap.put(pair, work);
   }
 
@@ -600,11 +605,12 @@ public class DiskBalancer {
   /**
    * Actual DataMover class for DiskBalancer.
    * <p/>
-   * TODO : Add implementation for this class. This is here as a place holder so
-   * that Datanode can make calls into this class.
    */
   public static class DiskBalancerMover implements BlockMover {
     private final FsDatasetSpi dataset;
+    private long diskBandwidth;
+    private long blockTolerance;
+    private long maxDiskErrors;
 
     /**
      * Constructs diskBalancerMover.
@@ -614,7 +620,42 @@ public class DiskBalancer {
      */
     public DiskBalancerMover(FsDatasetSpi dataset, Configuration conf) {
       this.dataset = dataset;
-      // TODO : Read Config values.
+
+      this.diskBandwidth = conf.getLong(
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT,
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT);
+
+      this.blockTolerance = conf.getLong(
+          DFSConfigKeys.DFS_DISK_BALANCER_BLOCK_TOLERANCE,
+          DFSConfigKeys.DFS_DISK_BALANCER_BLOCK_TOLERANCE_DEFAULT);
+
+      this.maxDiskErrors = conf.getLong(
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_ERRORS,
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT);
+
+      // Since these are user provided values make sure it is sane
+      // or ignore faulty values.
+      if (this.diskBandwidth <= 0) {
+        LOG.debug("Found 0 or less as max disk throughput, ignoring config " +
+            "value. value : " + diskBandwidth);
+        diskBandwidth =
+            DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT;
+      }
+
+      if (this.blockTolerance <= 0) {
+        LOG.debug("Found 0 or less for block tolerance value, ignoring config" +
+            "value. value : " + blockTolerance);
+        blockTolerance =
+            DFSConfigKeys.DFS_DISK_BALANCER_BLOCK_TOLERANCE_DEFAULT;
+
+      }
+
+      if (this.maxDiskErrors < 0) {
+        LOG.debug("Found  less than 0 for maxDiskErrors value, ignoring " +
+            "config value. value : " + maxDiskErrors);
+        maxDiskErrors =
+            DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT;
+      }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
index 75af0d6..9a493a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
@@ -19,8 +19,19 @@ package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonInclude;
+
+
+
+
 
 /**
+ * Ignore fields with default values. In most cases Throughtput, diskErrors
+ * tolerancePercent and bandwidth will be the system defaults.
+ * So we will avoid serializing them into JSON.
+ */
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+/**
  * Move step is a step that planner can execute that will move data from one
  * volume to another.
  */
@@ -31,6 +42,10 @@ public class MoveStep implements Step {
   private long bytesToMove;
   private String volumeSetID;
 
+  private long maxDiskErrors;
+  private long tolerancePercent;
+  private long bandwidth;
+
   /**
    * Constructs a MoveStep for the volume set.
    *
@@ -178,4 +193,64 @@ public class MoveStep implements Step {
   public String getSizeString(long size) {
     return StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1);
   }
+
+  /**
+   * Gets Maximum numbers of errors to be tolerated before this
+   * move operation is aborted.
+   * @return  long.
+   */
+  public long getMaxDiskErrors() {
+    return maxDiskErrors;
+  }
+
+  /**
+   * Sets the maximum numbers of Errors to be tolerated before this
+   * step is aborted.
+   * @param maxDiskErrors - long
+   */
+  public void setMaxDiskErrors(long maxDiskErrors) {
+    this.maxDiskErrors = maxDiskErrors;
+  }
+
+  /**
+   * Tolerance Percentage indicates when a move operation is considered good
+   * enough. This is a percentage of deviation from ideal that is considered
+   * fine.
+   *
+   * For example : if the ideal amount on each disk was 1 TB and the
+   * tolerance was 10%, then getting to 900 GB on the destination disk is
+   * considerd good enough.
+   *
+   * @return tolerance percentage.
+   */
+  public long getTolerancePercent() {
+    return tolerancePercent;
+  }
+
+  /**
+   * Sets the tolerance percentage.
+   * @param tolerancePercent  - long
+   */
+  public void setTolerancePercent(long tolerancePercent) {
+    this.tolerancePercent = tolerancePercent;
+  }
+
+  /**
+   * Gets the disk Bandwidth. That is the MB/Sec to copied. We will max out
+   * on this amount of throughput. This is useful to prevent too much I/O on
+   * datanode while data node is in use.
+   * @return  long.
+   */
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  /**
+   * Sets the maximum disk bandwidth per sec to use for this step.
+   * @param bandwidth  - Long, MB / Sec of data to be moved between
+   *                   source and destinatin volume.
+   */
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
index d87209e..f13909f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
@@ -65,4 +65,27 @@ public interface Step {
    */
   String getSizeString(long size);
 
+  /**
+   * Returns maximum number of disk erros tolerated.
+   * @return long.
+   */
+  long getMaxDiskErrors();
+
+  /**
+   * Returns tolerance percentage, the good enough value
+   * when we move data from one to disk to another.
+   * @return long.
+   */
+  long getTolerancePercent();
+
+  /**
+   * Returns max disk bandwidth that disk balancer will use.
+   * Expressed in MB/sec. For example, a value like 10
+   * indicates that disk balancer will only move 10 MB / sec
+   * while it is running.
+   * @return long.
+   */
+  long getBandwidth();
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index a65ed21..27cd8eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -5,9 +5,9 @@
  * 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
@@ -30,7 +30,9 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.hamcrest.*;
 import org.junit.After;
 import org.junit.Assert;
@@ -76,7 +78,8 @@ public class TestDiskBalancerRPC {
     String planHash = rpcTestHelper.getPlanHash();
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
   }
 
   @Test
@@ -91,7 +94,8 @@ public class TestDiskBalancerRPC {
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_HASH));
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
   }
 
   @Test
@@ -104,7 +108,8 @@ public class TestDiskBalancerRPC {
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_VERSION));
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
   }
 
   @Test
@@ -116,8 +121,8 @@ public class TestDiskBalancerRPC {
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN));
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, "");
-  }
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, "",
+        false);  }
 
   @Test
   public void testCancelPlan() throws Exception {
@@ -126,7 +131,8 @@ public class TestDiskBalancerRPC {
     String planHash = rpcTestHelper.getPlanHash();
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
@@ -189,15 +195,14 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
 
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
     String bandwidthString = dataNode.getDiskBalancerSetting(
         DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
     long value = Long.decode(bandwidthString);
     Assert.assertEquals(10L, value);
   }
 
-
-
   @Test
   public void testQueryPlan() throws Exception {
     RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
@@ -206,7 +211,8 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
 
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
     DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
     Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
         status.getResult() == PLAN_DONE);
@@ -221,7 +227,6 @@ public class TestDiskBalancerRPC {
     Assert.assertTrue(status.getResult() == NO_PLAN);
   }
 
-
   private class RpcTestHelper {
     private NodePlan plan;
     private int planVersion;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
index ed761ed..5032611 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -50,11 +50,14 @@ import org.junit.rules.ExpectedException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 public class TestDiskBalancerWithMockMover {
@@ -120,7 +123,7 @@ public class TestDiskBalancerWithMockMover {
                                  int version) throws IOException {
     String planJson = plan.toJson();
     String planID = DigestUtils.sha512Hex(planJson);
-    balancer.submitPlan(planID, version, planJson, 10, false);
+    balancer.submitPlan(planID, version, planJson, false);
   }
 
   private void executeSubmitPlan(NodePlan plan, DiskBalancer balancer)
@@ -209,7 +212,7 @@ public class TestDiskBalancerWithMockMover {
     thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
         .Result.INVALID_PLAN));
 
-    balancer.submitPlan(planID, 1, null, 10, false);
+    balancer.submitPlan(planID, 1, null, false);
   }
 
   @Test
@@ -228,7 +231,7 @@ public class TestDiskBalancerWithMockMover {
     thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
         .Result.INVALID_PLAN_HASH));
     balancer.submitPlan(planID.replace(planID.charAt(0), repChar),
-        1, planJson, 10, false);
+        1, planJson, false);
 
   }
 
@@ -278,6 +281,34 @@ public class TestDiskBalancerWithMockMover {
 
   }
 
+
+  /**
+   * Test Custom bandwidth.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testCustomBandwidth() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    for(Step step : plan.getVolumeSetPlans()){
+      MoveStep tempStep = (MoveStep) step;
+      tempStep.setBandwidth(100);
+    }
+    executeSubmitPlan(plan, balancer);
+    DiskBalancerWorkStatus status = balancer
+        .queryWorkStatus();
+    assertNotNull(status);
+
+    DiskBalancerWorkStatus.DiskBalancerWorkEntry entry =
+        balancer.queryWorkStatus().getCurrentState().get(0);
+    assertEquals(100L, entry.getWorkItem().getBandwidth());
+
+  }
+
+
   @Before
   public void setUp() throws Exception {
     Configuration conf = new HdfsConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05067707/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
index f756104..ad18075 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
-import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.junit.Assert;
@@ -48,7 +47,7 @@ public class TestPlanner {
       LoggerFactory.getLogger(TestPlanner.class);
 
   @Test
-  public void TestGreedyPlannerBalanceVolumeSet() throws Exception {
+  public void testGreedyPlannerBalanceVolumeSet() throws Exception {
     URI clusterJson = getClass()
         .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
     ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
@@ -65,7 +64,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerComputePlan() throws Exception {
+  public void testGreedyPlannerComputePlan() throws Exception {
     URI clusterJson = getClass()
         .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
     ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
@@ -90,13 +89,13 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerNoNodeCluster() throws Exception {
+  public void testGreedyPlannerNoNodeCluster() throws Exception {
     GreedyPlanner planner = new GreedyPlanner(10.0f, null);
     assertNotNull(planner);
   }
 
   @Test
-  public void TestGreedyPlannerNoVolumeTest() throws Exception {
+  public void testGreedyPlannerNoVolumeTest() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
     List<NodePlan> planList = cluster.computePlan(10.0f);
@@ -104,7 +103,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerOneVolumeNoPlanTest() throws Exception {
+  public void testGreedyPlannerOneVolumeNoPlanTest() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -127,7 +126,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerTwoVolume() throws Exception {
+  public void testGreedyPlannerTwoVolume() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -166,7 +165,7 @@ public class TestPlanner {
    * That is the plan should say move 10 GB from volume30 to volume10.
    */
   @Test
-  public void TestGreedyPlannerEqualizeData() throws Exception {
+  public void testGreedyPlannerEqualizeData() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -201,7 +200,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerEqualDisksNoMoves() throws Exception {
+  public void testGreedyPlannerEqualDisksNoMoves() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -232,7 +231,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerMoveFromSingleDisk() throws Exception {
+  public void testGreedyPlannerMoveFromSingleDisk() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -271,7 +270,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerThresholdTest() throws Exception {
+  public void testGreedyPlannerThresholdTest() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -327,7 +326,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestGreedyPlannerPlanWithDifferentDiskSizes() throws Exception {
+  public void testGreedyPlannerPlanWithDifferentDiskSizes() throws Exception {
     NullConnector nullConnector = new NullConnector();
     DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
 
@@ -381,7 +380,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestLoadsCorrectClusterConnector() throws Exception {
+  public void testLoadsCorrectClusterConnector() throws Exception {
     ClusterConnector connector = ConnectorFactory.getCluster(getClass()
             .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI()
         , null);
@@ -392,7 +391,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestPlannerScale() throws Exception {
+  public void testPlannerScale() throws Exception {
     final int diskCount = 256; // it is rare to see more than 48 disks
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerVolumeSet vSet =
@@ -428,7 +427,7 @@ public class TestPlanner {
   }
 
   @Test
-  public void TestNodePlanSerialize() throws Exception {
+  public void testNodePlanSerialize() throws Exception {
     final int diskCount = 12;
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerVolumeSet vSet =


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/49] hadoop git commit: HDFS-10541. Diskbalancer: When no actions in plan, error message says "Plan was generated more than 24 hours ago". Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10541. Diskbalancer: When no actions in plan, error message says "Plan was generated more than 24 hours ago". Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5b8e1c26
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5b8e1c26
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5b8e1c26

Branch: refs/heads/trunk
Commit: 5b8e1c26d702e42b606265860c5e475970876aa5
Parents: cb68e5b
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jun 17 23:29:57 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:00 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/diskbalancer/command/PlanCommand.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b8e1c26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 76bdc9f..c13399b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -140,7 +140,7 @@ public class PlanCommand extends Command {
           .getBytes(StandardCharsets.UTF_8));
     }
 
-    if (plan != null) {
+    if (plan != null && plan.getVolumeSetPlans().size() > 0) {
       LOG.info("Writing plan to : {}", getOutputPath());
       try (FSDataOutputStream planStream = create(String.format(
           DiskBalancer.PLAN_TEMPLATE,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/49] hadoop git commit: HDFS-9702. DiskBalancer: getVolumeMap implementation. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9702. DiskBalancer: getVolumeMap implementation. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/918722bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/918722bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/918722bd

Branch: refs/heads/trunk
Commit: 918722bdd202acbeda92d650ff0dcecbcd8a0697
Parents: 4b93dda
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Mar 9 09:44:22 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DataNode.java   | 17 +++--
 .../hdfs/server/datanode/DiskBalancer.java      | 26 ++++++++
 .../diskbalancer/DiskBalancerException.java     |  3 +-
 .../diskbalancer/TestDiskBalancerRPC.java       | 66 ++++++++++++++++++++
 4 files changed, 107 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/918722bd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 126deb4..00e124d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -169,6 +169,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.web.DatanodeHttpServer;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -3360,8 +3361,8 @@ public class DataNode extends ReconfigurableBase
   }
 
   /**
-   * Gets a run-time configuration value from running diskbalancer instance. For
-   * example : Disk Balancer bandwidth of a running instance.
+   * Gets a runtime configuration value from  diskbalancer instance. For
+   * example : DiskBalancer bandwidth.
    *
    * @param key - String that represents the run time key value.
    * @return value of the key as a string.
@@ -3370,7 +3371,15 @@ public class DataNode extends ReconfigurableBase
   @Override
   public String getDiskBalancerSetting(String key) throws IOException {
     checkSuperuserPrivilege();
-    throw new DiskBalancerException("Not Implemented",
-        DiskBalancerException.Result.INTERNAL_ERROR);
+    Preconditions.checkNotNull(key);
+    switch (key) {
+    case DiskBalancerConstants.DISKBALANCER_VOLUME_NAME:
+      return this.diskBalancer.getVolumeNames();
+    default:
+      LOG.error("Disk Balancer - Unknown key in get balancer setting. Key: " +
+          key);
+      throw new DiskBalancerException("Unknown key",
+          DiskBalancerException.Result.UNKNOWN_KEY);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/918722bd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index d5c402e..9e41d2e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.util.Time;
+import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -221,6 +222,31 @@ public class DiskBalancer {
     }
   }
 
+  /**
+   * Returns a volume ID to Volume base path map.
+   *
+   * @return Json string of the volume map.
+   * @throws DiskBalancerException
+   */
+  public String getVolumeNames() throws DiskBalancerException {
+    lock.lock();
+    try {
+      checkDiskBalancerEnabled();
+      Map<String, String> pathMap = new HashMap<>();
+      Map<String, FsVolumeSpi> volMap = getStorageIDToVolumeMap();
+      for (Map.Entry<String, FsVolumeSpi> entry : volMap.entrySet()) {
+        pathMap.put(entry.getKey(), entry.getValue().getBasePath());
+      }
+      ObjectMapper mapper = new ObjectMapper();
+      return mapper.writeValueAsString(pathMap);
+    } catch (IOException e) {
+      throw new DiskBalancerException("Internal error, Unable to " +
+          "create JSON string.", e,
+          DiskBalancerException.Result.INTERNAL_ERROR);
+    } finally {
+      lock.unlock();
+    }
+  }
 
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/918722bd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
index 00fe53d..38455a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -36,7 +36,8 @@ public class DiskBalancerException extends IOException {
     INVALID_VOLUME,
     INVALID_MOVE,
     INTERNAL_ERROR,
-    NO_SUCH_PLAN
+    NO_SUCH_PLAN,
+    UNKNOWN_KEY
   }
 
   private final Result result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/918722bd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index e29b3b7..37a6216 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -24,18 +24,24 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException.*;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.hamcrest.*;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
@@ -84,6 +90,8 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.INVALID_PLAN_HASH));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
   }
 
@@ -96,6 +104,8 @@ public class TestDiskBalancerRPC {
     planVersion++;
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.INVALID_PLAN_VERSION));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
   }
 
@@ -107,6 +117,8 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.INVALID_PLAN));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, "");
   }
 
@@ -131,6 +143,8 @@ public class TestDiskBalancerRPC {
     planHash = String.valueOf(hashArray);
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.NO_SUCH_PLAN));
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
@@ -141,9 +155,38 @@ public class TestDiskBalancerRPC {
     String planHash = "";
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.NO_SUCH_PLAN));
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
+  @Test
+  public void testGetDiskBalancerVolumeMapping() throws Exception {
+    final int dnIndex = 0;
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    String volumeNameJson = dataNode.getDiskBalancerSetting(
+        DiskBalancerConstants.DISKBALANCER_VOLUME_NAME);
+    Assert.assertNotNull(volumeNameJson);
+    ObjectMapper mapper = new ObjectMapper();
+
+    @SuppressWarnings("unchecked")
+    Map<String, String> volumemap =
+        mapper.readValue(volumeNameJson, HashMap.class);
+
+    Assert.assertEquals(2, volumemap.size());
+  }
+
+  @Test
+  public void testGetDiskBalancerInvalidSetting() throws Exception {
+    final int dnIndex = 0;
+    final String invalidSetting = "invalidSetting";
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.UNKNOWN_KEY));
+    dataNode.getDiskBalancerSetting(invalidSetting);
+  }
+
 
   @Test
   public void testQueryPlan() throws Exception {
@@ -173,6 +216,8 @@ public class TestDiskBalancerRPC {
     final int dnIndex = 0;
     DataNode dataNode = cluster.getDataNodes().get(dnIndex);
     thrown.expect(DiskBalancerException.class);
+    thrown.expect(new
+        ResultVerifier(Result.UNKNOWN_KEY));
     dataNode.getDiskBalancerSetting(
         DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
   }
@@ -223,4 +268,25 @@ public class TestDiskBalancerRPC {
       return this;
     }
   }
+
+  private class ResultVerifier
+      extends TypeSafeMatcher<DiskBalancerException> {
+    private final DiskBalancerException.Result expectedResult;
+
+    ResultVerifier(DiskBalancerException.Result expectedResult){
+      this.expectedResult = expectedResult;
+    }
+
+    @Override
+    protected boolean matchesSafely(DiskBalancerException exception) {
+      return (this.expectedResult == exception.getResult());
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("expects Result: ")
+          .appendValue(this.expectedResult);
+
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/49] hadoop git commit: HDFS-9611. DiskBalancer : Replace htrace json imports with jackson. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9611. DiskBalancer : Replace htrace json imports with jackson. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e325c6ad
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e325c6ad
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e325c6ad

Branch: refs/heads/trunk
Commit: e325c6ade9dab41e0fbbf5222bb72236d4d5a77f
Parents: 599eca0
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Jan 5 09:10:03 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt            | 4 ++++
 .../server/diskbalancer/datamodel/DiskBalancerVolumeSet.java     | 4 ++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e325c6ad/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 952813b..8220f88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -7,3 +7,7 @@ HDFS-1312 Change Log
     HDFS-9449. DiskBalancer: Add connectors. (Anu Engineer via szetszwo)
 
     HDFS-9526. Fix jackson annotation imports. (Xiaobing Zhou via szetszwo)
+
+    HDFS-9611. DiskBalancer : Replace htrace json imports with jackson.
+    (Anu Engineer via Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e325c6ad/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
index 15c21ac..49c8558 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.hdfs.server.diskbalancer.datamodel;
 import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.htrace.fasterxml.jackson.annotation.JsonProperty;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 import java.io.Serializable;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/49] hadoop git commit: HDFS-9595. DiskBalancer: Add cancelPlan RPC. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9595. DiskBalancer: Add cancelPlan RPC. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0501d430
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0501d430
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0501d430

Branch: refs/heads/trunk
Commit: 0501d430e2f6111ad8b65dc36f4a98d94cb9589b
Parents: 7100c0d
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jan 15 16:08:49 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |  7 +++
 .../ClientDatanodeProtocolTranslatorPB.java     | 19 +++++++
 .../src/main/proto/ClientDatanodeProtocol.proto | 19 +++++++
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |  3 ++
 ...tDatanodeProtocolServerSideTranslatorPB.java | 22 ++++++++
 .../hadoop/hdfs/server/datanode/DataNode.java   |  7 +++
 .../diskbalancer/planner/GreedyPlanner.java     |  4 ++
 .../diskbalancer/TestDiskBalancerRPC.java       | 56 ++++++++++++++++----
 8 files changed, 127 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index 6e9cef0..125a3c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -170,4 +170,11 @@ public interface ClientDatanodeProtocol {
   void submitDiskBalancerPlan(String planID, long planVersion, long bandwidth,
                               String plan) throws IOException;
 
+  /**
+   * Cancel an executing plan.
+   *
+   * @param planID - A SHA512 hash of the plan string.
+   */
+  void cancelDiskBalancePlan(String planID) throws IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index da8d962..e037fcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Shutdo
 import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.StartReconfigurationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -358,4 +359,22 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  /**
+   * Cancels an executing disk balancer plan.
+   * @param planID - A SHA512 hash of the plan string.
+   *
+   * @throws IOException on error
+   */
+  @Override
+  public void cancelDiskBalancePlan(String planID)
+      throws IOException {
+    try {
+      CancelPlanRequestProto request = CancelPlanRequestProto.newBuilder()
+          .setPlanID(planID).build();
+      rpcProxy.cancelDiskBalancerPlan(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index d11979b..b65766b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -167,6 +167,20 @@ message SubmitDiskBalancerPlanResponseProto {
 }
 
 /**
+ * This message describes a request to cancel an
+ * outstanding disk balancer plan
+ */
+message CancelPlanRequestProto {
+  required string planID = 1;
+}
+
+/**
+ * This is the response for the cancellation request
+ */
+message CancelPlanResponseProto {
+}
+
+/**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
  */
@@ -230,4 +244,9 @@ service ClientDatanodeProtocolService {
    */
   rpc submitDiskBalancerPlan(SubmitDiskBalancerPlanRequestProto)
       returns (SubmitDiskBalancerPlanResponseProto);
+  /**
+   * Cancel an executing plan
+   */
+  rpc cancelDiskBalancerPlan(CancelPlanRequestProto)
+      returns (CancelPlanResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 6d8cde0..8ceb45b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -16,3 +16,6 @@ HDFS-1312 Change Log
     HDFS-9588. DiskBalancer: Add submitDiskbalancer RPC. (Anu Engineer via
     Arpit Agarwal)
 
+    HDFS-9595. DiskBalancer: Add cancelPlan RPC. (Anu Engineer via
+    Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 824f050..27fe803 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -49,6 +49,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Trigge
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -259,4 +261,24 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  /**
+   * Cancel an executing plan.
+   * @param controller - RpcController
+   * @param request  - Request
+   * @return Response.
+   * @throws ServiceException
+   */
+  @Override
+  public CancelPlanResponseProto cancelDiskBalancerPlan(
+      RpcController controller, CancelPlanRequestProto request)
+      throws ServiceException {
+    try {
+      impl.cancelDiskBalancePlan(request.getPlanID());
+      return CancelPlanResponseProto.newBuilder().build();
+    }catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index e06555f..b2d9994 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3310,4 +3310,11 @@ public class DataNode extends ReconfigurableBase
     throw new DiskbalancerException("Not Implemented", 0);
   }
 
+  @Override
+  public void cancelDiskBalancePlan(String planID) throws
+      IOException {
+    checkSuperuserPrivilege();
+    throw new DiskbalancerException("Not Implemented", 0);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
index 43f9953..f0fc776 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
@@ -17,6 +17,7 @@
 
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
@@ -90,6 +91,9 @@ public class GreedyPlanner implements Planner {
   public void balanceVolumeSet(DiskBalancerDataNode node,
                                DiskBalancerVolumeSet vSet, NodePlan plan)
       throws Exception {
+    Preconditions.checkNotNull(vSet);
+    Preconditions.checkNotNull(plan);
+    Preconditions.checkNotNull(node);
     DiskBalancerVolumeSet currentSet = new DiskBalancerVolumeSet(vSet);
 
     while (currentSet.isBalancingNeeded(this.threshold)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0501d430/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index e047d5a..35d3f91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -42,10 +42,10 @@ public class TestDiskBalancerRPC {
   public ExpectedException thrown = ExpectedException.none();
 
   private MiniDFSCluster cluster;
-
+  private Configuration conf;
   @Before
   public void setUp() throws Exception {
-    Configuration conf = new HdfsConfiguration();
+    conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
   }
@@ -59,21 +59,54 @@ public class TestDiskBalancerRPC {
 
   @Test
   public void TestSubmitTestRpc() throws Exception {
-    URI clusterJson = getClass()
-        .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
-    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
-        null);
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(jsonConnector);
+    final int dnIndex = 0;
+    cluster.restartDataNode(dnIndex);
+    cluster.waitActive();
+    ClusterConnector nameNodeConnector =
+        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
     diskBalancerCluster.readClusterInfo();
-    Assert.assertEquals(3, diskBalancerCluster.getNodes().size());
+    Assert.assertEquals(cluster.getDataNodes().size(),
+                                    diskBalancerCluster.getNodes().size());
     diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
-    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
+    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(dnIndex);
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
     NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
         ());
-    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
+    final int planVersion = 0; // So far we support only one version.
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+
+    String planHash = DigestUtils.sha512Hex(plan.toJson());
+
+    // Since submitDiskBalancerPlan is not implemented yet, it throws an
+    // Exception, this will be modified with the actual implementation.
+    thrown.expect(DiskbalancerException.class);
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+
 
+  }
+
+  @Test
+  public void TestCancelTestRpc() throws Exception {
     final int dnIndex = 0;
+    cluster.restartDataNode(dnIndex);
+    cluster.waitActive();
+    ClusterConnector nameNodeConnector =
+        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
+    diskBalancerCluster.readClusterInfo();
+    Assert.assertEquals(cluster.getDataNodes().size(),
+        diskBalancerCluster.getNodes().size());
+    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
+
     final int planVersion = 0; // So far we support only one version.
     DataNode dataNode = cluster.getDataNodes().get(dnIndex);
     String planHash = DigestUtils.sha512Hex(plan.toJson());
@@ -83,5 +116,8 @@ public class TestDiskBalancerRPC {
     thrown.expect(DiskbalancerException.class);
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
 
+    thrown.expect(DiskbalancerException.class);
+    dataNode.cancelDiskBalancePlan(planHash);
+
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/49] hadoop git commit: HDFS-10545. DiskBalancer: PlanCommand should use -fs instead of -uri to be consistent with other hdfs commands. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10545. DiskBalancer: PlanCommand should use -fs instead of -uri to be consistent with other hdfs commands. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0774412e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0774412e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0774412e

Branch: refs/heads/trunk
Commit: 0774412e41856b4ed3eccfa9270165e216d10ab8
Parents: 5b8e1c2
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jun 17 23:31:21 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:01 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0774412e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 67703c4..8a900ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -60,7 +60,7 @@ public class DiskBalancer extends Configured implements Tool {
    * <p>
    * hdfs://namenode.uri or file:///data/myCluster.json
    */
-  public static final String NAMENODEURI = "uri";
+  public static final String NAMENODEURI = "fs";
   /**
    * Computes a plan for a given set of nodes.
    */


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/49] hadoop git commit: HDFS-9469. DiskBalancer: Add Planner. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9469. DiskBalancer: Add Planner. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5724a103
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5724a103
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5724a103

Branch: refs/heads/trunk
Commit: 5724a103161424f4b293ba937f0d0540179f36ac
Parents: e325c6a
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jan 7 14:45:56 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   4 +-
 .../datamodel/DiskBalancerCluster.java          | 114 ++++-
 .../diskbalancer/planner/GreedyPlanner.java     | 259 +++++++++++
 .../server/diskbalancer/planner/MoveStep.java   | 181 ++++++++
 .../server/diskbalancer/planner/NodePlan.java   | 190 ++++++++
 .../server/diskbalancer/planner/Planner.java    |  28 ++
 .../diskbalancer/planner/PlannerFactory.java    |  59 +++
 .../hdfs/server/diskbalancer/planner/Step.java  |  68 +++
 .../diskbalancer/planner/package-info.java      |  46 ++
 .../diskbalancer/DiskBalancerTestUtil.java      |   6 +-
 .../hdfs/server/diskbalancer/TestPlanner.java   | 462 +++++++++++++++++++
 .../diskBalancer/data-cluster-3node-3disk.json  | 380 +++++++++++++++
 12 files changed, 1792 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 8220f88..940e1b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -8,6 +8,8 @@ HDFS-1312 Change Log
 
     HDFS-9526. Fix jackson annotation imports. (Xiaobing Zhou via szetszwo)
 
-    HDFS-9611. DiskBalancer : Replace htrace json imports with jackson.
+    HDFS-9611. DiskBalancer: Replace htrace json imports with jackson.
     (Anu Engineer via Arpit Agarwal)
 
+    HDFS-9469. DiskBalancer: Add Planner. (Anu Engineer via Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
index 91f7eaa..af9e9af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -22,16 +22,26 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Planner;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.PlannerFactory;
 import org.codehaus.jackson.annotate.JsonIgnore;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 /**
  * DiskBalancerCluster represents the nodes that we are working against.
@@ -166,7 +176,7 @@ public class DiskBalancerCluster {
    */
   public void setThreshold(float thresholdPercent) {
     Preconditions.checkState((thresholdPercent >= 0.0f) &&
-        (thresholdPercent <= 100.0f),  "A percentage value expected.");
+        (thresholdPercent <= 100.0f), "A percentage value expected.");
     this.threshold = thresholdPercent;
   }
 
@@ -246,4 +256,106 @@ public class DiskBalancerCluster {
     File outFile = new File(getOutput() + "/" + snapShotName);
     FileUtils.writeStringToFile(outFile, json);
   }
+
+  /**
+   * Creates an Output directory for the cluster output.
+   *
+   * @throws IOException - On failure to create an new directory
+   */
+  public void createOutPutDirectory() throws IOException {
+    if (Files.exists(Paths.get(this.getOutput()))) {
+      LOG.fatal("An output directory already exists at this location. Path : " +
+          this.getOutput());
+      throw new IOException(
+          "An output directory already exists at this location. Path : " +
+              this.getOutput());
+    }
+
+    File f = new File(this.getOutput());
+    if (!f.mkdirs()) {
+      LOG.fatal("Unable to create the output directory. Path : " + this
+          .getOutput());
+      throw new IOException(
+          "Unable to create the output directory. Path : " + this.getOutput());
+    }
+    LOG.info("Output directory created. Path : " + this.getOutput());
+  }
+
+  /**
+   * Compute plan takes a node and constructs a planner that creates a plan that
+   * we would like to follow.
+   * <p/>
+   * This function creates a thread pool and executes a planner on each node
+   * that we are supposed to plan for. Each of these planners return a NodePlan
+   * that we can persist or schedule for execution with a diskBalancer
+   * Executor.
+   *
+   * @param thresholdPercent - in percentage
+   * @return list of NodePlans
+   */
+  public List<NodePlan> computePlan(float thresholdPercent) {
+    List<NodePlan> planList = new LinkedList<>();
+
+    if (nodesToProcess == null) {
+      LOG.warn("Nodes to process is null. No nodes processed.");
+      return planList;
+    }
+
+    int poolSize = computePoolSize(nodesToProcess.size());
+
+    ExecutorService executorService = Executors.newFixedThreadPool(poolSize);
+    List<Future<NodePlan>> futureList = new LinkedList<>();
+    for (int x = 0; x < nodesToProcess.size(); x++) {
+      final DiskBalancerDataNode node = nodesToProcess.get(x);
+      final Planner planner = PlannerFactory
+          .getPlanner(PlannerFactory.GREEDY_PLANNER, node,
+              thresholdPercent);
+      futureList.add(executorService.submit(new Callable<NodePlan>() {
+        @Override
+        public NodePlan call() throws Exception {
+          assert planner != null;
+          return planner.plan(node);
+        }
+      }));
+    }
+
+    for (Future<NodePlan> f : futureList) {
+      try {
+        planList.add(f.get());
+      } catch (InterruptedException e) {
+        LOG.error("Compute Node plan was cancelled or interrupted : ", e);
+      } catch (ExecutionException e) {
+        LOG.error("Unable to compute plan : ", e);
+      }
+    }
+    return planList;
+  }
+
+  /**
+   * Return the number of threads we should launch for this cluster.
+   * <p/>
+   * Here is the heuristic we are using.
+   * <p/>
+   * 1 thread per 100 nodes that we want to process. Minimum nodesToProcess
+   * threads in the pool. Maximum 100 threads in the pool.
+   * <p/>
+   * Generally return a rounded up multiple of 10.
+   *
+   * @return number
+   */
+  private int computePoolSize(int nodeCount) {
+
+    if (nodeCount < 10) {
+      return nodeCount;
+    }
+
+    int threadRatio = nodeCount / 100;
+    int modValue = threadRatio % 10;
+
+    if (((10 - modValue) + threadRatio) > 100) {
+      return 100;
+    } else {
+      return (10 - modValue) + threadRatio;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
new file mode 100644
index 0000000..43f9953
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
@@ -0,0 +1,259 @@
+/**
+ * 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.hdfs.server.diskbalancer.planner;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerVolumeSet;
+import org.apache.hadoop.util.Time;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+
+/**
+ * Greedy Planner is a simple planner that computes the largest possible move at
+ * any point of time given a volumeSet.
+ * <p/>
+ * This is done by choosing the disks with largest  amount of data above and
+ * below the idealStorage and then a move is scheduled between them.
+ */
+public class GreedyPlanner implements Planner {
+  public static final long MB = 1024L * 1024L;
+  public static final long GB = MB * 1024L;
+  public static final long TB = GB * 1024L;
+  static final Log LOG = LogFactory.getLog(GreedyPlanner.class);
+  private final float threshold;
+
+  /**
+   * Constructs a greedy planner.
+   *
+   * @param threshold - Disk tolerance that we are ok with
+   * @param node      - node on which this planner is operating upon
+   */
+  public GreedyPlanner(float threshold, DiskBalancerDataNode node) {
+    this.threshold = threshold;
+  }
+
+  /**
+   * Computes a node plan for the given node.
+   *
+   * @return NodePlan
+   * @throws Exception
+   */
+  @Override
+  public NodePlan plan(DiskBalancerDataNode node) throws Exception {
+    long startTime = Time.monotonicNow();
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
+    LOG.info("Starting plan for Node : " + node.getDataNodeUUID());
+    while (node.isBalancingNeeded(this.threshold)) {
+      for (DiskBalancerVolumeSet vSet : node.getVolumeSets().values()) {
+        balanceVolumeSet(node, vSet, plan);
+      }
+    }
+
+    long endTime = Time.monotonicNow();
+    String message = String
+        .format("Compute Plan for Node : %s took %d ms ",
+            node.getDataNodeUUID(), endTime - startTime);
+    LOG.info(message);
+    return plan;
+  }
+
+  /**
+   * Computes Steps to make a DiskBalancerVolumeSet Balanced.
+   *
+   * @param node
+   * @param vSet - DiskBalancerVolumeSet
+   * @param plan - NodePlan
+   */
+  public void balanceVolumeSet(DiskBalancerDataNode node,
+                               DiskBalancerVolumeSet vSet, NodePlan plan)
+      throws Exception {
+    DiskBalancerVolumeSet currentSet = new DiskBalancerVolumeSet(vSet);
+
+    while (currentSet.isBalancingNeeded(this.threshold)) {
+      removeSkipVolumes(currentSet);
+
+      DiskBalancerVolume lowVolume = currentSet.getSortedQueue().first();
+      DiskBalancerVolume highVolume = currentSet.getSortedQueue().last();
+
+      Step nextStep = null;
+      // ok both volumes bytes used are in the range that we expect
+      // Then we create a move request.
+      if (!lowVolume.isSkip() && !highVolume.isSkip()) {
+        nextStep = computeMove(currentSet, lowVolume, highVolume);
+      } else {
+        LOG.debug("Skipping compute move. lowVolume :" + lowVolume.getPath());
+        LOG.debug("Skipping compute move. highVolume :" + highVolume.getPath());
+      }
+
+      applyStep(nextStep, currentSet, lowVolume, highVolume);
+      if (nextStep != null) {
+        LOG.debug("Step : " + nextStep.toString());
+        plan.addStep(nextStep);
+      }
+    }
+
+    String message = String
+        .format("Disk Volume set %s Type : %s plan completed.",
+            currentSet.getSetID(),
+            currentSet.getVolumes().get(0).getStorageType());
+
+    plan.setNodeName(node.getDataNodeName());
+    plan.setNodeUUID(node.getDataNodeUUID());
+    plan.setTimeStamp(Time.now());
+    plan.setPort(node.getDataNodePort());
+    LOG.info(message);
+  }
+
+  /**
+   * Apply steps applies the current step on to a volumeSet so that we can
+   * compute next steps until we reach the desired goals.
+   *
+   * @param nextStep   - nextStep or Null
+   * @param currentSet - Current Disk BalancerVolume Set we are operating upon
+   * @param lowVolume  - volume
+   * @param highVolume - volume
+   */
+  private void applyStep(Step nextStep, DiskBalancerVolumeSet currentSet,
+                         DiskBalancerVolume lowVolume,
+                         DiskBalancerVolume highVolume) throws Exception {
+
+    long used;
+    if (nextStep != null) {
+      used = lowVolume.getUsed() + nextStep.getBytesToMove();
+      lowVolume.setUsed(used);
+
+      used = highVolume.getUsed() - nextStep.getBytesToMove();
+      highVolume.setUsed(used);
+    }
+
+    // since the volume data changed , we need to recompute the DataDensity.
+    currentSet.computeVolumeDataDensity();
+  }
+
+  /**
+   * Computes a data move from the largest disk we have to smallest disk.
+   *
+   * @param currentSet - Current Disk Set we are working with
+   * @param lowVolume  - Low Data Capacity Volume
+   * @param highVolume - High Data Capacity Volume
+   * @return Step
+   */
+  private Step computeMove(DiskBalancerVolumeSet currentSet,
+                           DiskBalancerVolume lowVolume,
+                           DiskBalancerVolume highVolume) {
+    // Compute how many bytes we can move. First Compute the maximum that
+    // low Volume Can receive, then compute maximum high volume can give
+    // Then take the minimum of those two numbers that is the bytesToMove.
+
+    long maxLowVolumeCanReceive = (long) (
+        (currentSet.getIdealUsed() * lowVolume.computeEffectiveCapacity()) -
+            lowVolume.getUsed());
+
+    // This disk cannot take any more data from any disk.
+    // Remove it from our computation matrix.
+    if (maxLowVolumeCanReceive <= 0) {
+      LOG.debug(lowVolume.getPath() +
+          " Skipping disk from computation. Maximum data size " +
+          "achieved.");
+      lowVolume.setSkip(true);
+    }
+
+    long maxHighVolumeCanGive = highVolume.getUsed() -
+        (long) (currentSet.getIdealUsed() *
+            highVolume.computeEffectiveCapacity());
+    // This volume cannot give any more data, remove it from the
+    // computation matrix
+    if (maxHighVolumeCanGive <= 0) {
+      LOG.debug(highVolume.getPath() +
+          " Skipping disk from computation. Minimum data size " +
+          "achieved.");
+      highVolume.setSkip(true);
+    }
+
+
+    long bytesToMove = Math.min(maxLowVolumeCanReceive, maxHighVolumeCanGive);
+    Step nextStep = null;
+
+    if (bytesToMove > 0) {
+      // Create a new step
+      nextStep = new MoveStep(highVolume, currentSet.getIdealUsed(), lowVolume,
+          bytesToMove, currentSet.getSetID());
+      LOG.debug(nextStep.toString());
+    }
+    return nextStep;
+  }
+
+  /**
+   * Skips this volume if needed.
+   *
+   * @param currentSet - Current Disk set
+   * @param volume     - Volume
+   */
+  private void skipVolume(DiskBalancerVolumeSet currentSet,
+                          DiskBalancerVolume volume) {
+
+    String message = String.format(
+        "Skipping volume. Volume : %s " +
+            "Type : %s Target " +
+            "Number of bytes : %f lowVolume dfsUsed : %d. Skipping this " +
+            "volume from all future balancing calls.", volume.getPath(),
+        volume.getStorageType(),
+        currentSet.getIdealUsed() * volume.getCapacity(), volume.getUsed());
+    volume.setSkip(true);
+    LOG.debug(message);
+  }
+
+  // Removes all volumes which are part of the volumeSet but skip flag is set.
+  private void removeSkipVolumes(DiskBalancerVolumeSet currentSet) {
+    List<DiskBalancerVolume> volumeList = currentSet.getVolumes();
+    Iterator<DiskBalancerVolume> volumeIterator = volumeList.iterator();
+    while (volumeIterator.hasNext()) {
+      DiskBalancerVolume vol = volumeIterator.next();
+      if (vol.isSkip() || vol.isFailed()) {
+        currentSet.removeVolume(vol);
+      }
+    }
+    currentSet.computeVolumeDataDensity();
+  }
+
+  /**
+   * This function is used only for debugging purposes to ensure queue looks
+   * correct.
+   *
+   * @param queue - Queue
+   */
+  private void printQueue(TreeSet<DiskBalancerVolume> queue) {
+    String format = String.format("First Volume : %s, DataDensity : %f",
+        queue.first().getPath(),
+        queue.first().getVolumeDataDensity());
+    LOG.info(format);
+
+    format = String
+        .format("Last Volume : %s, DataDensity : %f%n", queue.last().getPath(),
+            queue.last().getVolumeDataDensity());
+    LOG.info(format);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
new file mode 100644
index 0000000..75af0d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
@@ -0,0 +1,181 @@
+/**
+ * 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.hdfs.server.diskbalancer.planner;
+
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Move step is a step that planner can execute that will move data from one
+ * volume to another.
+ */
+public class MoveStep implements Step {
+  private DiskBalancerVolume sourceVolume;
+  private DiskBalancerVolume destinationVolume;
+  private float idealStorage;
+  private long bytesToMove;
+  private String volumeSetID;
+
+  /**
+   * Constructs a MoveStep for the volume set.
+   *
+   * @param sourceVolume      - Source Disk
+   * @param idealStorage      - Ideal Storage Value for this disk set
+   * @param destinationVolume - Destination dis
+   * @param bytesToMove       - number of bytes to move
+   * @param volumeSetID       - a diskBalancer generated id.
+   */
+  public MoveStep(DiskBalancerVolume sourceVolume, float idealStorage,
+                  DiskBalancerVolume destinationVolume, long bytesToMove,
+                  String volumeSetID) {
+    this.destinationVolume = destinationVolume;
+    this.idealStorage = idealStorage;
+    this.sourceVolume = sourceVolume;
+    this.bytesToMove = bytesToMove;
+    this.volumeSetID = volumeSetID;
+
+  }
+
+  /**
+   * Empty Constructor for JSON serialization.
+   */
+  public MoveStep() {
+  }
+
+  /**
+   * Returns number of bytes to move.
+   *
+   * @return - long
+   */
+  @Override
+  public long getBytesToMove() {
+    return bytesToMove;
+  }
+
+  /**
+   * Gets the destination volume.
+   *
+   * @return - volume
+   */
+  @Override
+  public DiskBalancerVolume getDestinationVolume() {
+    return destinationVolume;
+  }
+
+  /**
+   * Gets the IdealStorage.
+   *
+   * @return float
+   */
+  @Override
+  public float getIdealStorage() {
+    return idealStorage;
+  }
+
+  /**
+   * Gets Source Volume.
+   *
+   * @return -- Source Volume
+   */
+
+  @Override
+  public DiskBalancerVolume getSourceVolume() {
+    return sourceVolume;
+  }
+
+  /**
+   * Gets a volume Set ID.
+   *
+   * @return String
+   */
+  @Override
+  public String getVolumeSetID() {
+    return volumeSetID;
+  }
+
+  /**
+   * Set source volume.
+   *
+   * @param sourceVolume - volume
+   */
+  public void setSourceVolume(DiskBalancerVolume sourceVolume) {
+    this.sourceVolume = sourceVolume;
+  }
+
+  /**
+   * Sets destination volume.
+   *
+   * @param destinationVolume - volume
+   */
+  public void setDestinationVolume(DiskBalancerVolume destinationVolume) {
+    this.destinationVolume = destinationVolume;
+  }
+
+  /**
+   * Sets Ideal Storage.
+   *
+   * @param idealStorage - ideal Storage
+   */
+  public void setIdealStorage(float idealStorage) {
+    this.idealStorage = idealStorage;
+  }
+
+  /**
+   * Sets bytes to move.
+   *
+   * @param bytesToMove - number of bytes
+   */
+  public void setBytesToMove(long bytesToMove) {
+    this.bytesToMove = bytesToMove;
+  }
+
+  /**
+   * Sets volume id.
+   *
+   * @param volumeSetID - volume ID
+   */
+  public void setVolumeSetID(String volumeSetID) {
+    this.volumeSetID = volumeSetID;
+  }
+
+  /**
+   * Returns a string representation of the object.
+   *
+   * @return a string representation of the object.
+   */
+  @Override
+  public String toString() {
+    return String.format("%s\t %s\t %s\t %s%n",
+        this.getSourceVolume().getPath(),
+        this.getDestinationVolume().getPath(),
+        getSizeString(this.getBytesToMove()),
+        this.getDestinationVolume().getStorageType());
+
+  }
+
+  /**
+   * Returns human readable move sizes.
+   *
+   * @param size - bytes being moved.
+   * @return String
+   */
+  @Override
+  public String getSizeString(long size) {
+    return StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
new file mode 100644
index 0000000..995f4ab
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
@@ -0,0 +1,190 @@
+/**
+ * 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.hdfs.server.diskbalancer.planner;
+
+import com.google.common.base.Preconditions;
+import org.codehaus.jackson.annotate.JsonTypeInfo;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.JavaType;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * NodePlan is a set of volumeSetPlans.
+ */
+public class NodePlan {
+  @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS,
+      include = JsonTypeInfo.As.PROPERTY, property = "@class")
+  private List<Step> volumeSetPlans;
+  private String nodeName;
+  private String nodeUUID;
+  private int port;
+  private long timeStamp;
+
+  /**
+   * returns timestamp when this plan was created.
+   *
+   * @return long
+   */
+  public long getTimeStamp() {
+    return timeStamp;
+  }
+
+  /**
+   * Sets the timestamp when this plan was created.
+   *
+   * @param timeStamp
+   */
+  public void setTimeStamp(long timeStamp) {
+    this.timeStamp = timeStamp;
+  }
+
+  /**
+   * Constructs an Empty Node Plan.
+   */
+  public NodePlan() {
+    volumeSetPlans = new LinkedList<>();
+  }
+
+  /**
+   * Constructs an empty NodePlan.
+   */
+  public NodePlan(String datanodeName, int rpcPort) {
+    volumeSetPlans = new LinkedList<>();
+    this.nodeName = datanodeName;
+    this.port = rpcPort;
+  }
+
+  /**
+   * Returns a Map of  VolumeSetIDs and volumeSetPlans.
+   *
+   * @return Map
+   */
+  public List<Step> getVolumeSetPlans() {
+    return volumeSetPlans;
+  }
+
+  /**
+   * Adds a step to the existing Plan.
+   *
+   * @param nextStep - nextStep
+   */
+  void addStep(Step nextStep) {
+    Preconditions.checkNotNull(nextStep);
+    volumeSetPlans.add(nextStep);
+  }
+
+  /**
+   * Sets Node Name.
+   *
+   * @param nodeName - Name
+   */
+  public void setNodeName(String nodeName) {
+    this.nodeName = nodeName;
+  }
+
+  /**
+   * Sets a volume List plan.
+   *
+   * @param volumeSetPlans - List of plans.
+   */
+  public void setVolumeSetPlans(List<Step> volumeSetPlans) {
+    this.volumeSetPlans = volumeSetPlans;
+  }
+
+  /**
+   * Returns the DataNode URI.
+   *
+   * @return URI
+   */
+  public String getNodeName() {
+    return nodeName;
+  }
+
+  /**
+   * Sets the DataNodeURI.
+   *
+   * @param dataNodeName - String
+   */
+  public void setURI(String dataNodeName) {
+    this.nodeName = dataNodeName;
+  }
+
+  /**
+   * Gets the DataNode RPC Port.
+   *
+   * @return port
+   */
+  public int getPort() {
+    return port;
+  }
+
+  /**
+   * Sets the DataNode RPC Port.
+   *
+   * @param port - int
+   */
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  /**
+   * Parses a Json string and converts to NodePlan.
+   *
+   * @param json - Json String
+   * @return NodePlan
+   * @throws IOException
+   */
+  public static NodePlan parseJson(String json) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(json, NodePlan.class);
+  }
+
+  /**
+   * Returns a Json representation of NodePlan.
+   *
+   * @return - json String
+   * @throws IOException
+   */
+  public String toJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    JavaType planType = mapper.constructType(NodePlan.class);
+    return mapper.writerWithType(planType)
+        .writeValueAsString(this);
+  }
+
+  /**
+   * gets the Node UUID.
+   *
+   * @return Node UUID.
+   */
+  public String getNodeUUID() {
+    return nodeUUID;
+  }
+
+  /**
+   * Sets the Node UUID.
+   *
+   * @param nodeUUID - UUID of the node.
+   */
+  public void setNodeUUID(String nodeUUID) {
+    this.nodeUUID = nodeUUID;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Planner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Planner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Planner.java
new file mode 100644
index 0000000..b21b811
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Planner.java
@@ -0,0 +1,28 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.planner;
+
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+
+/**
+ * Planner interface allows different planners to be created.
+ */
+public interface Planner {
+  NodePlan plan(DiskBalancerDataNode node) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
new file mode 100644
index 0000000..ae18e05
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
@@ -0,0 +1,59 @@
+/**
+ * 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.hdfs.server.diskbalancer.planner;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+
+/**
+ * Returns a planner based on the user defined tags.
+ */
+public final class PlannerFactory {
+  static final Log LOG = LogFactory.getLog(PlannerFactory.class);
+
+  public static final String GREEDY_PLANNER = "greedyPlanner";
+
+  /**
+   *  Gets a planner object.
+   * @param plannerName - name of the planner.
+   * @param node - Datanode.
+   * @param threshold - percentage
+   * @return Planner
+   */
+  public static Planner getPlanner(String plannerName,
+                                   DiskBalancerDataNode node, float threshold) {
+    if (plannerName.equals(GREEDY_PLANNER)) {
+      if (LOG.isDebugEnabled()) {
+        String message = String
+            .format("Creating a %s for Node : %s IP : %s ID : %s",
+                GREEDY_PLANNER, node.getDataNodeName(), node.getDataNodeIP(),
+                node.getDataNodeUUID());
+        LOG.debug(message);
+      }
+      return new GreedyPlanner(threshold, node);
+    }
+
+    throw new IllegalArgumentException("Unrecognized planner name : " +
+        plannerName);
+  }
+
+  private PlannerFactory() {
+    // Never constructed
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
new file mode 100644
index 0000000..d87209e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
@@ -0,0 +1,68 @@
+/**
+ * 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.hdfs.server.diskbalancer.planner;
+
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+
+/**
+ * A step in the plan.
+ */
+public interface Step {
+  /**
+   * Return the number of bytes to move.
+   *
+   * @return bytes
+   */
+  long getBytesToMove();
+
+  /**
+   * Gets the destination volume.
+   *
+   * @return - volume
+   */
+  DiskBalancerVolume getDestinationVolume();
+
+  /**
+   * Gets the IdealStorage.
+   *
+   * @return idealStorage
+   */
+  float getIdealStorage();
+
+  /**
+   * Gets Source Volume.
+   *
+   * @return -- Source Volume
+   */
+  DiskBalancerVolume getSourceVolume();
+
+  /**
+   * Gets a volume Set ID.
+   *
+   * @return String
+   */
+  String getVolumeSetID();
+
+  /**
+   * Returns a String representation of the Step Size.
+   *
+   * @return String
+   */
+  String getSizeString(long size);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/package-info.java
new file mode 100644
index 0000000..bbcc121
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/package-info.java
@@ -0,0 +1,46 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.planner;
+/**
+ * Planner takes a DiskBalancerVolumeSet, threshold and
+ * computes a series of steps that lead to an even data
+ * distribution between volumes of this DiskBalancerVolumeSet.
+ *
+ * The main classes of this package are steps and planner.
+ *
+ * Here is a high level view of how planner operates:
+ *
+ * DiskBalancerVolumeSet current = volumeSet;
+ *
+ * while(current.isBalancingNeeded(thresholdValue)) {
+ *
+ *   // Creates a plan , like move 20 GB data from v1 -> v2
+ *   Step step = planner.plan(current, thresholdValue);
+ *
+ *   // we add that to our plan
+ *   planner.addStep(current, step);
+ *
+ *   // Apply the step to current state of the diskSet to
+ *   //compute the next state
+ *   current = planner.apply(current, step);
+ * }
+ *
+ * //when we are done , return the list of steps
+ * return planner;
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
index 5e3f4bf..9613919 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -33,9 +33,9 @@ import java.util.UUID;
  * Helper class to create various cluster configrations at run time.
  */
 public class DiskBalancerTestUtil {
-  // we modeling disks here, hence HDD style units
-  public static final long GB = 1000000000L;
-  public static final long TB = 1000000000000L;
+  public static final long MB = 1024 * 1024L;
+  public static final long GB = MB * 1024L;
+  public static final long TB = GB * 1024L;
   private static int[] diskSizes =
       {1, 2, 3, 4, 5, 6, 7, 8, 9, 100, 200, 300, 400, 500, 600, 700, 800, 900};
   Random rand;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
new file mode 100644
index 0000000..f756104
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
@@ -0,0 +1,462 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerVolumeSet;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.List;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestPlanner {
+  static final Logger LOG =
+      LoggerFactory.getLogger(TestPlanner.class);
+
+  @Test
+  public void TestGreedyPlannerBalanceVolumeSet() throws Exception {
+    URI clusterJson = getClass()
+        .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
+    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
+        null);
+    DiskBalancerCluster cluster = new DiskBalancerCluster(jsonConnector);
+    cluster.readClusterInfo();
+    Assert.assertEquals(3, cluster.getNodes().size());
+    cluster.setNodesToProcess(cluster.getNodes());
+    DiskBalancerDataNode node = cluster.getNodes().get(0);
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+  }
+
+  @Test
+  public void TestGreedyPlannerComputePlan() throws Exception {
+    URI clusterJson = getClass()
+        .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
+    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
+        null);
+    DiskBalancerCluster cluster = new DiskBalancerCluster(jsonConnector);
+    cluster.readClusterInfo();
+    Assert.assertEquals(3, cluster.getNodes().size());
+    cluster.setNodesToProcess(cluster.getNodes());
+    List<NodePlan> plan = cluster.computePlan(10.0f);
+    Assert.assertNotNull(plan);
+  }
+
+  private DiskBalancerVolume createVolume(String path, int capacityInGB,
+                                          int usedInGB) {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolume volume = util.createRandomVolume(StorageType.SSD);
+    volume.setPath(path);
+    volume.setCapacity(capacityInGB * DiskBalancerTestUtil.GB);
+    volume.setReserved(0);
+    volume.setUsed(usedInGB * DiskBalancerTestUtil.GB);
+    return volume;
+  }
+
+  @Test
+  public void TestGreedyPlannerNoNodeCluster() throws Exception {
+    GreedyPlanner planner = new GreedyPlanner(10.0f, null);
+    assertNotNull(planner);
+  }
+
+  @Test
+  public void TestGreedyPlannerNoVolumeTest() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+    List<NodePlan> planList = cluster.computePlan(10.0f);
+    assertNotNull(planList);
+  }
+
+  @Test
+  public void TestGreedyPlannerOneVolumeNoPlanTest() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
+    node.addVolume(volume30);
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    // With a single volume we should not have any plans for moves.
+    assertEquals(0, plan.getVolumeSetPlans().size());
+  }
+
+  @Test
+  public void TestGreedyPlannerTwoVolume() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
+    DiskBalancerVolume volume10 = createVolume("volume10", 100, 10);
+
+    node.addVolume(volume10);
+    node.addVolume(volume30);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    // We should have only one planned move from
+    // volume30 to volume10 of 10 GB Size.
+
+    assertEquals(1, plan.getVolumeSetPlans().size());
+    Step step = plan.getVolumeSetPlans().get(0);
+    assertEquals("volume30", step.getSourceVolume().getPath());
+    assertEquals("volume10", step.getDestinationVolume().getPath());
+    assertEquals("10 G", step.getSizeString(step.getBytesToMove()));
+  }
+
+  /**
+   * In this test we pass 3 volumes with 30, 20 and 10 GB of data used. We
+   * expect the planner to print out 20 GB on each volume.
+   * <p/>
+   * That is the plan should say move 10 GB from volume30 to volume10.
+   */
+  @Test
+  public void TestGreedyPlannerEqualizeData() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
+    DiskBalancerVolume volume20 = createVolume("volume20", 100, 20);
+    DiskBalancerVolume volume10 = createVolume("volume10", 100, 10);
+
+    node.addVolume(volume10);
+    node.addVolume(volume20);
+    node.addVolume(volume30);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    // We should have only one planned move from
+    // volume30 to volume10 of 10 GB Size.
+
+    assertEquals(1, plan.getVolumeSetPlans().size());
+    Step step = plan.getVolumeSetPlans().get(0);
+    assertEquals("volume30", step.getSourceVolume().getPath());
+    assertEquals("volume10", step.getDestinationVolume().getPath());
+    assertEquals("10 G", step.getSizeString(step.getBytesToMove()));
+  }
+
+  @Test
+  public void TestGreedyPlannerEqualDisksNoMoves() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // All disks have same capacity of data
+    DiskBalancerVolume volume1 = createVolume("volume1", 100, 30);
+    DiskBalancerVolume volume2 = createVolume("volume2", 100, 30);
+    DiskBalancerVolume volume3 = createVolume("volume3", 100, 30);
+
+    node.addVolume(volume1);
+    node.addVolume(volume2);
+    node.addVolume(volume3);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    // since we have same size of data in all disks , we should have
+    // no moves planned.
+    assertEquals(0, plan.getVolumeSetPlans().size());
+  }
+
+  @Test
+  public void TestGreedyPlannerMoveFromSingleDisk() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // All disks have same capacity of data
+    DiskBalancerVolume volume1 = createVolume("volume100", 200, 100);
+    DiskBalancerVolume volume2 = createVolume("volume0-1", 200, 0);
+    DiskBalancerVolume volume3 = createVolume("volume0-2", 200, 0);
+
+    node.addVolume(volume1);
+    node.addVolume(volume2);
+    node.addVolume(volume3);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    // We should see 2 move plans. One from volume100 to volume0-1
+    // and another from volume100 to volume0-2
+
+    assertEquals(2, plan.getVolumeSetPlans().size());
+    Step step = plan.getVolumeSetPlans().get(0);
+    assertEquals("volume100", step.getSourceVolume().getPath());
+    assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
+
+    step = plan.getVolumeSetPlans().get(1);
+    assertEquals("volume100", step.getSourceVolume().getPath());
+    assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
+  }
+
+  @Test
+  public void TestGreedyPlannerThresholdTest() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume volume1 = createVolume("volume100", 1000, 100);
+    DiskBalancerVolume volume2 = createVolume("volume0-1", 300, 0);
+    DiskBalancerVolume volume3 = createVolume("volume0-2", 300, 0);
+
+    node.addVolume(volume1);
+    node.addVolume(volume2);
+    node.addVolume(volume3);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    //We should see NO moves since the total data on the volume100
+    // is less than or equal to threashold value that we pass, which is 10%
+    assertEquals(0, plan.getVolumeSetPlans().size());
+
+    // for this new planner we are passing 1% as as threshold value
+    // hence planner must move data if possible.
+    GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
+    NodePlan newPlan = new NodePlan(node.getDataNodeName(), node
+        .getDataNodePort());
+    newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), newPlan);
+
+    assertEquals(2, newPlan.getVolumeSetPlans().size());
+
+    // Move size should say move 19 GB
+    // Here is how the math works out.
+    // TotalCapacity = 1000 + 300 + 300 = 1600 GB
+    // TotolUsed = 100
+    // Expected data% on each disk = 0.0625
+    // On Disk (volume0-1) = 300 * 0.0625 - 18.75 -- We round it up
+    // in the display string -- hence 18.8 GB, it will be same on volume 2 too.
+    // since they are equal sized disks with same used capacity
+
+    Step step = newPlan.getVolumeSetPlans().get(0);
+    assertEquals("volume100", step.getSourceVolume().getPath());
+    assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
+
+    step = newPlan.getVolumeSetPlans().get(1);
+    assertEquals("volume100", step.getSourceVolume().getPath());
+    assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
+  }
+
+  @Test
+  public void TestGreedyPlannerPlanWithDifferentDiskSizes() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume volume1 = createVolume("volume100", 1000, 100);
+    DiskBalancerVolume volume2 = createVolume("volume0-1", 500, 0);
+    DiskBalancerVolume volume3 = createVolume("volume0-2", 250, 0);
+
+    node.addVolume(volume1);
+    node.addVolume(volume2);
+    node.addVolume(volume3);
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
+    NodePlan newPlan = new NodePlan(node.getDataNodeName(), node
+        .getDataNodePort());
+    newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), newPlan);
+
+    assertEquals(2, newPlan.getVolumeSetPlans().size());
+
+    // Move size should say move 26.6 GB and 13.3 GB
+    // Here is how the math works out.
+    // TotalCapacity = 1000 + 500 + 250 = 1750 GB
+    // TotolUsed = 100
+    // Expected data% on each disk = 0.05714
+    // On Disk (volume0-1) = 500 * 0.05714 = 28.57
+    // on Voulume0-2 = 300 * 0.05714 = 14.28
+
+    for (Step step : newPlan.getVolumeSetPlans()) {
+
+      if (step.getDestinationVolume().getPath().equals("volume0-1")) {
+        assertEquals("volume100", step.getSourceVolume().getPath());
+        assertEquals("28.6 G",
+            step.getSizeString(step.getBytesToMove()));
+      }
+
+      if (step.getDestinationVolume().getPath().equals("volume0-2")) {
+        assertEquals("volume100", step.getSourceVolume().getPath());
+        assertEquals("14.3 G",
+            step.getSizeString(step.getBytesToMove()));
+      }
+    }
+
+    Step step = newPlan.getVolumeSetPlans().get(0);
+    assertEquals(0.05714f, step.getIdealStorage(), 0.001f);
+  }
+
+  @Test
+  public void TestLoadsCorrectClusterConnector() throws Exception {
+    ClusterConnector connector = ConnectorFactory.getCluster(getClass()
+            .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI()
+        , null);
+    assertEquals(connector.getClass().toString(),
+        "class org.apache.hadoop.hdfs.server.diskbalancer.connectors." +
+            "JsonNodeConnector");
+
+  }
+
+  @Test
+  public void TestPlannerScale() throws Exception {
+    final int diskCount = 256; // it is rare to see more than 48 disks
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolumeSet vSet =
+        util.createRandomVolumeSet(StorageType.SSD, diskCount);
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+    int diskNum = 0;
+    for (DiskBalancerVolume vol : vSet.getVolumes()) {
+      vol.setPath("volume" + diskNum++);
+      node.addVolume(vol);
+    }
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+
+    GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
+    NodePlan newPlan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
+    newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"),
+        newPlan);
+
+    // Assuming that our random disks at least generated one step
+    assertTrue("No Steps Generated from random disks, very unlikely",
+        newPlan.getVolumeSetPlans().size() > 0);
+
+    assertTrue("Steps Generated less than disk count - false",
+        newPlan.getVolumeSetPlans().size() < diskCount);
+    LOG.info("Number of steps are : %d%n", newPlan.getVolumeSetPlans().size());
+
+  }
+
+  @Test
+  public void TestNodePlanSerialize() throws Exception {
+    final int diskCount = 12;
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolumeSet vSet =
+        util.createRandomVolumeSet(StorageType.SSD, diskCount);
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+    int diskNum = 0;
+    for (DiskBalancerVolume vol : vSet.getVolumes()) {
+      vol.setPath("volume" + diskNum++);
+      node.addVolume(vol);
+    }
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+
+    GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
+    NodePlan newPlan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
+    newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"),
+        newPlan);
+    String planString = newPlan.toJson();
+    assertNotNull(planString);
+    NodePlan copy = NodePlan.parseJson(planString);
+    assertNotNull(copy);
+    assertEquals(newPlan.getVolumeSetPlans().size(),
+        copy.getVolumeSetPlans().size());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5724a103/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-3node-3disk.json
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-3node-3disk.json b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-3node-3disk.json
new file mode 100644
index 0000000..69ed496
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-3node-3disk.json
@@ -0,0 +1,380 @@
+{
+  "nodes": [
+    {
+      "nodeDataDensity": 1.4248891,
+      "volumeSets": {
+        "SSD": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/XRH5XWdG2x",
+              "capacity": 4000000000000,
+              "storageType": "SSD",
+              "used": 1993901091269,
+              "reserved": 769911586292,
+              "uuid": "766f11fc-78e0-4a0c-9e16-0061cdfd1ccf",
+              "failed": false,
+              "volumeDataDensity": -0.1983375,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/AL0GSv1PHW",
+              "capacity": 400000000000,
+              "storageType": "SSD",
+              "used": 127190645921,
+              "reserved": 35600180269,
+              "uuid": "1523689f-9774-4c7d-a756-ede0c2e16d7c",
+              "failed": false,
+              "volumeDataDensity": 0.069911,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/pn0NypyAVX",
+              "capacity": 7000000000000,
+              "storageType": "SSD",
+              "used": 2256250270190,
+              "reserved": 146185545100,
+              "uuid": "51faf521-14f2-4f45-b959-10f062ff8b27",
+              "failed": false,
+              "volumeDataDensity": 0.08975619,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "SSD"
+        },
+        "RAM_DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/3leXTZTkGL",
+              "capacity": 3000000000000,
+              "storageType": "RAM_DISK",
+              "used": 1555926085343,
+              "reserved": 341478213760,
+              "uuid": "a322a803-afc5-45f3-ab70-4e064ce5bcfc",
+              "failed": false,
+              "volumeDataDensity": 0.011353016,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/L91eKShSxW",
+              "capacity": 900000000000,
+              "storageType": "RAM_DISK",
+              "used": 604470250477,
+              "reserved": 123665018290,
+              "uuid": "35dea1b4-b33a-42e3-82a7-92ae089cfc04",
+              "failed": false,
+              "volumeDataDensity": -0.18200749,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/QogvU2WUij",
+              "capacity": 500000000000,
+              "storageType": "RAM_DISK",
+              "used": 178163834274,
+              "reserved": 15128599317,
+              "uuid": "0dba5c8e-74c8-4e42-a004-83c91211548c",
+              "failed": false,
+              "volumeDataDensity": 0.22916734,
+              "transient": true
+            }
+          ],
+          "transient": true,
+          "storageType" : "RAM_DISK"
+        },
+        "DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/hIDn1xAOE0",
+              "capacity": 100000000000,
+              "storageType": "DISK",
+              "used": 32390769198,
+              "reserved": 16882367031,
+              "uuid": "e40a4777-bc7e-4447-81c1-ab4bb13c879d",
+              "failed": false,
+              "volumeDataDensity": 0.43902066,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/lbAmdQf3Zl",
+              "capacity": 300000000000,
+              "storageType": "DISK",
+              "used": 291508834009,
+              "reserved": 8187128694,
+              "uuid": "0bd97d41-0373-4cfa-9613-cc9a5de16d81",
+              "failed": false,
+              "volumeDataDensity": -0.17023957,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/noTvhjLIXR",
+              "capacity": 400000000000,
+              "storageType": "DISK",
+              "used": 298210106531,
+              "reserved": 24241758276,
+              "uuid": "028b8ffc-0ed1-4985-8f47-3e1a3ab0b3ef",
+              "failed": false,
+              "volumeDataDensity": 0.035096347,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "DISK"
+        }
+      },
+      "dataNodeUUID": "21db0945-577a-4e7b-870a-96578581c6c9"
+    },
+    {
+      "nodeDataDensity": 0.8060421,
+      "volumeSets": {
+        "SSD": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/g1VJ6Lp28b",
+              "capacity": 200000000000,
+              "storageType": "SSD",
+              "used": 44933330586,
+              "reserved": 17521745353,
+              "uuid": "9b5653cb-898e-41fa-97b6-f779933691cc",
+              "failed": false,
+              "volumeDataDensity": 0.0001810193,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/ikZC6r6r4q",
+              "capacity": 500000000000,
+              "storageType": "SSD",
+              "used": 153055238218,
+              "reserved": 8802560618,
+              "uuid": "2a261be8-fe18-410d-8242-3b329694bb30",
+              "failed": false,
+              "volumeDataDensity": -0.06517579,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/tY2J60mopD",
+              "capacity": 800000000000,
+              "storageType": "SSD",
+              "used": 164404778126,
+              "reserved": 3045113974,
+              "uuid": "3d06571b-dac6-474c-9cd0-19e86e40d30b",
+              "failed": false,
+              "volumeDataDensity": 0.04012917,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "SSD"
+        },
+        "RAM_DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/i3f6OMVrET",
+              "capacity": 600000000000,
+              "storageType": "RAM_DISK",
+              "used": 246381206139,
+              "reserved": 69743311089,
+              "uuid": "29a0b57f-24a9-41ec-adf1-7eb8413f6498",
+              "failed": false,
+              "volumeDataDensity": 0.3652115,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/7kSdJOfJD1",
+              "capacity": 5000000000000,
+              "storageType": "RAM_DISK",
+              "used": 4392762782218,
+              "reserved": 82713440534,
+              "uuid": "bb992bd1-1170-468c-8069-d4352bb7d748",
+              "failed": false,
+              "volumeDataDensity": -0.063474,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/5xT8j5WcX8",
+              "capacity": 7000000000000,
+              "storageType": "RAM_DISK",
+              "used": 5527792007694,
+              "reserved": 196106476603,
+              "uuid": "2fbb7778-cdfa-4a69-bc3b-3fedf646447f",
+              "failed": false,
+              "volumeDataDensity": 0.017411172,
+              "transient": true
+            }
+          ],
+          "transient": true,
+          "storageType" : "RAM_DISK"
+        },
+        "DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/3MVCHjP1if",
+              "capacity": 800000000000,
+              "storageType": "DISK",
+              "used": 26874069736,
+              "reserved": 132601810938,
+              "uuid": "f37091af-c6e3-4b59-8e42-65ffeace0458",
+              "failed": false,
+              "volumeDataDensity": 0.19469382,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/r8k9R3Drwn",
+              "capacity": 2000000000000,
+              "storageType": "DISK",
+              "used": 567876985921,
+              "reserved": 54682979334,
+              "uuid": "78af1edc-1fb4-4fb0-a023-23f9b1851ff0",
+              "failed": false,
+              "volumeDataDensity": -0.05695927,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/zSAxOfDmNL",
+              "capacity": 7000000000000,
+              "storageType": "DISK",
+              "used": 1621163451565,
+              "reserved": 181715853004,
+              "uuid": "d6271e5a-48ef-4d14-a072-0697a19e8935",
+              "failed": false,
+              "volumeDataDensity": -0.0028063506,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "DISK"
+        }
+      },
+      "dataNodeUUID": "0fd72405-9a12-4c2b-bd47-240fe50b4f6f"
+    },
+    {
+      "nodeDataDensity": 2.3369348,
+      "volumeSets": {
+        "SSD": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/ya7mTDxsMl",
+              "capacity": 300000000000,
+              "storageType": "SSD",
+              "used": 46742894418,
+              "reserved": 56370966514,
+              "uuid": "85f70090-e554-4d8d-977f-8c20b3d8afd1",
+              "failed": false,
+              "volumeDataDensity": 0.23372014,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/EMm7IeWXLR",
+              "capacity": 2000000000000,
+              "storageType": "SSD",
+              "used": 1038557653395,
+              "reserved": 56968564294,
+              "uuid": "03f7c984-4bdf-4f3f-9705-e731b4790c55",
+              "failed": false,
+              "volumeDataDensity": -0.10892275,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/Qs8ZmyXQcz",
+              "capacity": 700000000000,
+              "storageType": "SSD",
+              "used": 84948151846,
+              "reserved": 136893558033,
+              "uuid": "554073cc-0daa-4c16-9339-f3185b6d19be",
+              "failed": false,
+              "volumeDataDensity": 0.27472478,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "SSD"
+        },
+        "RAM_DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/5ScZuQjsd6",
+              "capacity": 300000000000,
+              "storageType": "RAM_DISK",
+              "used": 6822681510,
+              "reserved": 7487147526,
+              "uuid": "d73d0226-88ea-4e68-801e-c84e02f83cda",
+              "failed": false,
+              "volumeDataDensity": 0.53381115,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/S4fqaBOges",
+              "capacity": 200000000000,
+              "storageType": "RAM_DISK",
+              "used": 155874561110,
+              "reserved": 19966896109,
+              "uuid": "dd88b2da-d274-4866-93c6-afbf2c00cd24",
+              "failed": false,
+              "volumeDataDensity": -0.308675,
+              "transient": true
+            },
+            {
+              "path": "\/tmp\/disk\/s480iw7GqH",
+              "capacity": 900000000000,
+              "storageType": "RAM_DISK",
+              "used": 600902618585,
+              "reserved": 1964017663,
+              "uuid": "56f4a981-3eca-492e-8169-bd37325ed611",
+              "failed": false,
+              "volumeDataDensity": -0.11199421,
+              "transient": true
+            }
+          ],
+          "transient": true,
+          "storageType" : "RAM_DISK"
+        },
+        "DISK": {
+          "volumes": [
+            {
+              "path": "\/tmp\/disk\/xH5Gyutu4r",
+              "capacity": 5000000000000,
+              "storageType": "DISK",
+              "used": 265260533721,
+              "reserved": 273894446207,
+              "uuid": "e03fb8d8-2a7c-4f7b-a588-42f18a4e687b",
+              "failed": false,
+              "volumeDataDensity": 0.104224004,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/ZFGQuCn4Y2",
+              "capacity": 700000000000,
+              "storageType": "DISK",
+              "used": 190052488732,
+              "reserved": 41189291634,
+              "uuid": "a454f1f2-fa9a-45c8-8909-22e63ae1dc3f",
+              "failed": false,
+              "volumeDataDensity": -0.12812747,
+              "transient": false
+            },
+            {
+              "path": "\/tmp\/disk\/DD1sDuwvA4",
+              "capacity": 900000000000,
+              "storageType": "DISK",
+              "used": 531016632774,
+              "reserved": 133837244479,
+              "uuid": "50d7ede3-5b2c-4ca9-9253-ba5da8a17cd8",
+              "failed": false,
+              "volumeDataDensity": -0.5327353,
+              "transient": false
+            }
+          ],
+          "transient": false,
+          "storageType" : "DISK"
+        }
+      },
+      "dataNodeUUID": "de5f4f9c-a639-4877-8baf-2cd869f0594c"
+    }
+  ],
+  "exclusionList": [
+
+  ],
+  "inclusionList": [
+
+  ],
+  "threshold": 0
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/49] hadoop git commit: HDFS-9461. DiskBalancer: Add Report Command. Contributed by Xiaobing Zhou.

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-64node-3disk.json
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-64node-3disk.json b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-64node-3disk.json
new file mode 100644
index 0000000..4293a84
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-64node-3disk.json
@@ -0,0 +1,9484 @@
+{
+   "exclusionList":[
+
+   ],
+   "inclusionList":[
+
+   ],
+   "nodes":[
+      {
+         "nodeDataDensity":1.4280236,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/lDbBDbMk40",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":2460994155387,
+                     "reserved":215232314484,
+                     "uuid":"4e893114-c28b-4b4a-8254-011bb2fb04d2",
+                     "failed":false,
+                     "volumeDataDensity":-0.033956468,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tseMsFTGAW",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":4912132221384,
+                     "reserved":1080424540820,
+                     "uuid":"25a81a58-ac24-44dd-ab21-bd1a95789ebc",
+                     "failed":false,
+                     "volumeDataDensity":0.019966006,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/U2L8YlldVc",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":464539264350,
+                     "reserved":81147677025,
+                     "uuid":"f39ad7e9-2dd5-495c-8e56-27f68afae5e7",
+                     "failed":false,
+                     "volumeDataDensity":-0.045543134,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"ba2f77f6-5e22-4e7b-9dd6-a9bd0a66a781",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/JyjJn550Km",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":88789869571,
+                     "reserved":3536734091,
+                     "uuid":"979ad2e1-eb98-47aa-93d4-ec0ac7b6585b",
+                     "failed":false,
+                     "volumeDataDensity":-0.21953386,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/lllcBYVFqp",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":112503075304,
+                     "reserved":87005858853,
+                     "uuid":"81fe7b93-cc01-49b7-94d9-e73112c12c0b",
+                     "failed":false,
+                     "volumeDataDensity":0.42851037,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Z7sx2FUNbz",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":811474207110,
+                     "reserved":64543764212,
+                     "uuid":"0f83d59c-b584-4023-882f-f4a49c7c3c26",
+                     "failed":false,
+                     "volumeDataDensity":-0.1665448,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"87b66d63-2e4e-4842-b56c-4eba8925b547",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/RXmrzV9NBe",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":2871238941190,
+                     "reserved":433979300088,
+                     "uuid":"9dd322af-a681-4391-a9b1-1f4d6956543b",
+                     "failed":false,
+                     "volumeDataDensity":-0.23244298,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/B6CXulqDoZ",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":2667119852903,
+                     "reserved":849827345950,
+                     "uuid":"cd877bca-e7ea-4a63-b2f5-2b05d862281b",
+                     "failed":false,
+                     "volumeDataDensity":0.19970834,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/XV965gvRZy",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":4792424862969,
+                     "reserved":678185920093,
+                     "uuid":"06eac0f1-4762-43f9-91a2-ec2ae4c114a7",
+                     "failed":false,
+                     "volumeDataDensity":-0.08181757,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"f9aaed43-3ba3-41a3-8349-e843d337b690",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"40776825-7619-44f5-8ef3-cc78cc29f92e",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.258185,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/UdU674y4Le",
+                     "capacity":200000000000,
+                     "storageType":"DISK",
+                     "used":97341014786,
+                     "reserved":34802455330,
+                     "uuid":"dfb5269f-8b7a-4158-8900-b108c134e248",
+                     "failed":false,
+                     "volumeDataDensity":-0.26675877,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/0u2JzFRe7X",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":429153603662,
+                     "reserved":54437524292,
+                     "uuid":"658d95fb-00cc-4feb-a991-b547a366a6da",
+                     "failed":false,
+                     "volumeDataDensity":-0.6406914,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Y1COYV7NUw",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":293450715314,
+                     "reserved":68146575084,
+                     "uuid":"551f10dc-a93b-4eab-a754-8561d0ae08f6",
+                     "failed":false,
+                     "volumeDataDensity":0.17058018,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"b4bb6155-ac06-4285-a418-6acf77bba24e",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/FeTegfBq9T",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":82485249867,
+                     "reserved":40161845337,
+                     "uuid":"c3716133-679c-4ced-b144-834e54ef4b5e",
+                     "failed":false,
+                     "volumeDataDensity":0.25487903,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/5MZO6UoDvc",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":158361631495,
+                     "reserved":7613498953,
+                     "uuid":"e3478ec3-9d52-4814-8f54-526f2335215d",
+                     "failed":false,
+                     "volumeDataDensity":-0.13940075,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/T0ijHlS78u",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1150112429189,
+                     "reserved":393991064796,
+                     "uuid":"b4d68a48-7e47-4587-8999-89bf370d0e05",
+                     "failed":false,
+                     "volumeDataDensity":-0.039114237,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"ce9a253b-7d04-47b9-8daa-33c3012db299",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/eOGwJDhONn",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":1744049587341,
+                     "reserved":621917739371,
+                     "uuid":"33dd825d-c98a-48ae-b056-4a731bf1b24c",
+                     "failed":false,
+                     "volumeDataDensity":0.043477967,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/X6qlqTB6nb",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":341953995354,
+                     "reserved":30310644213,
+                     "uuid":"c29704ac-7056-4c2b-a6de-3cdbc3047ca0",
+                     "failed":false,
+                     "volumeDataDensity":-0.10099676,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/1Vr4pzbV3G",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":377602412191,
+                     "reserved":57807422763,
+                     "uuid":"12f1962e-9c8f-49d8-99df-6a9384a901de",
+                     "failed":false,
+                     "volumeDataDensity":-0.602286,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"fb39c3e4-6d15-481d-bbde-352fe141af33",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"ae7fb310-e968-413e-921b-d1de58e491a8",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7007312,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DwFdOqOaTW",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":99168160561,
+                     "reserved":137211891116,
+                     "uuid":"c181372e-666f-4e49-87f3-9f7bbbaff1b3",
+                     "failed":false,
+                     "volumeDataDensity":0.31557345,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ljcYXTUkHS",
+                     "capacity":9000000000000,
+                     "storageType":"DISK",
+                     "used":3619972625187,
+                     "reserved":1469441796592,
+                     "uuid":"c53c8b78-0161-4e17-83ba-fd2de0be5756",
+                     "failed":false,
+                     "volumeDataDensity":-0.035123497,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/5SOOyTYOVs",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":12041899900,
+                     "reserved":19597517194,
+                     "uuid":"9c74929e-37a1-4de1-95d1-315d71a7f10b",
+                     "failed":false,
+                     "volumeDataDensity":0.29581067,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"01a1a0b3-38ad-4c25-9d40-05fe259443f3",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/wj9qG4IhYy",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1455580728735,
+                     "reserved":738605855282,
+                     "uuid":"89c662ad-3445-4845-9696-1d7e3b679bfa",
+                     "failed":false,
+                     "volumeDataDensity":-0.037234724,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/e9VPWCY0PQ",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2288368986641,
+                     "reserved":345518725926,
+                     "uuid":"a56a599d-dac8-4b5f-87ff-a5270331214d",
+                     "failed":false,
+                     "volumeDataDensity":-0.104465574,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/r6FmT6DaSi",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":857858924121,
+                     "reserved":695088410600,
+                     "uuid":"026ed3bd-4c6f-4e93-896a-745399ea6380",
+                     "failed":false,
+                     "volumeDataDensity":0.121982425,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"9df73bb6-30d8-4a9a-809a-eb270ac8a1b7",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/swJswpkvRs",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":588291065076,
+                     "reserved":242685800023,
+                     "uuid":"5f2ad8e0-1a72-4216-b319-7f91ddf1c3e6",
+                     "failed":false,
+                     "volumeDataDensity":0.20023128,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/KhBiEleI9E",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":102256062721,
+                     "reserved":10811440676,
+                     "uuid":"95a71fc2-0082-477c-a6e9-0abd22e960f3",
+                     "failed":false,
+                     "volumeDataDensity":-0.06630537,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Gc5FHFaeN3",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":2065149709238,
+                     "reserved":454502820734,
+                     "uuid":"f9883090-a0a6-4043-898b-d34e8ae5be47",
+                     "failed":false,
+                     "volumeDataDensity":-0.5240041,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"34fa1fc5-c171-48e6-8051-8db2f426ba7b",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"a2f2b5f2-1d7d-453f-827c-fd7029bec3d1",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7328286,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/uFDeIR28Nh",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":2658190506687,
+                     "reserved":656855219905,
+                     "uuid":"99c42086-cc16-4a58-8a1f-245317af0be5",
+                     "failed":false,
+                     "volumeDataDensity":0.07108402,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8Fgier51ru",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":227897747788,
+                     "reserved":468068973,
+                     "uuid":"ba715c9a-5c48-4491-9bff-a229dbeb43c8",
+                     "failed":false,
+                     "volumeDataDensity":-0.07771945,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ra98HSUe4y",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":957364784699,
+                     "reserved":16410806207,
+                     "uuid":"a4b3b9d5-5f57-4975-aaf2-5f6fabd04cd2",
+                     "failed":false,
+                     "volumeDataDensity":-0.2902112,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"bd40e4ed-3dcc-4e4e-b6b0-2e36119338da",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/6IvCbQyBe2",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":172904840421,
+                     "reserved":39039216509,
+                     "uuid":"d79d570d-3156-4053-a411-8aaf01a92f36",
+                     "failed":false,
+                     "volumeDataDensity":-0.4951154,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/gkBUhdkmDJ",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":9731949053,
+                     "reserved":41360556778,
+                     "uuid":"80ee07d6-a620-42b9-b771-a8f304b7b6e5",
+                     "failed":false,
+                     "volumeDataDensity":0.15462668,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/aU4fkiT0ca",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4658882988,
+                     "reserved":1115597025,
+                     "uuid":"7ecc9a71-a479-47e8-8b02-a525edd0ada3",
+                     "failed":false,
+                     "volumeDataDensity":0.1203404,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"924f689d-29a6-4e4a-9ba8-b3264bbb4a82",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/3CUBl1Djia",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":4573882347156,
+                     "reserved":581462411054,
+                     "uuid":"9dc04655-dc29-476d-a4cd-4b8dbcd3ca9f",
+                     "failed":false,
+                     "volumeDataDensity":0.030687451,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/qrCDyQSXj6",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":57576879899,
+                     "reserved":19127812291,
+                     "uuid":"e1a70eb5-5406-49ba-ae31-62a4862cc96d",
+                     "failed":false,
+                     "volumeDataDensity":-0.13795084,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/h7ntGGMWFB",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":646757200269,
+                     "reserved":3882110491,
+                     "uuid":"ab82cf3c-5e39-440b-a3c3-3fcb3ad8834e",
+                     "failed":false,
+                     "volumeDataDensity":-0.3550933,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"45191b87-579b-483f-a2cb-74507c606cc8",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"37e2a461-dfe1-45a4-ae43-7094e4403121",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.5308468,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/hDaw9bjjVY",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":255916093716,
+                     "reserved":6826603386,
+                     "uuid":"094655a6-8649-400b-9ffe-361a08eeb0fd",
+                     "failed":false,
+                     "volumeDataDensity":0.23470002,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/RUishrCcdm",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":3809696766728,
+                     "reserved":98528272818,
+                     "uuid":"c36dd08a-5811-42dd-9515-c58f6e89b791",
+                     "failed":false,
+                     "volumeDataDensity":-0.023638606,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/yAkoKWgQXe",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":222111125772,
+                     "reserved":5119328068,
+                     "uuid":"dfc0c7aa-576c-4a20-81eb-65d061510160",
+                     "failed":false,
+                     "volumeDataDensity":3.9339066E-4,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"aa2dad53-5496-4a33-8c10-a1b6fbac248c",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/fuqbWDjV78",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4172490338,
+                     "reserved":9729408276,
+                     "uuid":"3d861e10-5e0b-4ee7-85ea-510f83b36f27",
+                     "failed":false,
+                     "volumeDataDensity":0.36708134,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/4kKs3Oy6au",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3148459442506,
+                     "reserved":1004145049688,
+                     "uuid":"14cd742f-10dc-417f-90b8-5bade257e663",
+                     "failed":false,
+                     "volumeDataDensity":0.01954192,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/a0nOAhp0HP",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":431791493109,
+                     "reserved":13505225027,
+                     "uuid":"deb624b2-7d94-4784-8a8b-cb06fc4d166d",
+                     "failed":false,
+                     "volumeDataDensity":-0.32292056,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"dfab7a7b-28f3-450a-bf4b-5516c0f392fe",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/eD6NhDRpHN",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":198409397569,
+                     "reserved":35217057236,
+                     "uuid":"7be3deb1-530e-4d74-a006-2779bcd24c63",
+                     "failed":false,
+                     "volumeDataDensity":-0.355349,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/0PpCLUjJUQ",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":2850181336999,
+                     "reserved":185442348941,
+                     "uuid":"f13cf5d4-ecc6-4657-bf46-ad8a4a5768cc",
+                     "failed":false,
+                     "volumeDataDensity":0.02925235,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YwBqdPJ7hh",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":432261135257,
+                     "reserved":44231584303,
+                     "uuid":"27ad3fd1-84a1-4641-b5ac-5b18029e91e1",
+                     "failed":false,
+                     "volumeDataDensity":-0.1779697,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"6df74488-ec21-4b6d-b023-394e0405fc26",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"2f3a0401-378f-48d2-8665-4b2dd6d9e259",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7692424,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/6C2oWdxxvb",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":3815874591397,
+                     "reserved":575843635069,
+                     "uuid":"1f0c9ceb-3e40-4cbf-a722-e2fa6a55246b",
+                     "failed":false,
+                     "volumeDataDensity":-0.60439783,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ahwQsjVcf1",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":693308313656,
+                     "reserved":105816724218,
+                     "uuid":"624c38d3-20cf-430b-980d-8aa78b98f30f",
+                     "failed":false,
+                     "volumeDataDensity":0.15754682,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/9PhfTCJkZD",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":160057007542,
+                     "reserved":1228297101824,
+                     "uuid":"14cafab2-565d-43ed-aa80-cebe28928a27",
+                     "failed":false,
+                     "volumeDataDensity":0.23447491,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"32516392-1c11-4f64-a110-8b11e88c875a",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Iz0bW4jl2w",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3199964494719,
+                     "reserved":474303236589,
+                     "uuid":"75b1c501-b62f-4a56-8afb-6809aec033a8",
+                     "failed":false,
+                     "volumeDataDensity":0.02165535,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/s4wnzP34md",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3318842511538,
+                     "reserved":1165658452759,
+                     "uuid":"67f7c141-8979-4c1a-a257-951c332a0506",
+                     "failed":false,
+                     "volumeDataDensity":-0.03875214,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/s1Z2XtyNFa",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":96216697539,
+                     "reserved":56707240610,
+                     "uuid":"184e96bb-0fcd-41a9-99fb-72de49b3ffcc",
+                     "failed":false,
+                     "volumeDataDensity":0.22981045,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"577955b1-efae-4f83-9103-73485d461df1",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/hFwrcbPuXX",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":429317594868,
+                     "reserved":59814283159,
+                     "uuid":"20a7f50a-510d-480c-b1f0-26dbea40edfe",
+                     "failed":false,
+                     "volumeDataDensity":-0.43321043,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/lYwPycl13y",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":306193668903,
+                     "reserved":49189837392,
+                     "uuid":"4e26cbb1-b5ac-419a-ba07-ce6cf7b17f79",
+                     "failed":false,
+                     "volumeDataDensity":0.0016639531,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/guWTLDWh5w",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":1529285563897,
+                     "reserved":1126843041453,
+                     "uuid":"c070a6e1-8998-445f-8e80-be98bb5e8458",
+                     "failed":false,
+                     "volumeDataDensity":0.047730535,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"a3e6594d-a82b-468a-9dc6-61d150017703",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"748097d8-7889-41b0-83a5-289a78a88650",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.1701257,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/WcqFaYcKfp",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":210301266504,
+                     "reserved":34583292724,
+                     "uuid":"8f92e2eb-67be-47f3-bab8-3f33e0e2c5dd",
+                     "failed":false,
+                     "volumeDataDensity":0.12676129,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/deqgrgNNiw",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":1420001641076,
+                     "reserved":508136513703,
+                     "uuid":"526b508a-bced-43f7-873a-8a4fd0a0cb78",
+                     "failed":false,
+                     "volumeDataDensity":-0.0711537,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xy3IRPckmz",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":3780526756982,
+                     "reserved":207445968321,
+                     "uuid":"c9a7c333-398d-4673-9b40-9d72e4d3725a",
+                     "failed":false,
+                     "volumeDataDensity":0.013555586,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"5a3e75b7-ad9f-4c58-9b47-c22631fd5fdd",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Aj8IyTcGpz",
+                     "capacity":2000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":984118483938,
+                     "reserved":197323850092,
+                     "uuid":"a5b639a2-2876-4e2b-92d1-20fe62fc6071",
+                     "failed":false,
+                     "volumeDataDensity":0.1191923,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/WvDiWrLFDL",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":453950453440,
+                     "reserved":133039277436,
+                     "uuid":"2f4670c7-dbae-4d34-8b9f-67f6b209deec",
+                     "failed":false,
+                     "volumeDataDensity":0.14150178,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/hwSEd8S4Vw",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4345290661051,
+                     "reserved":974334403734,
+                     "uuid":"dc11d8d6-0602-4168-b258-2cadbab4afbb",
+                     "failed":false,
+                     "volumeDataDensity":-0.05601728,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"04bd8d4e-2a81-4639-9741-0a65060eb1ed",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/eeoQ42YKDv",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":290572789796,
+                     "reserved":52683150406,
+                     "uuid":"4874f0ec-2bd3-4000-832a-80e633dfbf28",
+                     "failed":false,
+                     "volumeDataDensity":-0.13371289,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/7DmqZkrf8S",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":5608341290461,
+                     "reserved":1089427475981,
+                     "uuid":"98cfcdf7-81f7-4142-8bdc-15f380343cf5",
+                     "failed":false,
+                     "volumeDataDensity":-0.0060590506,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/sLckafGncK",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":37723722613,
+                     "reserved":12073801018,
+                     "uuid":"c6886667-7ab6-4acd-813b-f74da79f1530",
+                     "failed":false,
+                     "volumeDataDensity":0.5021719,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"7b970bb5-0b33-41cb-b75e-8a843fe660dd",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"3eb15f52-bbe5-4d79-9336-2b881c1f37f5",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.0164268,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/IwJmrS2tBI",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":599037876739,
+                     "reserved":872170058,
+                     "uuid":"a1d307e8-454a-47a9-bb0f-5a888369d414",
+                     "failed":false,
+                     "volumeDataDensity":-0.27239504,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/JKNOhjsZtX",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":313112469916,
+                     "reserved":149478817748,
+                     "uuid":"30d4291f-a09b-4e40-8218-d3658cb4340b",
+                     "failed":false,
+                     "volumeDataDensity":-0.08747724,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/AV2jT0UOXI",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":2242077113096,
+                     "reserved":540911071005,
+                     "uuid":"5f4262b2-57cd-497e-af87-1b8d6bd4dcc6",
+                     "failed":false,
+                     "volumeDataDensity":0.04672855,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"a713ae1f-3e73-4640-8918-07c8b3cc51cb",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/v3aMkOw3dz",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":272152568337,
+                     "reserved":53765463773,
+                     "uuid":"ff35f13a-03e7-46bc-8baf-a9c818449f88",
+                     "failed":false,
+                     "volumeDataDensity":-0.38344312,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/e5BvdjCDiw",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":157609365479,
+                     "reserved":513722430414,
+                     "uuid":"ed76a75a-7f2d-4aa3-8c0a-19a8d0e78793",
+                     "failed":false,
+                     "volumeDataDensity":0.07965936,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mO9D6sN60I",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":203030027335,
+                     "reserved":119945349028,
+                     "uuid":"cfe5f02a-2668-4c6b-8fc2-0519b2d20fa4",
+                     "failed":false,
+                     "volumeDataDensity":-0.30814028,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"9576c722-2e1b-459e-82b9-013d3c063a3a",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/CR2VXpHcWc",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":23845273688,
+                     "reserved":199921586862,
+                     "uuid":"b14c0683-f6c4-4e71-a0ba-5e066dc970f8",
+                     "failed":false,
+                     "volumeDataDensity":0.19425383,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Yv5D2gvjsa",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":156544994403,
+                     "reserved":78080351171,
+                     "uuid":"180a538e-344c-4854-ac42-0f353ee30605",
+                     "failed":false,
+                     "volumeDataDensity":-0.14697284,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/NTD2Gw7tdg",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":135487369815,
+                     "reserved":12191936613,
+                     "uuid":"05beec6d-d12d-48ea-bd61-4d48689b05d4",
+                     "failed":false,
+                     "volumeDataDensity":-0.4973566,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"d9ce479a-eaeb-4639-9aef-1666aca419b0",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"4997b34f-9394-4494-b1ff-4a739e77e65c",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.8723485,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/D72RLVGNVP",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":424210334934,
+                     "reserved":291470719761,
+                     "uuid":"016f9e3d-032e-463f-8ba0-27e71814f412",
+                     "failed":false,
+                     "volumeDataDensity":0.37841192,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/stc3UQuT8c",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":3436202648422,
+                     "reserved":658902800476,
+                     "uuid":"32ea02c3-50c1-49a5-b14e-78f20dd94494",
+                     "failed":false,
+                     "volumeDataDensity":0.06695455,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/dGubZGza1G",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":3371515851630,
+                     "reserved":532813043432,
+                     "uuid":"8c03c134-e83a-4d3c-9962-51755fed67b4",
+                     "failed":false,
+                     "volumeDataDensity":-0.43737453,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"0f3e605a-afd7-423a-8bd7-fac6191ddc26",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/AF4kDUQS5u",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":143107439678,
+                     "reserved":91963787273,
+                     "uuid":"bd298a00-4046-4bea-bf07-ceb4c779d216",
+                     "failed":false,
+                     "volumeDataDensity":0.23739417,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Pbwi5NC8gO",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4207863955162,
+                     "reserved":110702623811,
+                     "uuid":"e73e266a-c796-4576-b974-944bfdf6fb35",
+                     "failed":false,
+                     "volumeDataDensity":-0.24173915,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/5mUvnf9C6c",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1008520163847,
+                     "reserved":160593915570,
+                     "uuid":"957093bb-ef79-4ef4-b775-97e294861944",
+                     "failed":false,
+                     "volumeDataDensity":0.26435673,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"1e0bb45a-777b-4569-8f39-c51d4418f417",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/YzMdyyGz5m",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":642410701301,
+                     "reserved":45847803260,
+                     "uuid":"1500238c-9490-4819-aed2-f989cac2bb3b",
+                     "failed":false,
+                     "volumeDataDensity":-0.008626759,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ctT6ZaHbkr",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":287314038615,
+                     "reserved":3783971799,
+                     "uuid":"fd4e4e21-59af-4a01-a1d1-e8ee9daa10f1",
+                     "failed":false,
+                     "volumeDataDensity":-0.22647107,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/kKw6Caqa4I",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":4948772342950,
+                     "reserved":243599931936,
+                     "uuid":"f629cb3f-136f-4d88-a9a1-44b83180cb52",
+                     "failed":false,
+                     "volumeDataDensity":0.0110195875,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"1281a631-af25-461a-b323-3ac98047008f",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"c100991c-508f-4e9f-b502-a1cb39621d19",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.0572937,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/HNSC3Gf0Ao",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":2858006204593,
+                     "reserved":1210791440317,
+                     "uuid":"e2d23a14-b5d7-4810-9e59-54b8a2788d56",
+                     "failed":false,
+                     "volumeDataDensity":0.113719165,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/lSu0CmBBRj",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":2345454452439,
+                     "reserved":387242302317,
+                     "uuid":"bbf913a3-be52-469e-b9a2-ffa22ca213aa",
+                     "failed":false,
+                     "volumeDataDensity":-0.29029566,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/p76dH4BDn9",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":103229722538,
+                     "reserved":65197172937,
+                     "uuid":"5ff934f5-58db-45ca-96b8-76559a0cbd4f",
+                     "failed":false,
+                     "volumeDataDensity":0.29906747,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"2c7b4de4-f9c8-4e04-8721-fdbdf9d0e0dd",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/hOduouQR7h",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":66742824225,
+                     "reserved":11538412874,
+                     "uuid":"f127a284-745c-433d-9cd0-a5c54c4ced36",
+                     "failed":false,
+                     "volumeDataDensity":-0.16812682,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/OEqOIHVWDG",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":68492668182,
+                     "reserved":233758305,
+                     "uuid":"dfb517a8-2edc-4057-a14e-df6ec68a2c9b",
+                     "failed":false,
+                     "volumeDataDensity":-0.27961582,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/6z5Tli7Kk3",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":182433481421,
+                     "reserved":465654790545,
+                     "uuid":"c885f9f9-6544-45b1-b131-f4de9896ac0b",
+                     "failed":false,
+                     "volumeDataDensity":0.023014553,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"789e7f9b-bc1b-43df-825a-6888318f6d90",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/9dteQlCgww",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":206601275057,
+                     "reserved":34493106659,
+                     "uuid":"bb6a699a-3a75-44d1-9d01-1e87fde19fc7",
+                     "failed":false,
+                     "volumeDataDensity":0.100215495,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YQaBsJO0Rf",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":3805886528377,
+                     "reserved":848501973219,
+                     "uuid":"26c9cf2d-cd1e-49e9-a910-495fad84ebec",
+                     "failed":false,
+                     "volumeDataDensity":-0.038395703,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/mrhD219vO3",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":23802623920,
+                     "reserved":21718534496,
+                     "uuid":"934f0b1d-14a6-471d-ae0c-56dd50eda928",
+                     "failed":false,
+                     "volumeDataDensity":0.744843,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"0de213b4-8dff-4beb-806d-62bc917dd0ea",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"56802bcd-edf1-4e75-ab2d-bfdcd6896995",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.702493,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/B2cwd5YgvZ",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":598437199027,
+                     "reserved":122613020551,
+                     "uuid":"5899d3d4-912c-4df9-962a-f3a32bcee204",
+                     "failed":false,
+                     "volumeDataDensity":-0.14198738,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tN0DLET3OT",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":554967080236,
+                     "reserved":136255591559,
+                     "uuid":"7c5e1c5a-d2d2-4fbd-bc58-f4b39cda89bb",
+                     "failed":false,
+                     "volumeDataDensity":-0.44435018,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/elxl3k2kJd",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":3523216462223,
+                     "reserved":782008531656,
+                     "uuid":"96cadc33-ead1-4b0e-8f1e-b7c3f567fd39",
+                     "failed":false,
+                     "volumeDataDensity":0.051964283,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"3e94a6b6-0db8-4ecb-94e5-3a78e18b8056",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/wSOKL03H0m",
+                     "capacity":2000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1355280228782,
+                     "reserved":333443355962,
+                     "uuid":"22f288dd-b235-4b28-86c2-9b4e211a986d",
+                     "failed":false,
+                     "volumeDataDensity":-0.41805956,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/vNjoi1f5zg",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2318305520667,
+                     "reserved":325409801366,
+                     "uuid":"37ed90b7-9ea0-437c-aad3-2bc48be41d79",
+                     "failed":false,
+                     "volumeDataDensity":0.09308681,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/FZzFLKEPzW",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":93734723510,
+                     "reserved":7543975081,
+                     "uuid":"9b0c64e1-e54c-4b5b-9e95-9604c66def01",
+                     "failed":false,
+                     "volumeDataDensity":-0.09188256,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"dfcd11ce-8c1f-456b-b56c-eff11dab94a7",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DulW2SYW8m",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":220375240580,
+                     "reserved":102318228113,
+                     "uuid":"297f8186-3bf7-4d7b-8786-cba0911a6f91",
+                     "failed":false,
+                     "volumeDataDensity":0.24164852,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/inJzBTNuQX",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":600187138789,
+                     "reserved":31830189350,
+                     "uuid":"2d57253f-9b63-4ca2-a640-38329363f791",
+                     "failed":false,
+                     "volumeDataDensity":-0.21380335,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/H0JJocfwJO",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":2685548159271,
+                     "reserved":43344380632,
+                     "uuid":"7a9c9bc2-a088-4c80-b80d-993005a46b4e",
+                     "failed":false,
+                     "volumeDataDensity":0.005710125,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"6350d129-ac1c-4081-98d4-99d183e79e82",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"5bac8cc5-4292-4344-964d-2784d549047d",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.990529,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/g0BEspGA0y",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":212443362958,
+                     "reserved":32373359674,
+                     "uuid":"4a6386e2-0dde-4e15-a579-b988285aaa7c",
+                     "failed":false,
+                     "volumeDataDensity":0.28445172,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/SLK0D2Zc5B",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":68102246402,
+                     "reserved":163521219734,
+                     "uuid":"37a6215e-d42f-4b11-90a7-bb3e89ccf8cc",
+                     "failed":false,
+                     "volumeDataDensity":0.4478921,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/2NQ4ivWbhb",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":2068251002974,
+                     "reserved":266615774003,
+                     "uuid":"931426c8-6021-4ace-afd9-4214879a3a01",
+                     "failed":false,
+                     "volumeDataDensity":-0.22735566,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"31430c0f-a904-4ee5-a1e2-dc51da061f43",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/xDdGTI2zDX",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1592828158024,
+                     "reserved":321331291184,
+                     "uuid":"6856390e-3086-4f3a-b5e3-24dfc915bbc0",
+                     "failed":false,
+                     "volumeDataDensity":-0.033958018,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/VxdOPMEaVm",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":662197420238,
+                     "reserved":129327924848,
+                     "uuid":"46a2d61f-869e-4d14-86ef-bdde1ae5c23c",
+                     "failed":false,
+                     "volumeDataDensity":-0.29857045,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/guRYHwfEd9",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1970200736246,
+                     "reserved":913393868912,
+                     "uuid":"b9286604-02a0-4bda-b8b9-8e2ae24a194a",
+                     "failed":false,
+                     "volumeDataDensity":0.078564465,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"9cad2fb6-3d81-4d90-bcd0-0026e4ef4f0d",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/EFNi68A7l8",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":3731412303786,
+                     "reserved":1192263275229,
+                     "uuid":"da89a996-e4e6-46b8-81a6-5f60d4ae1d76",
+                     "failed":false,
+                     "volumeDataDensity":-0.31974775,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/vBh4ieulaP",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":966530651703,
+                     "reserved":512985605369,
+                     "uuid":"bcda479a-7d0e-4d7a-a92e-b2f0cfc1c628",
+                     "failed":false,
+                     "volumeDataDensity":0.107335985,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/P6yDrH2sIP",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":928737699867,
+                     "reserved":860769791705,
+                     "uuid":"399af9c0-6879-4afe-a2fc-1e7c1737d898",
+                     "failed":false,
+                     "volumeDataDensity":0.19265282,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"3efef5a3-40c8-4c08-9fde-75bf71d27e97",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"09bd23ae-efc6-4123-9bea-8aaaf62b91fd",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.9702007,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/9ek6JB6iV0",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":97347552551,
+                     "reserved":18312678619,
+                     "uuid":"5ea71594-b28b-4923-92fe-2af639a6004d",
+                     "failed":false,
+                     "volumeDataDensity":0.14713643,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/0FRtKUYsFf",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":583454679043,
+                     "reserved":72278509143,
+                     "uuid":"bde36f80-9b5d-4a87-be75-c174a7cc16ac",
+                     "failed":false,
+                     "volumeDataDensity":-0.452522,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8wzr5xmQJU",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":1741002963273,
+                     "reserved":274776495260,
+                     "uuid":"25b9f982-6621-421b-be2a-9806f41d4fee",
+                     "failed":false,
+                     "volumeDataDensity":0.04513997,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"c17a8841-395b-4982-b620-8b4989b74bdd",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/vhEiAkyYGF",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":530027947601,
+                     "reserved":153859230596,
+                     "uuid":"60df4e34-aeed-434f-9e05-82ee7684182d",
+                     "failed":false,
+                     "volumeDataDensity":-0.11932409,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/IKOewYVTky",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":73026741946,
+                     "reserved":9265317970,
+                     "uuid":"d306d733-7552-439a-bd34-c857cdb57622",
+                     "failed":false,
+                     "volumeDataDensity":-0.10386443,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Nmp7GgZMQN",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":361468044436,
+                     "reserved":60900139506,
+                     "uuid":"fb1ab26b-7af2-4f0c-ba89-8c5b53a1738c",
+                     "failed":false,
+                     "volumeDataDensity":0.1353845,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"990e3dce-4977-4422-91c9-aa644f7071b8",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/TkhjLaBbb7",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":110103097846,
+                     "reserved":16360070716,
+                     "uuid":"01ea91f9-d857-4a29-8a02-9e4df241f474",
+                     "failed":false,
+                     "volumeDataDensity":0.6195963,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Vxy1mrscfH",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":79346639490,
+                     "reserved":11878325025,
+                     "uuid":"f83c4c8e-f9df-49cd-af9e-dbc59e832ea0",
+                     "failed":false,
+                     "volumeDataDensity":-0.1562233,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UMD0rMmKDp",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":2613226915266,
+                     "reserved":205725695026,
+                     "uuid":"efef9c33-380b-4e84-b75c-9318deb8644c",
+                     "failed":false,
+                     "volumeDataDensity":-0.19100964,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"db34d61d-983d-4674-9dc5-f16f13e06d41",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"d5eda952-3bf7-4805-b186-5f64bb6673da",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.5863423,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/1ie3gtFPft",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":214822127017,
+                     "reserved":51372388061,
+                     "uuid":"0ed9362b-f949-4e56-9f10-befeeebaedab",
+                     "failed":false,
+                     "volumeDataDensity":-0.46936452,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YSNz7Fs6Bp",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":2256502112159,
+                     "reserved":1314846117719,
+                     "uuid":"85302be0-47da-4fc7-a3a6-5777ca9d16af",
+                     "failed":false,
+                     "volumeDataDensity":0.057127863,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tvgkteIzYE",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":462102070594,
+                     "reserved":101123092999,
+                     "uuid":"e5ff625c-5528-4293-b770-a365ad5dfc90",
+                     "failed":false,
+                     "volumeDataDensity":-0.5316176,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"57806ecf-4e5a-409f-baf3-1cbbf8febab8",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/RF3hwipQah",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1918091144429,
+                     "reserved":245336800062,
+                     "uuid":"ff3e9768-a485-434a-bc23-3b8f47da8f17",
+                     "failed":false,
+                     "volumeDataDensity":-0.025074095,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/39YDHDTwcY",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":9475960500,
+                     "reserved":167440542465,
+                     "uuid":"92e2d418-9cf3-442c-ae87-54770fad91cc",
+                     "failed":false,
+                     "volumeDataDensity":0.29530123,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/hfBQWsQxHS",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":154184588586,
+                     "reserved":33477693271,
+                     "uuid":"9722a7af-0222-4b34-bc4d-89800ee48c25",
+                     "failed":false,
+                     "volumeDataDensity":-0.27026868,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"ecef5a9d-7b48-4043-9709-c70629d50b92",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/c57WEVnzzN",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":3113819590397,
+                     "reserved":189532031981,
+                     "uuid":"df5f4d36-0905-4f03-a10c-61eecccbb685",
+                     "failed":false,
+                     "volumeDataDensity":-0.32282406,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/RQlZdakT5a",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":502699823481,
+                     "reserved":241709835717,
+                     "uuid":"501322e9-e143-4a1e-8d33-195cb693be17",
+                     "failed":false,
+                     "volumeDataDensity":0.25009397,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/9rYVfkYpp9",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":259415221892,
+                     "reserved":23570479113,
+                     "uuid":"5d99f7bb-23d9-4eca-86b8-93db5c251432",
+                     "failed":false,
+                     "volumeDataDensity":-0.36467022,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"a40bab02-a8f6-43f5-bc7d-cf393a0493f5",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"ae209781-734a-42c9-84a5-2e83e80ddb78",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.6381848,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/VuDD8dbFYq",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":6890645602244,
+                     "reserved":309053464433,
+                     "uuid":"66f4b2fb-f900-4fcd-b40c-be7d1d674ba4",
+                     "failed":false,
+                     "volumeDataDensity":-0.20853692,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/GbN93CY2QD",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":247645896656,
+                     "reserved":8301114984,
+                     "uuid":"a34f0b1d-0569-484c-87ee-3a32a532689b",
+                     "failed":false,
+                     "volumeDataDensity":0.05517012,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YTtqFiU333",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":928295414232,
+                     "reserved":347814980586,
+                     "uuid":"c4722f44-d8fd-44de-aa7d-428fe6930ae6",
+                     "failed":false,
+                     "volumeDataDensity":0.43323016,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"78074977-5afe-475c-b363-15d61f766e9c",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/AxZwQBhUC8",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":354469900161,
+                     "reserved":70845065470,
+                     "uuid":"7d84393e-52b0-46c4-b1da-74dc144c69c2",
+                     "failed":false,
+                     "volumeDataDensity":-0.30316573,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/X6DJZuDrMg",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":49252815072,
+                     "reserved":16883799354,
+                     "uuid":"b359da24-f67f-438d-9a3b-b0a6a552a186",
+                     "failed":false,
+                     "volumeDataDensity":-0.06977159,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/7wFhSIfdWD",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1359332927472,
+                     "reserved":139977947907,
+                     "uuid":"ddd77755-aa10-4239-9ae6-0588bc066158",
+                     "failed":false,
+                     "volumeDataDensity":0.04751858,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"0ef0e995-7ab0-47a7-8746-d2a72f1870a9",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/M9U7VnzhR0",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":2997924103561,
+                     "reserved":237642448598,
+                     "uuid":"8ce24a12-c09f-4206-84b2-f801bd4ea779",
+                     "failed":false,
+                     "volumeDataDensity":-0.22215062,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/P2xJGfd6ha",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":1173456135095,
+                     "reserved":425663787231,
+                     "uuid":"8b28983c-8f09-48ce-b123-cfc5295b99e9",
+                     "failed":false,
+                     "volumeDataDensity":0.19684312,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/PKfI6477qk",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":196607763282,
+                     "reserved":13852001573,
+                     "uuid":"e23d8c9f-5304-4962-a218-39e6c9e0d214",
+                     "failed":false,
+                     "volumeDataDensity":-0.10179776,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"2382d7a0-3e70-4439-8892

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/49] hadoop git commit: HDFS-10547. DiskBalancer: fix whitespace issue in doc files. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10547. DiskBalancer: fix whitespace issue in doc files. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c6ed5480
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c6ed5480
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c6ed5480

Branch: refs/heads/trunk
Commit: c6ed54808d086fffbc1cdf25f602c78798de789a
Parents: 0774412
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Jun 20 09:48:34 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:01 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/HDFSCommands.md           |  2 +-
 .../src/site/markdown/HDFSDiskbalancer.md       | 25 ++++++++++++--------
 2 files changed, 16 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6ed5480/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index f868118..39e8991 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -436,7 +436,7 @@ Runs a HDFS dfsadmin client.
 Usage:
 
        hdfs diskbalancer
-         [-plan <datanode> -uri <namenodeURI>]
+         [-plan <datanode> -fs <namenodeURI>]
          [-execute <planfile>]
          [-query <datanode>]
          [-cancel <planfile>]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6ed5480/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
index 388a4c6..522dc5a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
@@ -64,11 +64,11 @@ The following sections discusses what commands are supported by disk balancer
 
 | COMMAND\_OPTION    | Description |
 |:---- |:---- |
-| `-out`|	Allows user to control the output location of the plan file.|
-| `-bandwidth`|	Since datanode is operational and might be running other jobs, diskbalancer limits the amount of data moved per second. This parameter allows user to set the maximum bandwidth to be used. This is not required to be set since diskBalancer will use the deafult bandwidth if this is not specified.|
-| `-thresholdPercentage`|	Since we operate against a snap-shot of datanode, themove operations have a tolerance percentage to declare success. If user specifies 10% and move operation is say 20GB in size, if we can move 18GB that operation is considered successful. This is to accomodate the changes in datanode in real time. This parameter is not needed and a default is used if not specified.|
-| `-maxerror` |	Max error allows users to specify how many block copy operations must fail before we abort a move step. Once again, this is not a needed parameter and a system-default is used if not specified.|
-| `-v`|	Verbose mode, specifying this parameter forces the plan command to print out a summary of the plan on stdout.|
+| `-out`| Allows user to control the output location of the plan file.|
+| `-bandwidth`| Since datanode is operational and might be running other jobs, diskbalancer limits the amount of data moved per second. This parameter allows user to set the maximum bandwidth to be used. This is not required to be set since diskBalancer will use the deafult bandwidth if this is not specified.|
+| `-thresholdPercentage`| Since we operate against a snap-shot of datanode, themove operations have a tolerance percentage to declare success. If user specifies 10% and move operation is say 20GB in size, if we can move 18GB that operation is considered successful. This is to accomodate the changes in datanode in real time. This parameter is not needed and a default is used if not specified.|
+| `-maxerror` | Max error allows users to specify how many block copy operations must fail before we abort a move step. Once again, this is not a needed parameter and a system-default is used if not specified.|
+| `-v`| Verbose mode, specifying this parameter forces the plan command to print out a summary of the plan on stdout.|
 
 The plan command writes two output files. They are `<nodename>.before.json` which
 captures the state of the datanode before the diskbalancer is run, and `<nodename>.plan.json`.
@@ -89,7 +89,7 @@ Query command gets the current status of the diskbalancer from a datanode.
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-|`-v` |	Verbose mode, Prints out status of individual moves|
+|`-v` | Verbose mode, Prints out status of individual moves|
 
 
 ### Cancel
@@ -103,6 +103,11 @@ or
 
 Plan ID can be read from datanode using query command.
 
+### Report
+Report command provides detailed report about a node.
+
+`hdfs diskbalancer -fs http://namenode.uri -report -node {DataNodeID | IP | Hostname}`
+
 
 Settings
 --------
@@ -111,7 +116,7 @@ There is a set of diskbalancer settings that can be controlled via hdfs-site.xml
 
 | Setting | Description |
 |:---- |:---- |
-|`dfs.disk.balancer.enabled`|	This parameter controls if diskbalancer is enabled for a cluster. if this is not enabled, any execute command will be rejected by the datanode.The default value is false.|
-|`dfs.disk.balancer.max.disk.throughputInMBperSec` |	This controls the maximum disk bandwidth consumed by diskbalancer while copying data. If a value like 10MB is specified then diskbalancer on the average will only copy 10MB/S. The default value is 10MB/S.|
-|`dfs.disk.balancer.max.disk.errors`|	sets the value of maximum number of errors we can ignore for a specific move between two disks before it is abandoned. For example, if a plan has 3 pair of disks to copy between , and the first disk set encounters more than 5 errors, then we abandon the first copy and start the second copy in the plan. The default value of max errors is set to 5.|
-|`dfs.disk.balancer.block.tolerance.percent`|	The tolerance percent sepcifies when we have reached a good enough value for any copy step. For example, if you specify 10% then getting close to 10% of the target value is good enough.|
+|`dfs.disk.balancer.enabled`| This parameter controls if diskbalancer is enabled for a cluster. if this is not enabled, any execute command will be rejected by the datanode.The default value is false.|
+|`dfs.disk.balancer.max.disk.throughputInMBperSec` | This controls the maximum disk bandwidth consumed by diskbalancer while copying data. If a value like 10MB is specified then diskbalancer on the average will only copy 10MB/S. The default value is 10MB/S.|
+|`dfs.disk.balancer.max.disk.errors`| sets the value of maximum number of errors we can ignore for a specific move between two disks before it is abandoned. For example, if a plan has 3 pair of disks to copy between , and the first disk set encounters more than 5 errors, then we abandon the first copy and start the second copy in the plan. The default value of max errors is set to 5.|
+|`dfs.disk.balancer.block.tolerance.percent`| The tolerance percent sepcifies when we have reached a good enough value for any copy step. For example, if you specify 10% then getting close to 10% of the target value is good enough.|


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/49] hadoop git commit: HDFS-9547. DiskBalancer: Add user documentation. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9547. DiskBalancer: Add user documentation. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/06a9799d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/06a9799d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/06a9799d

Branch: refs/heads/trunk
Commit: 06a9799d84bef013e1573d382f824b485aa0c329
Parents: 43eee50
Author: Anu Engineer <ae...@apache.org>
Authored: Thu May 26 10:23:08 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/HDFSCommands.md           |  22 ++++
 .../src/site/markdown/HDFSDiskbalancer.md       | 117 +++++++++++++++++++
 2 files changed, 139 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06a9799d/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 5bb1a87..f868118 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -37,6 +37,7 @@ HDFS Commands Guide
     * [crypto](#crypto)
     * [datanode](#datanode)
     * [dfsadmin](#dfsadmin)
+    * [diskbalancer](#diskbalancer)
     * [erasurecode](#erasurecode)
     * [haadmin](#haadmin)
     * [journalnode](#journalnode)
@@ -430,6 +431,27 @@ Usage:
 
 Runs a HDFS dfsadmin client.
 
+### `diskbalancer`
+
+Usage:
+
+       hdfs diskbalancer
+         [-plan <datanode> -uri <namenodeURI>]
+         [-execute <planfile>]
+         [-query <datanode>]
+         [-cancel <planfile>]
+         [-cancel <planID> -node <datanode>]
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+|-plan| Creates a disbalancer plan|
+|-execute| Executes a given plan on a datanode|
+|-query| Gets the current diskbalancer status from a datanode|
+|-cancel| Cancels a running plan|
+
+
+Runs the diskbalancer CLI. See [HDFS Diskbalancer](./HDFSDiskbalancer.html) for more information on this command.
+
 ### `erasurecode`
 
 Usage:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06a9799d/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
new file mode 100644
index 0000000..388a4c6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
@@ -0,0 +1,117 @@
+<!---
+  Licensed 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
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+
+HDFS Disk Balancer
+===================
+
+* [Overview](#Overview)
+* [Architecture](#Architecture)
+* [Commands](#Commands)
+* [Settings](#Settings)
+
+
+Overview
+--------
+
+Diskbalancer is a command line tool that distributes data evenly on all disks of a datanode.
+This tool is different from  [Balancer](./HdfsUserGuide.html#Balancer)  which
+takes care of cluster-wide data balancing. Data can have uneven spread between
+disks on a node due to several reasons. This can happen due to large amount of
+writes and deletes or due to a disk replacement.This tool operates against a given datanode and moves blocks from one disk to another.
+
+
+
+Architecture
+------------
+
+Disk Balancer operates by creating a plan and goes on to execute that plan on the datanode.
+A plan is a set of statements that describe how much data should move between two disks.
+A plan is composed of multiple move steps. A move step has source disk, destination
+disk and number of bytes to move.A plan can be executed against an operational data node. Disk balancer should not
+interfere with other processes since it throttles how much data is copied
+every second. Please note that disk balancer is not enabled by default on a cluster.
+To enable diskbalancer `dfs.disk.balancer.enabled` must be set to `true` in hdfs-site.xml.
+
+
+Commands
+--------
+The following sections discusses what commands are supported by disk balancer
+ and how to use them.
+
+### Plan
+
+ The plan command can be run against a given datanode by running
+
+ `hdfs diskbalancer -uri hdfs://mycluster.com -plan node1.mycluster.com`
+
+ uri is the address of the namenode and -plan points to the datanode that we
+ need to plan for. By deafult, plan command writes its output to
+ **/system/diskbalancer**.
+
+ The plan command also has a set of parameters that allows user to control
+ the output and execution of the plan.
+
+| COMMAND\_OPTION    | Description |
+|:---- |:---- |
+| `-out`|	Allows user to control the output location of the plan file.|
+| `-bandwidth`|	Since datanode is operational and might be running other jobs, diskbalancer limits the amount of data moved per second. This parameter allows user to set the maximum bandwidth to be used. This is not required to be set since diskBalancer will use the deafult bandwidth if this is not specified.|
+| `-thresholdPercentage`|	Since we operate against a snap-shot of datanode, themove operations have a tolerance percentage to declare success. If user specifies 10% and move operation is say 20GB in size, if we can move 18GB that operation is considered successful. This is to accomodate the changes in datanode in real time. This parameter is not needed and a default is used if not specified.|
+| `-maxerror` |	Max error allows users to specify how many block copy operations must fail before we abort a move step. Once again, this is not a needed parameter and a system-default is used if not specified.|
+| `-v`|	Verbose mode, specifying this parameter forces the plan command to print out a summary of the plan on stdout.|
+
+The plan command writes two output files. They are `<nodename>.before.json` which
+captures the state of the datanode before the diskbalancer is run, and `<nodename>.plan.json`.
+
+### Execute
+
+Execute command takes a plan command executes it against the datanode that plan was generated against.
+
+`hdfs diskbalancer -execute /system/diskbalancer/nodename.plan.json`
+
+This executes the plan by reading datanode\u2019s address from the plan file.
+
+### Query
+
+Query command gets the current status of the diskbalancer from a datanode.
+
+`hdfs diskbalancer -query nodename.mycluster.com`
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+|`-v` |	Verbose mode, Prints out status of individual moves|
+
+
+### Cancel
+Cancel command cancels a running plan. Restarting datanode has the same effect as cancel command since plan information on the datanode is transient.
+
+`hdfs diskbalancer -cancel /system/diskbalancer/nodename.plan.json`
+
+or
+
+`hdfs diskbalancer -cancel planID -node nodename`
+
+Plan ID can be read from datanode using query command.
+
+
+Settings
+--------
+
+There is a set of diskbalancer settings that can be controlled via hdfs-site.xml
+
+| Setting | Description |
+|:---- |:---- |
+|`dfs.disk.balancer.enabled`|	This parameter controls if diskbalancer is enabled for a cluster. if this is not enabled, any execute command will be rejected by the datanode.The default value is false.|
+|`dfs.disk.balancer.max.disk.throughputInMBperSec` |	This controls the maximum disk bandwidth consumed by diskbalancer while copying data. If a value like 10MB is specified then diskbalancer on the average will only copy 10MB/S. The default value is 10MB/S.|
+|`dfs.disk.balancer.max.disk.errors`|	sets the value of maximum number of errors we can ignore for a specific move between two disks before it is abandoned. For example, if a plan has 3 pair of disks to copy between , and the first disk set encounters more than 5 errors, then we abandon the first copy and start the second copy in the plan. The default value of max errors is set to 5.|
+|`dfs.disk.balancer.block.tolerance.percent`|	The tolerance percent sepcifies when we have reached a good enough value for any copy step. For example, if you specify 10% then getting close to 10% of the target value is good enough.|


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/49] hadoop git commit: Fix a build break in HDFS-1312

Posted by ar...@apache.org.
Fix a build break in HDFS-1312


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9be97037
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9be97037
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9be97037

Branch: refs/heads/trunk
Commit: 9be9703716d2787cd6ee0ebbbe44a18b1f039018
Parents: ec60167
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Mar 17 16:14:48 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/diskbalancer/DiskBalancerException.java   | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9be97037/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
index c3571c9..a55bcf3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -33,7 +33,9 @@ public class DiskBalancerException extends IOException {
     PLAN_ALREADY_IN_PROGRESS,
     INVALID_VOLUME,
     INVALID_MOVE,
-    INTERNAL_ERROR
+    INTERNAL_ERROR,
+    NO_SUCH_PLAN,
+    UNKNOWN_KEY
   }
 
   private final Result result;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/49] hadoop git commit: HDFS-10571. TestDiskBalancerCommand#testPlanNode failed with IllegalArgumentException. Contributed by Xiaobing Zhou.

Posted by ar...@apache.org.
HDFS-10571. TestDiskBalancerCommand#testPlanNode failed with IllegalArgumentException. Contributed by Xiaobing Zhou.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b2584bee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b2584bee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b2584bee

Branch: refs/heads/trunk
Commit: b2584bee457192ea5789667c1317236f47fa6060
Parents: 8a6e354
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Jun 23 14:48:40 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:02 2016 -0700

----------------------------------------------------------------------
 .../command/TestDiskBalancerCommand.java           | 17 ++++++++++++++---
 1 file changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2584bee/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index b0821e2..e55c418 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -280,7 +280,7 @@ public class TestDiskBalancerCommand {
     final String cmdLine = String
         .format(
             "hdfs diskbalancer %s", planArg);
-    runCommand(cmdLine);
+    runCommand(cmdLine, cluster);
   }
 
   /* Test that illegal arguments are handled correctly*/
@@ -335,12 +335,12 @@ public class TestDiskBalancerCommand {
     runCommand(cmdLine);
   }
 
-  private List<String> runCommand(final String cmdLine) throws Exception {
+  private List<String> runCommandInternal(final String cmdLine) throws
+      Exception {
     String[] cmds = StringUtils.split(cmdLine, ' ');
     org.apache.hadoop.hdfs.tools.DiskBalancer db =
         new org.apache.hadoop.hdfs.tools.DiskBalancer(conf);
 
-    FileSystem.setDefaultUri(conf, clusterJson);
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
     db.run(cmds, out);
@@ -353,6 +353,17 @@ public class TestDiskBalancerCommand {
     return outputs;
   }
 
+  private List<String> runCommand(final String cmdLine) throws Exception {
+    FileSystem.setDefaultUri(conf, clusterJson);
+    return runCommandInternal(cmdLine);
+  }
+
+  private List<String> runCommand(final String cmdLine,
+                                  MiniDFSCluster miniCluster) throws Exception {
+    FileSystem.setDefaultUri(conf, miniCluster.getURI());
+    return runCommandInternal(cmdLine);
+  }
+
   /**
    * Making sure that we can query the node without having done a submit.
    * @throws Exception


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/49] hadoop git commit: HDFS-9735. DiskBalancer : Refactor moveBlockAcrossStorage to be used by disk balancer. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9735. DiskBalancer : Refactor moveBlockAcrossStorage to be used by disk balancer. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7820737c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7820737c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7820737c

Branch: refs/heads/trunk
Commit: 7820737cfa178d9de1bcbb1e99b9677d70901914
Parents: 0506770
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Apr 11 15:58:06 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:20:24 2016 -0700

----------------------------------------------------------------------
 .../server/datanode/fsdataset/FsDatasetSpi.java | 11 +++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 95 +++++++++++++++-----
 .../datanode/fsdataset/impl/FsVolumeImpl.java   | 13 +++
 .../server/datanode/SimulatedFSDataset.java     |  7 ++
 .../extdataset/ExternalDatasetImpl.java         |  8 ++
 .../diskbalancer/DiskBalancerTestUtil.java      | 62 ++++++++++---
 .../diskbalancer/TestDiskBalancerRPC.java       | 53 +++++++++--
 7 files changed, 210 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
index 277b271..eeab098 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
@@ -628,4 +628,15 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Confirm whether the block is deleting
    */
   boolean isDeletingBlock(String bpid, long blockId);
+
+  /**
+   * Moves a given block from one volume to another volume. This is used by disk
+   * balancer.
+   *
+   * @param block       - ExtendedBlock
+   * @param destination - Destination volume
+   * @return Old replica info
+   */
+  ReplicaInfo moveBlockAcrossVolumes(final ExtendedBlock block,
+      FsVolumeSpi destination) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index b042297..2b40538 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -947,29 +947,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       volumeRef = volumes.getNextVolume(targetStorageType, block.getNumBytes());
     }
     try {
-      File oldBlockFile = replicaInfo.getBlockFile();
-      File oldMetaFile = replicaInfo.getMetaFile();
-      FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
-      // Copy files to temp dir first
-      File[] blockFiles = copyBlockFiles(block.getBlockId(),
-          block.getGenerationStamp(), oldMetaFile, oldBlockFile,
-          targetVolume.getTmpDir(block.getBlockPoolId()),
-          replicaInfo.isOnTransientStorage(), smallBufferSize, conf);
-
-      ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
-          replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
-          targetVolume, blockFiles[0].getParentFile(), 0);
-      newReplicaInfo.setNumBytes(blockFiles[1].length());
-      // Finalize the copied files
-      newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
-      synchronized (this) {
-        // Increment numBlocks here as this block moved without knowing to BPS
-        FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
-        volume.getBlockPoolSlice(block.getBlockPoolId()).incrNumBlocks();
-      }
-
-      removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile,
-          oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId());
+      moveBlock(block, replicaInfo, volumeRef);
     } finally {
       if (volumeRef != null) {
         volumeRef.close();
@@ -981,6 +959,77 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   }
 
   /**
+   * Moves a block from a given volume to another.
+   *
+   * @param block       - Extended Block
+   * @param replicaInfo - ReplicaInfo
+   * @param volumeRef   - Volume Ref - Closed by caller.
+   * @return newReplicaInfo
+   * @throws IOException
+   */
+  private ReplicaInfo moveBlock(ExtendedBlock block, ReplicaInfo replicaInfo,
+                                FsVolumeReference volumeRef) throws
+      IOException {
+    File oldBlockFile = replicaInfo.getBlockFile();
+    File oldMetaFile = replicaInfo.getMetaFile();
+    FsVolumeImpl targetVolume = (FsVolumeImpl) volumeRef.getVolume();
+    // Copy files to temp dir first
+    File[] blockFiles = copyBlockFiles(block.getBlockId(),
+        block.getGenerationStamp(), oldMetaFile, oldBlockFile,
+        targetVolume.getTmpDir(block.getBlockPoolId()),
+        replicaInfo.isOnTransientStorage(), smallBufferSize, conf);
+
+    ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
+        replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
+        targetVolume, blockFiles[0].getParentFile(), 0);
+    newReplicaInfo.setNumBytes(blockFiles[1].length());
+    // Finalize the copied files
+    newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
+    synchronized (this) {
+      // Increment numBlocks here as this block moved without knowing to BPS
+      FsVolumeImpl volume = (FsVolumeImpl) newReplicaInfo.getVolume();
+      volume.getBlockPoolSlice(block.getBlockPoolId()).incrNumBlocks();
+    }
+
+    removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile,
+        oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId());
+    return newReplicaInfo;
+  }
+
+  /**
+   * Moves a given block from one volume to another volume. This is used by disk
+   * balancer.
+   *
+   * @param block       - ExtendedBlock
+   * @param destination - Destination volume
+   * @return Old replica info
+   */
+  @Override
+  public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, FsVolumeSpi
+      destination) throws IOException {
+    ReplicaInfo replicaInfo = getReplicaInfo(block);
+    if (replicaInfo.getState() != ReplicaState.FINALIZED) {
+      throw new ReplicaNotFoundException(
+          ReplicaNotFoundException.UNFINALIZED_REPLICA + block);
+    }
+
+    FsVolumeReference volumeRef = null;
+
+    synchronized (this) {
+      volumeRef = destination.obtainReference();
+    }
+
+    try {
+      moveBlock(block, replicaInfo, volumeRef);
+    } finally {
+      if (volumeRef != null) {
+        volumeRef.close();
+      }
+    }
+    return replicaInfo;
+  }
+
+  /**
    * Compute and store the checksum for a block file that does not already have
    * its checksum computed.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
index 68e2537..4a446d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
@@ -711,6 +711,13 @@ public class FsVolumeImpl implements FsVolumeSpi {
                     actualBlockDir.getPath());
                 continue;
               }
+
+              File blkFile = getBlockFile(bpid, block);
+              File metaFile = FsDatasetUtil.findMetaFile(blkFile);
+              block.setGenerationStamp(
+                  Block.getGenerationStamp(metaFile.getName()));
+              block.setNumBytes(blkFile.length());
+
               LOG.trace("nextBlock({}, {}): advancing to {}",
                   storageID, bpid, block);
               return block;
@@ -732,6 +739,12 @@ public class FsVolumeImpl implements FsVolumeSpi {
       }
     }
 
+    private File getBlockFile(String bpid, ExtendedBlock blk)
+        throws IOException {
+      return new File(DatanodeUtil.idToBlockDir(getFinalizedDir(bpid),
+          blk.getBlockId()).toString() + "/" + blk.getBlockName());
+    }
+
     @Override
     public boolean atEnd() {
       return state.atEnd;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 25034c6..24f4a52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -1359,5 +1359,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public boolean isDeletingBlock(String bpid, long blockId) {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block,
+    FsVolumeSpi destination) throws IOException {
+    return null;
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
index c872e61..8518ddd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
@@ -442,4 +442,12 @@ public class ExternalDatasetImpl implements FsDatasetSpi<ExternalVolumeImpl> {
   public boolean isDeletingBlock(String bpid, long blockId) {
     return false;
   }
+
+  @Override
+  public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block,
+                                            FsVolumeSpi destination)
+      throws IOException {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
index 9613919..43bb184 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.hdfs.server.diskbalancer;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
@@ -26,6 +29,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.util.Time;
 
+import java.io.IOException;
 import java.util.Random;
 import java.util.UUID;
 
@@ -53,7 +57,6 @@ public class DiskBalancerTestUtil {
    * Returns a random string.
    *
    * @param length - Number of chars in the string
-   *
    * @return random String
    */
   private String getRandomName(int length) {
@@ -122,7 +125,6 @@ public class DiskBalancerTestUtil {
    * Creates a Random Volume for testing purpose.
    *
    * @param type - StorageType
-   *
    * @return DiskBalancerVolume
    */
   public DiskBalancerVolume createRandomVolume(StorageType type) {
@@ -142,11 +144,9 @@ public class DiskBalancerTestUtil {
   /**
    * Creates a RandomVolumeSet.
    *
-   * @param type -Storage Type
+   * @param type      - Storage Type
    * @param diskCount - How many disks you need.
-   *
    * @return volumeSet
-   *
    * @throws Exception
    */
   public DiskBalancerVolumeSet createRandomVolumeSet(StorageType type,
@@ -168,9 +168,7 @@ public class DiskBalancerTestUtil {
    *
    * @param diskTypes - Storage types needed in the Node
    * @param diskCount - Disk count - that many disks of each type is created
-   *
    * @return DataNode
-   *
    * @throws Exception
    */
   public DiskBalancerDataNode createRandomDataNode(StorageType[] diskTypes,
@@ -195,11 +193,9 @@ public class DiskBalancerTestUtil {
    * Creates a RandomCluster.
    *
    * @param dataNodeCount - How many nodes you need
-   * @param diskTypes - StorageTypes you need in each node
-   * @param diskCount - How many disks you need of each type.
-   *
+   * @param diskTypes     - StorageTypes you need in each node
+   * @param diskCount     - How many disks you need of each type.
    * @return Cluster
-   *
    * @throws Exception
    */
   public DiskBalancerCluster createRandCluster(int dataNodeCount,
@@ -224,4 +220,48 @@ public class DiskBalancerTestUtil {
     return cluster;
   }
 
+  /**
+   * Returns the number of blocks on a volume.
+   *
+   * @param source - Source Volume.
+   * @return Number of Blocks.
+   * @throws IOException
+   */
+  public static int getBlockCount(FsVolumeSpi source) throws IOException {
+    int count = 0;
+    for (String blockPoolID : source.getBlockPoolList()) {
+      FsVolumeSpi.BlockIterator sourceIter =
+          source.newBlockIterator(blockPoolID, "TestDiskBalancerSource");
+      while (!sourceIter.atEnd()) {
+        ExtendedBlock block = sourceIter.nextBlock();
+        if (block != null) {
+          count++;
+        }
+      }
+    }
+    return count;
+  }
+
+  /**
+   * Moves all blocks to the destination volume.
+   *
+   * @param fsDataset - Dataset
+   * @param source    - Source Volume.
+   * @param dest      - Destination Volume.
+   * @throws IOException
+   */
+  public static void moveAllDataToDestVolume(FsDatasetSpi fsDataset,
+    FsVolumeSpi source, FsVolumeSpi dest) throws IOException {
+
+    for (String blockPoolID : source.getBlockPoolList()) {
+      FsVolumeSpi.BlockIterator sourceIter =
+          source.newBlockIterator(blockPoolID, "TestDiskBalancerSource");
+      while (!sourceIter.atEnd()) {
+        ExtendedBlock block = sourceIter.nextBlock();
+        if (block != null) {
+          fsDataset.moveBlockAcrossVolumes(block, dest);
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7820737c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 27cd8eb..81a0609 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -19,35 +19,39 @@ package org.apache.hadoop.hdfs.server.diskbalancer;
 
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
-import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException.*;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException.Result;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
-import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
-import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
-import org.hamcrest.*;
+import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-import org.codehaus.jackson.map.ObjectMapper;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Random;
 
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
+import static org.junit.Assert.assertTrue;
 
 public class TestDiskBalancerRPC {
   @Rule
@@ -227,6 +231,45 @@ public class TestDiskBalancerRPC {
     Assert.assertTrue(status.getResult() == NO_PLAN);
   }
 
+  @Test
+  public void testMoveBlockAcrossVolume() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    final int DEFAULT_BLOCK_SIZE = 100;
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    String fileName = "/tmp.txt";
+    Path filePath = new Path(fileName);
+    final int numDatanodes = 1;
+    final int dnIndex = 0;
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes).build();
+    FsVolumeImpl source = null;
+    FsVolumeImpl dest = null;
+    try {
+      cluster.waitActive();
+      Random r = new Random();
+      FileSystem fs = cluster.getFileSystem(dnIndex);
+      DFSTestUtil.createFile(fs, filePath, 10 * 1024,
+          (short) 1, r.nextLong());
+      DataNode dnNode = cluster.getDataNodes().get(dnIndex);
+      FsDatasetSpi.FsVolumeReferences refs =
+          dnNode.getFSDataset().getFsVolumeReferences();
+      try {
+        source = (FsVolumeImpl) refs.get(0);
+        dest = (FsVolumeImpl) refs.get(1);
+        DiskBalancerTestUtil.moveAllDataToDestVolume(dnNode.getFSDataset(),
+            source, dest);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) == 0);
+      } finally {
+        refs.close();
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+
   private class RpcTestHelper {
     private NodePlan plan;
     private int planVersion;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/49] hadoop git commit: HDFS-10517. DiskBalancer: Support help command. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10517. DiskBalancer: Support help command. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/af11ab34
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/af11ab34
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/af11ab34

Branch: refs/heads/trunk
Commit: af11ab34d0cff3883885a25aa918be4f98566142
Parents: b502102
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Jun 13 14:02:04 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/CancelCommand.java     |  20 +-
 .../server/diskbalancer/command/Command.java    |   4 +-
 .../diskbalancer/command/ExecuteCommand.java    |  17 +-
 .../diskbalancer/command/HelpCommand.java       | 108 +++++++++
 .../diskbalancer/command/PlanCommand.java       |  28 ++-
 .../diskbalancer/command/QueryCommand.java      |  15 +-
 .../diskbalancer/command/ReportCommand.java     |  18 +-
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 233 ++++++++++++++-----
 8 files changed, 360 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
index 3834d9b..740292d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -126,12 +127,21 @@ public class CancelCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Cancels a running command. e.g -cancel <PlanFile> or -cancel " +
-        "<planID> -node <datanode>";
+  public void printHelp() {
+    String header = "Cancel command cancels a running disk balancer operation" +
+        ".\n\n";
+
+    String footer = "\nCancel command can be run via pointing to a plan file," +
+        " or by reading the plan ID using the query command and then using " +
+        "planID and hostname. Examples of how to run this command are \n" +
+        "hdfs diskbalancer -cancel <planfile> \n" +
+        "hdfs diskbalancer -cancel <planID> -node <hostname>";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -cancel <planFile> | -cancel " +
+        "<planID> -node <hostname>",
+        header, DiskBalancer.getCancelOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index bbf91ca..d2813e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -100,10 +100,8 @@ public abstract class Command extends Configured {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
-  protected abstract String getHelp();
+  public abstract void printHelp();
 
   /**
    * verifies user provided URL.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index 85f2a86..5fd1f0a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -98,12 +99,18 @@ public class ExecuteCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Execute command takes a plan and runs it against the node. e.g. " +
-        "hdfs diskbalancer -execute <nodename.plan.json> ";
+  public void printHelp() {
+    String header = "Execute command runs a submits a plan for execution on " +
+        "the given data node.\n\n";
+
+    String footer = "\nExecute command submits the job to data node and " +
+        "returns immediately. The state of job can be monitored via query " +
+        "command. ";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -execute <planfile>",
+        header, DiskBalancer.getExecuteOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
new file mode 100644
index 0000000..205df3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
@@ -0,0 +1,108 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
+/**
+ * Help Command prints out detailed help about each command.
+ */
+public class HelpCommand extends Command {
+
+  /**
+   * Constructs a help command.
+   *
+   * @param conf - config
+   */
+  public HelpCommand(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.debug("Processing help Command.");
+    if (cmd == null) {
+      this.printHelp();
+      return;
+    }
+
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.HELP));
+    verifyCommandOptions(DiskBalancer.HELP, cmd);
+    String helpCommand = cmd.getOptionValue(DiskBalancer.HELP);
+    if (helpCommand == null || helpCommand.isEmpty()) {
+      this.printHelp();
+      return;
+    }
+
+    helpCommand = helpCommand.trim();
+    helpCommand = helpCommand.toLowerCase();
+    Command command = null;
+    switch (helpCommand) {
+    case DiskBalancer.PLAN:
+      command = new PlanCommand(getConf());
+      break;
+    case DiskBalancer.EXECUTE:
+      command = new ExecuteCommand(getConf());
+      break;
+    case DiskBalancer.QUERY:
+      command = new QueryCommand(getConf());
+      break;
+    case DiskBalancer.CANCEL:
+      command = new CancelCommand(getConf());
+      break;
+    case DiskBalancer.REPORT:
+      command = new ReportCommand(getConf(), null);
+      break;
+    default:
+      command = this;
+      break;
+    }
+    command.printHelp();
+
+  }
+
+  /**
+   * Gets extended help for this command.
+   */
+  @Override
+  public void printHelp() {
+    String header = "\nDiskBalancer distributes data evenly between " +
+        "different disks on a datanode. " +
+        "DiskBalancer operates by generating a plan, that tells datanode " +
+        "how to move data between disks. Users can execute a plan by " +
+        "submitting it to the datanode. \nTo get specific help on a " +
+        "particular command please run \n\n hdfs diskbalancer -help <command>.";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer [command] [options]",
+        header, DiskBalancer.getHelpOptions(), "");
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 91ab7fb..76bdc9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -19,15 +19,18 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
@@ -182,12 +185,19 @@ public class PlanCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "This commands creates a disk balancer plan for given datanode";
+  public void printHelp() {
+    String header = "creates a plan that describes how much data should be " +
+        "moved between disks.\n\n";
+
+    String footer = "\nPlan command creates a set of steps that represent a " +
+        "planned data move. A plan file can be executed on a data node, which" +
+        " will balance the data.";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -uri <namenode> -plan " +
+        "<hostname> [options]", header, DiskBalancer.getPlanOptions(), footer);
   }
 
   /**
@@ -221,9 +231,9 @@ public class PlanCommand extends Command {
 
     System.out.println(
         StringUtils.center("Source Disk", 30) +
-        StringUtils.center("Dest.Disk", 30) +
-        StringUtils.center("Size", 10) +
-        StringUtils.center("Type", 10));
+            StringUtils.center("Dest.Disk", 30) +
+            StringUtils.center("Size", 10) +
+            StringUtils.center("Type", 10));
 
     for (NodePlan plan : plans) {
       for (Step step : plan.getVolumeSetPlans()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index ea7dbcc..3a3b97f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -86,11 +87,17 @@ public class QueryCommand extends Command {
 
   /**
    * Gets extended help for this command.
-   *
-   * @return Help Message
    */
   @Override
-  protected String getHelp() {
-    return "Gets the status of disk balancing on a given node";
+  public void printHelp() {
+    String header = "Query Plan queries a given data node about the " +
+        "current state of disk balancer execution.\n\n";
+
+    String footer = "\nQuery command retrievs the plan ID and the current " +
+        "running state. ";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -query <hostname>  [options]",
+        header, DiskBalancer.getQueryOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
index acf9ff2..eb6afcc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.ListIterator;
 
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.conf.Configuration;
@@ -184,14 +185,23 @@ public class ReportCommand extends Command {
     }
   }
 
+  /**
+   * Prints the help message.
+   */
   @Override
-  protected String getHelp() {
-    return "Report volume information for a specific DataNode or top X "
-        + "one(s) benefiting from running DiskBalancer, "
-        + "top defaults to " + getDefaultTop() + ". E.g.:\n"
+  public void printHelp() {
+    String header = "Report command reports the volume information of a given" +
+        " datanode, or prints out the list of nodes that will benefit from " +
+        "running disk balancer. Top defaults to " + getDefaultTop();
+    String footer = ". E.g.:\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report -top 5\n"
         + "hdfs diskbalancer -uri http://namenode.uri -report "
         + "-node {DataNodeID | IP | Hostname}";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -uri http://namenode.uri " +
+        "-report [options]",
+        header, DiskBalancer.getReportOptions(), footer);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af11ab34/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 1251e96..d83a49c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -18,8 +18,8 @@ package org.apache.hadoop.hdfs.tools;
 
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.CancelCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.HelpCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.QueryCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ReportCommand;
@@ -139,6 +140,13 @@ public class DiskBalancer extends Configured implements Tool {
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancer.class);
 
+  private static final Options PLAN_OPTIONS = new Options();
+  private static final Options EXECUTE_OPTIONS = new Options();
+  private static final Options QUERY_OPTIONS = new Options();
+  private static final Options HELP_OPTIONS = new Options();
+  private static final Options CANCEL_OPTIONS = new Options();
+  private static final Options REPORT_OPTIONS = new Options();
+
   /**
    * Construct a DiskBalancer.
    *
@@ -182,7 +190,7 @@ public class DiskBalancer extends Configured implements Tool {
    * Execute the command with the given arguments.
    *
    * @param args command specific arguments.
-   * @param out the output stream used for printing
+   * @param out  the output stream used for printing
    * @return exit code.
    * @throws Exception
    */
@@ -200,6 +208,7 @@ public class DiskBalancer extends Configured implements Tool {
   private Options getOpts() {
     Options opts = new Options();
     addPlanCommands(opts);
+    addHelpCommands(opts);
     addExecuteCommands(opts);
     addQueryCommands(opts);
     addCancelCommands(opts);
@@ -208,101 +217,216 @@ public class DiskBalancer extends Configured implements Tool {
   }
 
   /**
+   * Returns Plan options.
+   *
+   * @return Options.
+   */
+  public static Options getPlanOptions() {
+    return PLAN_OPTIONS;
+  }
+
+  /**
+   * Returns help options.
+   *
+   * @return - help options.
+   */
+  public static Options getHelpOptions() {
+    return HELP_OPTIONS;
+  }
+
+  /**
+   * Retuns execute options.
+   *
+   * @return - execute options.
+   */
+  public static Options getExecuteOptions() {
+    return EXECUTE_OPTIONS;
+  }
+
+  /**
+   * Returns Query Options.
+   *
+   * @return query Options
+   */
+  public static Options getQueryOptions() {
+    return QUERY_OPTIONS;
+  }
+
+  /**
+   * Returns Cancel Options.
+   *
+   * @return Options
+   */
+  public static Options getCancelOptions() {
+    return CANCEL_OPTIONS;
+  }
+
+  /**
+   * Returns Report Options.
+   *
+   * @return Options
+   */
+  public static Options getReportOptions() {
+    return REPORT_OPTIONS;
+  }
+
+  /**
    * Adds commands for plan command.
    *
-   * @param opt - Options
+   * @return Options.
    */
   private void addPlanCommands(Options opt) {
 
-    Option nameNodeUri =
-        new Option(NAMENODEURI, true, "NameNode URI. e.g http://namenode" +
-            ".mycluster.com or file:///myCluster" +
-            ".json");
-    opt.addOption(nameNodeUri);
+    Option uri = OptionBuilder.withLongOpt(NAMENODEURI)
+        .withDescription("Address of the Namenode. e,g. hdfs://namenode")
+        .hasArg()
+        .create();
+    getPlanOptions().addOption(uri);
+    opt.addOption(uri);
+
+    Option plan = OptionBuilder.withLongOpt(PLAN)
+        .withDescription("creates a plan for datanode.")
+        .hasArg()
+        .create();
+    getPlanOptions().addOption(plan);
+    opt.addOption(plan);
 
-    Option outFile =
-        new Option(OUTFILE, true, "File to write output to, if not specified " +
-            "defaults will be used." +
-            "e.g -out outfile.txt");
-    opt.addOption(outFile);
 
-    Option plan = new Option(PLAN, true , "create a plan for the given node. " +
-        "e.g -plan <nodename> | <nodeIP> | <nodeUUID>");
-    opt.addOption(plan);
+    Option outFile = OptionBuilder.withLongOpt(OUTFILE)
+        .hasArg()
+        .withDescription("File to write output to, if not specified " +
+            "defaults will be used.")
+        .create();
+    getPlanOptions().addOption(outFile);
+    opt.addOption(outFile);
 
-    Option bandwidth = new Option(BANDWIDTH, true, "Maximum disk bandwidth to" +
-        " be consumed by diskBalancer. " +
-        "Expressed as MBs per second.");
+    Option bandwidth = OptionBuilder.withLongOpt(BANDWIDTH)
+        .hasArg()
+        .withDescription("Maximum disk bandwidth to be consumed by " +
+            "diskBalancer. e.g. 10")
+        .create();
+    getPlanOptions().addOption(bandwidth);
     opt.addOption(bandwidth);
 
-    Option threshold = new Option(THRESHOLD, true, "Percentage skew that we " +
-        "tolerate before diskbalancer starts working or stops when reaching " +
-        "that range.");
+    Option threshold = OptionBuilder.withLongOpt(THRESHOLD)
+        .hasArg()
+        .withDescription("Percentage skew that we" +
+            "tolerate before diskbalancer starts working e.g. 10")
+        .create();
+    getPlanOptions().addOption(threshold);
     opt.addOption(threshold);
 
-    Option maxErrors = new Option(MAXERROR, true, "Describes how many errors " +
-        "can be tolerated while copying between a pair of disks.");
-    opt.addOption(maxErrors);
 
-    Option help =
-        new Option(HELP, true, "Help about a command or this message");
-    opt.addOption(help);
+    Option maxError = OptionBuilder.withLongOpt(MAXERROR)
+        .hasArg()
+        .withDescription("Describes how many errors " +
+            "can be tolerated while copying between a pair of disks.")
+        .create();
+    getPlanOptions().addOption(maxError);
+    opt.addOption(maxError);
 
-    Option verbose = new Option(VERBOSE, "Print out the summary of the plan");
+    Option verbose = OptionBuilder.withLongOpt(VERBOSE)
+        .withDescription("Print out the summary of the plan on console")
+        .create();
+    getPlanOptions().addOption(verbose);
     opt.addOption(verbose);
+  }
 
+  /**
+   * Adds Help to the options.
+   */
+  private void addHelpCommands(Options opt) {
+    Option help = OptionBuilder.withLongOpt(HELP)
+        .hasOptionalArg()
+        .withArgName(HELP)
+        .withDescription("valid commands are plan | execute | query | cancel" +
+            " | report")
+        .create();
+    getHelpOptions().addOption(help);
+    opt.addOption(help);
   }
 
   /**
    * Adds execute command options.
+   *
    * @param opt Options
    */
   private void addExecuteCommands(Options opt) {
-    Option execute = new Option(EXECUTE, true , "Takes a plan file and " +
-        "submits it for execution to the datanode. e.g -execute <planfile>");
+    Option execute = OptionBuilder.withLongOpt(EXECUTE)
+        .hasArg()
+        .withDescription("Takes a plan file and " +
+            "submits it for execution by the datanode.")
+        .create();
+    getExecuteOptions().addOption(execute);
     opt.addOption(execute);
   }
 
   /**
    * Adds query command options.
+   *
    * @param opt Options
    */
   private void addQueryCommands(Options opt) {
-    Option query = new Option(QUERY, true, "Queries the disk balancer " +
-        "status of a given datanode. e.g. -query <nodename>");
+    Option query = OptionBuilder.withLongOpt(QUERY)
+        .hasArg()
+        .withDescription("Queries the disk balancer " +
+            "status of a given datanode.")
+        .create();
+    getQueryOptions().addOption(query);
     opt.addOption(query);
+
+    // Please note: Adding this only to Query options since -v is already
+    // added to global table.
+    Option verbose = OptionBuilder.withLongOpt(VERBOSE)
+        .withDescription("Prints details of the plan that is being executed " +
+            "on the node.")
+        .create();
+    getQueryOptions().addOption(verbose);
   }
 
   /**
    * Adds cancel command options.
+   *
    * @param opt Options
    */
   private void addCancelCommands(Options opt) {
-    Option cancel = new Option(CANCEL, true, "Cancels a running plan. -cancel" +
-        " <planFile> or -cancel <planID> -node <datanode:port>");
+    Option cancel = OptionBuilder.withLongOpt(CANCEL)
+        .hasArg()
+        .withDescription("Cancels a running plan using a plan file.")
+        .create();
+    getCancelOptions().addOption(cancel);
     opt.addOption(cancel);
-    Option node = new Option(NODE, true, "Name of the datanode in name:port " +
-        "format");
+
+    Option node = OptionBuilder.withLongOpt(NODE)
+        .hasArg()
+        .withDescription("Cancels a running plan using a plan ID and hostName")
+        .create();
+
+    getCancelOptions().addOption(node);
     opt.addOption(node);
   }
 
   /**
    * Adds report command options.
+   *
    * @param opt Options
    */
   private void addReportCommands(Options opt) {
-    Option report = new Option(REPORT, false,
-        "Report volume information of DataNode(s)"
-            + " benefiting from running DiskBalancer. "
-            + "-report [top -X] | [-node {DataNodeID | IP | Hostname}].");
+    Option report = OptionBuilder.withLongOpt(REPORT)
+        .withDescription("List nodes that will benefit from running " +
+            "DiskBalancer.")
+        .create();
+    getReportOptions().addOption(report);
     opt.addOption(report);
 
     Option top = new Option(TOP, true,
-        "specify the top number of nodes to be processed.");
+        "specify the number of nodes to be listed which has data imbalance.");
+    getReportOptions().addOption(top);
     opt.addOption(top);
 
     Option node = new Option(NODE, true,
-        "Name of the datanode in the format of DataNodeID, IP or hostname.");
+        "Datanode address, it can be DataNodeID, IP or hostname.");
+    getReportOptions().addOption(node);
     opt.addOption(node);
   }
 
@@ -322,9 +446,9 @@ public class DiskBalancer extends Configured implements Tool {
   /**
    * Dispatches calls to the right command Handler classes.
    *
-   * @param cmd - CommandLine
+   * @param cmd  - CommandLine
    * @param opts options of command line
-   * @param out the output stream used for printing
+   * @param out  the output stream used for printing
    * @throws IOException
    * @throws URISyntaxException
    */
@@ -338,15 +462,15 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new PlanCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.EXECUTE)) {
+      if (cmd.hasOption(DiskBalancer.EXECUTE)) {
         currentCommand = new ExecuteCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.QUERY)) {
+      if (cmd.hasOption(DiskBalancer.QUERY)) {
         currentCommand = new QueryCommand(getConf());
       }
 
-      if(cmd.hasOption(DiskBalancer.CANCEL)) {
+      if (cmd.hasOption(DiskBalancer.CANCEL)) {
         currentCommand = new CancelCommand(getConf());
       }
 
@@ -354,13 +478,16 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new ReportCommand(getConf(), out);
       }
 
-      if(currentCommand == null) {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",
-            "disk balancer commands", opts,
-            "Please correct your command and try again.");
+      if (cmd.hasOption(DiskBalancer.HELP)) {
+        currentCommand = new HelpCommand(getConf());
+      }
+
+      // Invoke Main help here.
+      if (currentCommand == null) {
+        new HelpCommand(getConf()).execute(null);
         return 1;
       }
+
       currentCommand.execute(cmd);
     } catch (Exception ex) {
       System.err.printf(ex.getMessage());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/49] hadoop git commit: HDFS-9817. Use SLF4J in new classes. Contributed by Anu Engineer

Posted by ar...@apache.org.
HDFS-9817. Use SLF4J in new classes. Contributed by Anu Engineer


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/747227e9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/747227e9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/747227e9

Branch: refs/heads/trunk
Commit: 747227e9dea10ac6b5f601b7cf4dcc418b10d9c8
Parents: 3df0781
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Mar 4 20:16:13 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/DiskBalancer.java      |  7 +++---
 .../connectors/ConnectorFactory.java            | 15 +++++++------
 .../connectors/DBNameNodeConnector.java         |  9 ++++----
 .../connectors/JsonNodeConnector.java           |  7 +++---
 .../datamodel/DiskBalancerCluster.java          | 11 +++++-----
 .../datamodel/DiskBalancerVolumeSet.java        |  9 ++++----
 .../diskbalancer/planner/GreedyPlanner.java     | 23 ++++++++++----------
 .../diskbalancer/planner/PlannerFactory.java    |  7 +++---
 8 files changed, 47 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 81dbb2d..d5c402e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -60,7 +60,8 @@ import java.util.concurrent.locks.ReentrantLock;
 @InterfaceAudience.Private
 public class DiskBalancer {
 
-  private static final Log LOG = LogFactory.getLog(DiskBalancer.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DiskBalancer
+      .class);
   private final FsDatasetSpi<?> dataset;
   private final String dataNodeUUID;
   private final BlockMover blockMover;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
index 040923a..484a64b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
@@ -16,8 +16,8 @@
  */
 package org.apache.hadoop.hdfs.server.diskbalancer.connectors;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
@@ -28,7 +28,8 @@ import java.net.URISyntaxException;
  * Connector factory creates appropriate connector based on the URL.
  */
 public final class ConnectorFactory {
-  static final Log LOG = LogFactory.getLog(ConnectorFactory.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectorFactory.class);
 
   /**
    * Constructs an appropriate connector based on the URL.
@@ -37,13 +38,13 @@ public final class ConnectorFactory {
    */
   public static ClusterConnector getCluster(URI clusterURI, Configuration
       conf) throws IOException, URISyntaxException {
-    LOG.info("Cluster URI : " + clusterURI);
-    LOG.info("scheme : " + clusterURI.getScheme());
+    LOG.debug("Cluster URI : {}" , clusterURI);
+    LOG.debug("scheme : {}" , clusterURI.getScheme());
     if (clusterURI.getScheme().startsWith("file")) {
-      LOG.info("Creating a JsonNodeConnector");
+      LOG.debug("Creating a JsonNodeConnector");
       return new JsonNodeConnector(clusterURI.toURL());
     } else {
-      LOG.info("Creating NameNode connector");
+      LOG.debug("Creating NameNode connector");
       return new DBNameNodeConnector(clusterURI, conf);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
index c35e934..acf1fa1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.connectors;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -42,7 +42,8 @@ import java.util.List;
  * given cluster.
  */
 class DBNameNodeConnector implements ClusterConnector {
-  static final Log LOG = LogFactory.getLog(DBNameNodeConnector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DBNameNodeConnector.class);
   static final Path DISKBALANCER_ID_PATH = new Path("/system/diskbalancer.id");
   private final URI clusterURI;
   private final NameNodeConnector connector;
@@ -159,4 +160,4 @@ class DBNameNodeConnector implements ClusterConnector {
     }
 
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
index bf5aebb..cc79648 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.connectors;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerDataNode;
@@ -33,7 +33,8 @@ import java.util.List;
  * A connector that understands JSON data cluster models.
  */
 public class JsonNodeConnector implements ClusterConnector {
-  static final Log LOG = LogFactory.getLog(JsonNodeConnector.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JsonNodeConnector.class);
   private final URL clusterURI;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
index c86fc9a..7b82278 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.server.diskbalancer.datamodel;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Planner;
@@ -66,7 +66,8 @@ import java.util.concurrent.Future;
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class DiskBalancerCluster {
 
-  static final Log LOG = LogFactory.getLog(DiskBalancerCluster.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerCluster.class);
   private final Set<String> exclusionList;
   private final Set<String> inclusionList;
   private ClusterConnector clusterConnector;
@@ -264,7 +265,7 @@ public class DiskBalancerCluster {
    */
   public void createOutPutDirectory() throws IOException {
     if (Files.exists(Paths.get(this.getOutput()))) {
-      LOG.fatal("An output directory already exists at this location. Path : " +
+      LOG.error("An output directory already exists at this location. Path : " +
           this.getOutput());
       throw new IOException(
           "An output directory already exists at this location. Path : " +
@@ -273,7 +274,7 @@ public class DiskBalancerCluster {
 
     File f = new File(this.getOutput());
     if (!f.mkdirs()) {
-      LOG.fatal("Unable to create the output directory. Path : " + this
+      LOG.error("Unable to create the output directory. Path : " + this
           .getOutput());
       throw new IOException(
           "Unable to create the output directory. Path : " + this.getOutput());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
index 49c8558..2faf249 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
@@ -19,8 +19,8 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.datamodel;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.codehaus.jackson.annotate.JsonIgnore;
 import org.codehaus.jackson.annotate.JsonProperty;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
@@ -40,7 +40,8 @@ import java.util.UUID;
  */
 @JsonIgnoreProperties({"sortedQueue", "volumeCount", "idealUsed"})
 public class DiskBalancerVolumeSet {
-  static final Log LOG = LogFactory.getLog(DiskBalancerVolumeSet.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancerVolumeSet.class);
   private final int maxDisks = 256;
 
   @JsonProperty("transient")
@@ -172,7 +173,7 @@ public class DiskBalancerVolumeSet {
                                       volume.getStorageType(),
                                       volume.getUuid());
 
-    LOG.fatal(errMessage);
+    LOG.error(errMessage);
     volume.setSkip(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
index f0fc776..88ddca4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
@@ -42,7 +42,8 @@ public class GreedyPlanner implements Planner {
   public static final long MB = 1024L * 1024L;
   public static final long GB = MB * 1024L;
   public static final long TB = GB * 1024L;
-  static final Log LOG = LogFactory.getLog(GreedyPlanner.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(GreedyPlanner.class);
   private final float threshold;
 
   /**
@@ -108,13 +109,13 @@ public class GreedyPlanner implements Planner {
       if (!lowVolume.isSkip() && !highVolume.isSkip()) {
         nextStep = computeMove(currentSet, lowVolume, highVolume);
       } else {
-        LOG.debug("Skipping compute move. lowVolume :" + lowVolume.getPath());
-        LOG.debug("Skipping compute move. highVolume :" + highVolume.getPath());
+        LOG.debug("Skipping compute move. lowVolume: {} highVolume: {}",
+            lowVolume.getPath(), highVolume.getPath());
       }
 
       applyStep(nextStep, currentSet, lowVolume, highVolume);
       if (nextStep != null) {
-        LOG.debug("Step : " + nextStep.toString());
+        LOG.debug("Step : {} ",  nextStep.toString());
         plan.addStep(nextStep);
       }
     }
@@ -179,9 +180,8 @@ public class GreedyPlanner implements Planner {
     // This disk cannot take any more data from any disk.
     // Remove it from our computation matrix.
     if (maxLowVolumeCanReceive <= 0) {
-      LOG.debug(lowVolume.getPath() +
-          " Skipping disk from computation. Maximum data size " +
-          "achieved.");
+      LOG.debug("{} Skipping disk from computation. Maximum data size " +
+          "achieved.", lowVolume.getPath());
       lowVolume.setSkip(true);
     }
 
@@ -191,9 +191,8 @@ public class GreedyPlanner implements Planner {
     // This volume cannot give any more data, remove it from the
     // computation matrix
     if (maxHighVolumeCanGive <= 0) {
-      LOG.debug(highVolume.getPath() +
-          " Skipping disk from computation. Minimum data size " +
-          "achieved.");
+      LOG.debug(" {} Skipping disk from computation. Minimum data size " +
+          "achieved.", highVolume.getPath());
       highVolume.setSkip(true);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/747227e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
index ae18e05..24f2970 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
@@ -16,8 +16,8 @@
  */
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerDataNode;
 
@@ -25,7 +25,8 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
  * Returns a planner based on the user defined tags.
  */
 public final class PlannerFactory {
-  static final Log LOG = LogFactory.getLog(PlannerFactory.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PlannerFactory.class);
 
   public static final String GREEDY_PLANNER = "greedyPlanner";
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/49] hadoop git commit: HDFS-9856. Suppress Jenkins warning for sample JSON file. Contributed by Xiaobing Zhou

Posted by ar...@apache.org.
HDFS-9856. Suppress Jenkins warning for sample JSON file. Contributed by Xiaobing Zhou


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3df0781a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3df0781a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3df0781a

Branch: refs/heads/trunk
Commit: 3df0781aa7dceadfdd3434ff81c47e73d9da008a
Parents: 9847640
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Mar 3 21:27:57 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3df0781a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 3696797..93e7392 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -396,6 +396,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/static/json-bignum.js</exclude>
             <exclude>src/main/webapps/static/dataTables.bootstrap.css</exclude>
             <exclude>src/main/webapps/static/dataTables.bootstrap.js</exclude>
+            <exclude>src/test/resources/diskBalancer/data-cluster-3node-3disk.json</exclude>
+            <exclude>HDFS-1312_CHANGES.txt</exclude>
           </excludes>
         </configuration>
       </plugin>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/49] hadoop git commit: HDFS-9449. DiskBalancer: Add connectors. Contributed by Anu Engineer

Posted by ar...@apache.org.
HDFS-9449. DiskBalancer: Add connectors. Contributed by Anu Engineer


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/30c6ebd6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/30c6ebd6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/30c6ebd6

Branch: refs/heads/trunk
Commit: 30c6ebd69919a477a582e599fb253ffe5c2982e1
Parents: 91a5c48
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Dec 1 14:43:06 2015 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   2 +
 .../connectors/ConnectorFactory.java            |  54 +++++++
 .../connectors/DBNameNodeConnector.java         | 162 +++++++++++++++++++
 .../connectors/JsonNodeConnector.java           |  77 +++++++++
 .../diskbalancer/connectors/package-info.java   |  10 +-
 .../server/diskbalancer/TestConnectors.java     |  82 ++++++++++
 6 files changed, 386 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 5a71032..cad8e49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -3,3 +3,5 @@ HDFS-1312 Change Log
   NEW FEATURES
 
     HDFS-9420. Add DataModels for DiskBalancer. (Anu Engineer via szetszwo)
+
+    HDFS-9449. DiskBalancer: Add connectors. (Anu Engineer via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
new file mode 100644
index 0000000..040923a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
@@ -0,0 +1,54 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Connector factory creates appropriate connector based on the URL.
+ */
+public final class ConnectorFactory {
+  static final Log LOG = LogFactory.getLog(ConnectorFactory.class);
+
+  /**
+   * Constructs an appropriate connector based on the URL.
+   * @param clusterURI - URL
+   * @return ClusterConnector
+   */
+  public static ClusterConnector getCluster(URI clusterURI, Configuration
+      conf) throws IOException, URISyntaxException {
+    LOG.info("Cluster URI : " + clusterURI);
+    LOG.info("scheme : " + clusterURI.getScheme());
+    if (clusterURI.getScheme().startsWith("file")) {
+      LOG.info("Creating a JsonNodeConnector");
+      return new JsonNodeConnector(clusterURI.toURL());
+    } else {
+      LOG.info("Creating NameNode connector");
+      return new DBNameNodeConnector(clusterURI, conf);
+    }
+  }
+
+  private ConnectorFactory() {
+    // never constructed
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
new file mode 100644
index 0000000..c35e934
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
@@ -0,0 +1,162 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * DBNameNodeConnector connects to Namenode and extracts information from a
+ * given cluster.
+ */
+class DBNameNodeConnector implements ClusterConnector {
+  static final Log LOG = LogFactory.getLog(DBNameNodeConnector.class);
+  static final Path DISKBALANCER_ID_PATH = new Path("/system/diskbalancer.id");
+  private final URI clusterURI;
+  private final NameNodeConnector connector;
+
+  /**
+   * Constructs a DBNameNodeConnector.
+   *
+   * @param clusterURI - URL to connect to.
+   */
+  public DBNameNodeConnector(URI clusterURI, Configuration conf) throws
+      IOException, URISyntaxException {
+
+    // we don't care how many instances of disk balancers run.
+    // The admission is controlled at the data node, where we will
+    // execute only one plan at a given time.
+    NameNodeConnector.setWrite2IdFile(false);
+
+    try {
+      connector = new NameNodeConnector("DiskBalancer",
+          clusterURI, DISKBALANCER_ID_PATH, null, conf, 1);
+    } catch (IOException ex) {
+      LOG.error("Unable to connect to NameNode " + ex.toString());
+      throw ex;
+    }
+
+    this.clusterURI = clusterURI;
+  }
+
+  /**
+   * getNodes function returns a list of DiskBalancerDataNodes.
+   *
+   * @return Array of DiskBalancerDataNodes
+   */
+  @Override
+  public List<DiskBalancerDataNode> getNodes() throws Exception {
+    Preconditions.checkNotNull(this.connector);
+    List<DiskBalancerDataNode> nodeList = new LinkedList<>();
+    DatanodeStorageReport[] reports = this.connector
+        .getLiveDatanodeStorageReport();
+
+    for (DatanodeStorageReport report : reports) {
+      DiskBalancerDataNode datanode = getBalancerNodeFromDataNode(
+          report.getDatanodeInfo());
+      getVolumeInfoFromStorageReports(datanode, report.getStorageReports());
+      nodeList.add(datanode);
+    }
+    return nodeList;
+  }
+
+  /**
+   * Returns info about the connector.
+   *
+   * @return String.
+   */
+  @Override
+  public String getConnectorInfo() {
+    return "Name Node Connector : " + clusterURI.toString();
+  }
+
+  /**
+   * This function maps the required fields from DataNodeInfo to disk
+   * BalancerDataNode.
+   *
+   * @param nodeInfo
+   * @return DiskBalancerDataNode
+   */
+  private DiskBalancerDataNode
+      getBalancerNodeFromDataNode(DatanodeInfo nodeInfo) {
+    Preconditions.checkNotNull(nodeInfo);
+    DiskBalancerDataNode dbDataNode = new DiskBalancerDataNode(nodeInfo
+        .getDatanodeUuid());
+    dbDataNode.setDataNodeIP(nodeInfo.getIpAddr());
+    dbDataNode.setDataNodeName(nodeInfo.getHostName());
+    dbDataNode.setDataNodePort(nodeInfo.getIpcPort());
+    return dbDataNode;
+  }
+
+  /**
+   * Reads the relevant fields from each storage volume and populate the
+   * DiskBalancer Node.
+   *
+   * @param node    - Disk Balancer Node
+   * @param reports - Array of StorageReport
+   */
+  private void getVolumeInfoFromStorageReports(DiskBalancerDataNode node,
+                                               StorageReport[] reports)
+      throws Exception {
+    Preconditions.checkNotNull(node);
+    Preconditions.checkNotNull(reports);
+    for (StorageReport report : reports) {
+      DatanodeStorage storage = report.getStorage();
+      DiskBalancerVolume volume = new DiskBalancerVolume();
+      volume.setCapacity(report.getCapacity());
+      volume.setFailed(report.isFailed());
+      volume.setUsed(report.getDfsUsed());
+
+      // TODO : Should we do BlockPool level balancing at all ?
+      // Does it make sense ? Balancer does do that. Right now
+      // we only deal with volumes and not blockPools
+
+      volume.setUsed(report.getDfsUsed());
+
+      volume.setUuid(storage.getStorageID());
+
+      // we will skip this volume for disk balancer if
+      // it is read-only since we will not be able to delete
+      // or if it is already failed.
+      volume.setSkip((storage.getState() == DatanodeStorage.State
+          .READ_ONLY_SHARED) || report.isFailed());
+      volume.setStorageType(storage.getStorageType().name());
+      volume.setIsTransient(storage.getStorageType().isTransient());
+      //volume.setPath(storage.getVolumePath());
+      node.addVolume(volume);
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
new file mode 100644
index 0000000..bf5aebb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
@@ -0,0 +1,77 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+
+/**
+ * A connector that understands JSON data cluster models.
+ */
+public class JsonNodeConnector implements ClusterConnector {
+  static final Log LOG = LogFactory.getLog(JsonNodeConnector.class);
+  private final URL clusterURI;
+
+  /**
+   * Constructs a JsonNodeConnector.
+   * @param clusterURI - A file URL that contains cluster information.
+   */
+  public JsonNodeConnector(URL clusterURI) {
+    this.clusterURI = clusterURI;
+  }
+
+  /**
+   * getNodes function connects to a cluster definition file
+   * and returns nodes defined in that file.
+   *
+   * @return Array of DiskBalancerDataNodes
+   */
+  @Override
+  public List<DiskBalancerDataNode> getNodes() throws Exception {
+    Preconditions.checkNotNull(this.clusterURI);
+    String dataFilePath = this.clusterURI.getPath();
+    LOG.info("Reading cluster info from file : " + dataFilePath);
+    ObjectMapper mapper = new ObjectMapper();
+    DiskBalancerCluster cluster =
+        mapper.readValue(new File(dataFilePath), DiskBalancerCluster.class);
+    String message = String.format("Found %d node(s)",
+        cluster.getNodes().size());
+    LOG.info(message);
+    return cluster.getNodes();
+  }
+
+  /**
+   * Returns info about the connector.
+   *
+   * @return String.
+   */
+  @Override
+  public String getConnectorInfo() {
+    return "Json Cluster Connector : Connects to a JSON file that describes a" +
+        " cluster : " + clusterURI.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
index 8164804..b4b4437 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
@@ -21,7 +21,15 @@ package org.apache.hadoop.hdfs.server.diskbalancer.connectors;
  * Connectors package is a set of logical connectors that connect
  * to various data sources to read the hadoop cluster information.
  *
- * We currently have 1 connector in this package. it is
+ * We currently have 3 connectors in this package. They are
+ *
+ * DBNameNodeConnector - This uses the connector from the original
+ * balancer package to connect to a real hadoop cluster.
+ *
+ * JsonNodeConnector - This connects to a file and reads the data about a
+ * cluster. We can generate a cluster json from a real cluster using
+ * the diskBalancer tool or hand-craft it. There are some sample Json files
+ * checked in under test/resources/diskBalancer directory.
  *
  * NullConnector - This is an in-memory connector that is useful in testing.
  * we can crate dataNodes on the fly and attach to this connector and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30c6ebd6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
new file mode 100644
index 0000000..bec488e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
@@ -0,0 +1,82 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestConnectors {
+  private MiniDFSCluster cluster;
+  final int numDatanodes = 3;
+  final int volumeCount = 2; // default volumes in MiniDFSCluster.
+  Configuration conf;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes).build();
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void TestNameNodeConnector() throws Exception {
+    cluster.waitActive();
+    ClusterConnector nameNodeConnector =
+        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
+        (nameNodeConnector);
+    diskBalancerCluster.readClusterInfo();
+    Assert.assertEquals("Expected number of Datanodes not found.",
+        numDatanodes, diskBalancerCluster.getNodes().size());
+    Assert.assertEquals("Expected number of volumes not found.",
+        volumeCount, diskBalancerCluster.getNodes().get(0).getVolumeCount());
+  }
+
+  @Test
+  public void TestJsonConnector() throws Exception {
+    cluster.waitActive();
+    ClusterConnector nameNodeConnector =
+        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
+        (nameNodeConnector);
+    diskBalancerCluster.readClusterInfo();
+    String diskBalancerJson = diskBalancerCluster.toJson();
+    DiskBalancerCluster serializedCluster = DiskBalancerCluster.parseJson
+        (diskBalancerJson);
+    Assert.assertEquals("Parsed cluster is not equal to persisted info.",
+        diskBalancerCluster.getNodes().size(), serializedCluster.getNodes()
+            .size());
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/49] hadoop git commit: HDFS-10551. o.a.h.h.s.diskbalancer.command.Command does not actually verify options as expected. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10551. o.a.h.h.s.diskbalancer.command.Command does not actually verify options as expected. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e8de2818
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e8de2818
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e8de2818

Branch: refs/heads/trunk
Commit: e8de28181a3ed0053d5cd5f196434739880ee978
Parents: 66fa34c
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Jun 22 17:29:34 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:02 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/Command.java    |   9 +-
 .../diskbalancer/command/ExecuteCommand.java    |   1 -
 .../diskbalancer/command/HelpCommand.java       |   1 +
 .../diskbalancer/command/PlanCommand.java       |   1 +
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  |  14 ++-
 .../src/main/resources/hdfs-default.xml         |  40 ++++++++
 .../command/TestDiskBalancerCommand.java        | 100 ++++++++++++++++---
 7 files changed, 141 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 3ea1b03..de77365 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -82,8 +82,6 @@ public abstract class Command extends Configured {
   public Command(Configuration conf) {
     super(conf);
     // These arguments are valid for all commands.
-    addValidCommandParameters(DiskBalancer.HELP, "Help for this command");
-    addValidCommandParameters("arg", "");
     topNodes = 0;
   }
 
@@ -248,12 +246,13 @@ public abstract class Command extends Configured {
     Iterator<Option> iter = cmd.iterator();
     while (iter.hasNext()) {
       Option opt = iter.next();
-      if (!validArgs.containsKey(opt.getArgName())) {
+
+      if (!validArgs.containsKey(opt.getLongOpt())) {
         String errMessage = String
             .format("%nInvalid argument found for command %s : %s%n",
-                commandName, opt.getArgName());
+                commandName, opt.getLongOpt());
         StringBuilder validArguments = new StringBuilder();
-        validArguments.append("Valid arguments are : %n");
+        validArguments.append(String.format("Valid arguments are : %n"));
         for (Map.Entry<String, String> args : validArgs.entrySet()) {
           String key = args.getKey();
           String desc = args.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index 5fd1f0a..91bce37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -47,7 +47,6 @@ public class ExecuteCommand extends Command {
   public ExecuteCommand(Configuration conf) {
     super(conf);
     addValidCommandParameters(DiskBalancer.EXECUTE, "Executes a given plan.");
-    addValidCommandParameters(DiskBalancer.NODE, "Name of the target node.");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
index 205df3d..3c2fd0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
@@ -37,6 +37,7 @@ public class HelpCommand extends Command {
    */
   public HelpCommand(Configuration conf) {
     super(conf);
+    addValidCommandParameters(DiskBalancer.HELP, "Help Command");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 20b4c6f..54a63ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -74,6 +74,7 @@ public class PlanCommand extends Command {
         "between 2 disks");
     addValidCommandParameters(DiskBalancer.VERBOSE, "Run plan command in " +
         "verbose mode.");
+    addValidCommandParameters(DiskBalancer.PLAN, "Plan Command");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 612aa2c..70912d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -399,13 +399,19 @@ public class DiskBalancer extends Configured implements Tool {
     getReportOptions().addOption(report);
     opt.addOption(report);
 
-    Option top = new Option(TOP, true,
-        "specify the number of nodes to be listed which has data imbalance.");
+    Option top = OptionBuilder.withLongOpt(TOP)
+        .hasArg()
+        .withDescription("specify the number of nodes to be listed which has" +
+            " data imbalance.")
+        .create();
     getReportOptions().addOption(top);
     opt.addOption(top);
 
-    Option node = new Option(NODE, true,
-        "Datanode address, it can be DataNodeID, IP or hostname.");
+    Option node =  OptionBuilder.withLongOpt(NODE)
+        .hasArg()
+        .withDescription("Datanode address, " +
+            "it can be DataNodeID, IP or hostname.")
+        .create();
     getReportOptions().addOption(node);
     opt.addOption(node);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index fc2f942..856e6b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4086,4 +4086,44 @@
     Truststore password for HTTPS SSL configuration
   </description>
 </property>
+
+<!--Disk baalncer properties-->
+  <property>
+    <name>dfs.disk.balancer.max.disk.throughputInMBperSec</name>
+    <value>10</value>
+    <description>Maximum disk bandwidth used by diskbalancer
+      during read from a source disk. The unit is MB/sec.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.disk.balancer.block.tolerance.percent</name>
+    <value>10</value>
+    <description>
+      When a disk balancer copy operation is proceeding, the datanode is still
+      active. So it might not be possible to move the exactly specified
+      amount of data. So tolerance allows us to define a percentage which
+      defines a good enough move.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.disk.balancer.max.disk.errors</name>
+    <value>5</value>
+    <description>
+      During a block move from a source to destination disk, we might
+      encounter various errors. This defines how many errors we can tolerate
+      before we declare a move between 2 disks (or a step) has failed.
+    </description>
+  </property>
+
+
+  <property>
+    <name>dfs.disk.balancer.enabled</name>
+    <value>false</value>
+    <description>
+        This enables the diskbalancer feature on a cluster. By default, disk
+      balancer is disabled.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8de2818/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index c1c137d..ceb762f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -5,9 +5,9 @@
  * 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/>
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ * <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
@@ -44,16 +44,27 @@ import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
+import static org.apache.hadoop.hdfs.tools.DiskBalancer.CANCEL;
+import static org.apache.hadoop.hdfs.tools.DiskBalancer.HELP;
+import static org.apache.hadoop.hdfs.tools.DiskBalancer.NODE;
+import static org.apache.hadoop.hdfs.tools.DiskBalancer.PLAN;
+import static org.apache.hadoop.hdfs.tools.DiskBalancer.QUERY;
+
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
 /**
  * Tests various CLI commands of DiskBalancer.
  */
 public class TestDiskBalancerCommand {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
   private MiniDFSCluster cluster;
   private URI clusterJson;
+  private Configuration conf = new HdfsConfiguration();
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
         .storagesPerDatanode(2).build();
@@ -73,7 +84,7 @@ public class TestDiskBalancerCommand {
   }
 
   /* test basic report */
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReportSimple() throws Exception {
     final String cmdLine = "hdfs diskbalancer -report";
     final List<String> outputs = runCommand(cmdLine);
@@ -101,7 +112,7 @@ public class TestDiskBalancerCommand {
   }
 
   /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReportLessThanTotal() throws Exception {
     final String cmdLine = "hdfs diskbalancer -report -top 32";
     final List<String> outputs = runCommand(cmdLine);
@@ -124,7 +135,7 @@ public class TestDiskBalancerCommand {
   }
 
   /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReportMoreThanTotal() throws Exception {
     final String cmdLine = "hdfs diskbalancer -report -top 128";
     final List<String> outputs = runCommand(cmdLine);
@@ -148,7 +159,7 @@ public class TestDiskBalancerCommand {
   }
 
   /* test invalid top limit, e.g., -report -top xx */
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReportInvalidTopLimit() throws Exception {
     final String cmdLine = "hdfs diskbalancer -report -top xx";
     final List<String> outputs = runCommand(cmdLine);
@@ -174,10 +185,10 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
   }
 
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReportNode() throws Exception {
     final String cmdLine =
-            "hdfs diskbalancer -report -node " +
+        "hdfs diskbalancer -report -node " +
             "a87654a9-54c7-4693-8dd9-c9c7021dc340";
     final List<String> outputs = runCommand(cmdLine);
 
@@ -249,11 +260,8 @@ public class TestDiskBalancerCommand {
             containsString("0.25 free: 490407853993/2000000000000"))));
   }
 
-  @Test(timeout=60000)
+  @Test(timeout = 60000)
   public void testReadClusterFromJson() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
-
     ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
         conf);
     DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(
@@ -262,10 +270,72 @@ public class TestDiskBalancerCommand {
     assertEquals(64, diskBalancerCluster.getNodes().size());
   }
 
-  private List<String> runCommand(final String cmdLine) throws Exception {
+  /* test -plan  DataNodeID */
+  @Test(timeout = 60000)
+  public void testPlanNode() throws Exception {
+    final String planArg = String.format("-%s %s", PLAN,
+        cluster.getDataNodes().get(0).getDatanodeUuid());
+
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer %s", planArg);
+    runCommand(cmdLine);
+  }
+
+  /* Test that illegal arguments are handled correctly*/
+  @Test(timeout = 60000)
+  public void testIllegalArgument() throws Exception {
+    final String planArg = String.format("-%s %s", PLAN,
+        "a87654a9-54c7-4693-8dd9-c9c7021dc340");
+
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer %s -report", planArg);
+    // -plan and -report cannot be used together.
+    // tests the validate command line arguments function.
+    thrown.expect(java.lang.IllegalArgumentException.class);
+    runCommand(cmdLine);
+  }
+
+  @Test(timeout = 60000)
+  public void testCancelCommand() throws Exception {
+    final String cancelArg = String.format("-%s %s", CANCEL, "nosuchplan");
+    final String nodeArg = String.format("-%s %s", NODE,
+        cluster.getDataNodes().get(0).getDatanodeUuid());
 
+    // Port:Host format is expected. So cancel command will throw.
+    thrown.expect(java.lang.IllegalArgumentException.class);
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer  %s %s", cancelArg, nodeArg);
+    runCommand(cmdLine);
+  }
+
+  /*
+   Makes an invalid query attempt to non-existent Datanode.
+   */
+  @Test(timeout = 60000)
+  public void testQueryCommand() throws Exception {
+    final String queryArg = String.format("-%s %s", QUERY,
+        cluster.getDataNodes().get(0).getDatanodeUuid());
+    thrown.expect(java.net.UnknownHostException.class);
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer %s", queryArg);
+    runCommand(cmdLine);
+  }
+
+  @Test(timeout = 60000)
+  public void testHelpCommand() throws Exception {
+    final String helpArg = String.format("-%s", HELP);
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer %s", helpArg);
+    runCommand(cmdLine);
+  }
+
+  private List<String> runCommand(final String cmdLine) throws Exception {
     String[] cmds = StringUtils.split(cmdLine, ' ');
-    Configuration conf = new HdfsConfiguration();
     org.apache.hadoop.hdfs.tools.DiskBalancer db =
         new org.apache.hadoop.hdfs.tools.DiskBalancer(conf);
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/49] hadoop git commit: HDFS-9461. DiskBalancer: Add Report Command. Contributed by Xiaobing Zhou.

Posted by ar...@apache.org.
HDFS-9461. DiskBalancer: Add Report Command. Contributed by Xiaobing Zhou.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b502102b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b502102b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b502102b

Branch: refs/heads/trunk
Commit: b502102bb1a1f416f43dd1227886c57ccad70fcc
Parents: 121142c
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jun 10 21:15:54 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/Command.java    |   67 +
 .../diskbalancer/command/ReportCommand.java     |  197 +
 .../datamodel/DiskBalancerVolume.java           |   30 +
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  |   61 +-
 .../command/TestDiskBalancerCommand.java        |  299 +
 .../diskBalancer/data-cluster-64node-3disk.json | 9484 ++++++++++++++++++
 6 files changed, 10136 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 94a21d1..bbf91ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -70,6 +72,7 @@ public abstract class Command extends Configured {
   private URI clusterURI;
   private FileSystem fs = null;
   private DiskBalancerCluster cluster = null;
+  private int topNodes;
 
   private static final Path DEFAULT_LOG_DIR = new Path("/system/diskbalancer");
 
@@ -83,6 +86,7 @@ public abstract class Command extends Configured {
     // These arguments are valid for all commands.
     addValidCommandParameters(DiskBalancer.HELP, "Help for this command");
     addValidCommandParameters("arg", "");
+    topNodes = 0;
   }
 
   /**
@@ -391,4 +395,67 @@ public abstract class Command extends Configured {
   protected DiskBalancerCluster getCluster() {
     return cluster;
   }
+
+  /**
+   * returns default top number of nodes.
+   * @return default top number of nodes.
+   */
+  protected int getDefaultTop() {
+    return DiskBalancer.DEFAULT_TOP;
+  }
+
+  /**
+   * Put output line to log and string buffer.
+   * */
+  protected void recordOutput(final StrBuilder result,
+      final String outputLine) {
+    LOG.info(outputLine);
+    result.appendln(outputLine);
+  }
+
+  /**
+   * Parse top number of nodes to be processed.
+   * @return top number of nodes to be processed.
+   */
+  protected int parseTopNodes(final CommandLine cmd, final StrBuilder result) {
+    String outputLine = "";
+    int nodes = 0;
+    final String topVal = cmd.getOptionValue(DiskBalancer.TOP);
+    if (StringUtils.isBlank(topVal)) {
+      outputLine = String.format(
+          "No top limit specified, using default top value %d.",
+          getDefaultTop());
+      LOG.info(outputLine);
+      result.appendln(outputLine);
+      nodes = getDefaultTop();
+    } else {
+      try {
+        nodes = Integer.parseInt(topVal);
+      } catch (NumberFormatException nfe) {
+        outputLine = String.format(
+            "Top limit input is not numeric, using default top value %d.",
+            getDefaultTop());
+        LOG.info(outputLine);
+        result.appendln(outputLine);
+        nodes = getDefaultTop();
+      }
+    }
+
+    return Math.min(nodes, cluster.getNodes().size());
+  }
+
+  /**
+   * Set top number of nodes to be processed.
+   * */
+  public void setTopNodes(int topNodes) {
+    this.topNodes = topNodes;
+  }
+
+  /**
+   * Get top number of nodes to be processed.
+   * @return top number of nodes to be processed.
+   * */
+  public int getTopNodes() {
+    return topNodes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
new file mode 100644
index 0000000..acf9ff2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -0,0 +1,197 @@
+/**
+ * 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.hdfs.server.diskbalancer.command;
+
+import java.io.PrintStream;
+import java.util.Collections;
+import java.util.ListIterator;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.text.StrBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Executes the report command.
+ *
+ * This command will report volume information for a specific DataNode or top X
+ * DataNode(s) benefiting from running DiskBalancer.
+ *
+ * This is done by reading the cluster info, sorting the DiskbalancerNodes by
+ * their NodeDataDensity and printing out the info.
+ */
+public class ReportCommand extends Command {
+
+  private PrintStream out;
+
+  public ReportCommand(Configuration conf, final PrintStream out) {
+    super(conf);
+    this.out = out;
+
+    addValidCommandParameters(DiskBalancer.REPORT,
+        "Report volume information of nodes.");
+
+    String desc = String.format(
+        "Top number of nodes to be processed. Default: %d", getDefaultTop());
+    addValidCommandParameters(DiskBalancer.TOP, desc);
+
+    desc = String.format("Print out volume information for a DataNode.");
+    addValidCommandParameters(DiskBalancer.NODE, desc);
+  }
+
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    StrBuilder result = new StrBuilder();
+    String outputLine = "Processing report command";
+    recordOutput(result, outputLine);
+
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.REPORT));
+    verifyCommandOptions(DiskBalancer.REPORT, cmd);
+    readClusterInfo(cmd);
+
+    final String nodeFormat =
+        "%d/%d %s[%s:%d] - <%s>: %d volumes with node data density %.2f.";
+    final String nodeFormatWithoutSequence =
+        "%s[%s:%d] - <%s>: %d volumes with node data density %.2f.";
+    final String volumeFormat =
+        "[%s: volume-%s] - %.2f used: %d/%d, %.2f free: %d/%d, "
+        + "isFailed: %s, isReadOnly: %s, isSkip: %s, isTransient: %s.";
+
+    if (cmd.hasOption(DiskBalancer.NODE)) {
+      /*
+       * Reporting volume information for a specific DataNode
+       */
+      handleNodeReport(cmd, result, nodeFormatWithoutSequence, volumeFormat);
+
+    } else { // handle TOP
+      /*
+       * Reporting volume information for top X DataNode(s)
+       */
+      handleTopReport(cmd, result, nodeFormat);
+    }
+
+    out.println(result.toString());
+  }
+
+  private void handleTopReport(final CommandLine cmd, final StrBuilder result,
+      final String nodeFormat) {
+    Collections.sort(getCluster().getNodes(), Collections.reverseOrder());
+
+    /* extract value that identifies top X DataNode(s) */
+    setTopNodes(parseTopNodes(cmd, result));
+
+    /*
+     * Reporting volume information of top X DataNode(s) in summary
+     */
+    final String outputLine = String.format(
+        "Reporting top %d DataNode(s) benefiting from running DiskBalancer.",
+        getTopNodes());
+    recordOutput(result, outputLine);
+
+    ListIterator<DiskBalancerDataNode> li = getCluster().getNodes()
+        .listIterator();
+
+    for (int i = 0; i < getTopNodes() && li.hasNext(); i++) {
+      DiskBalancerDataNode dbdn = li.next();
+      result.appendln(String.format(nodeFormat,
+          i+1,
+          getTopNodes(),
+          dbdn.getDataNodeName(),
+          dbdn.getDataNodeIP(),
+          dbdn.getDataNodePort(),
+          dbdn.getDataNodeUUID(),
+          dbdn.getVolumeCount(),
+          dbdn.getNodeDataDensity()));
+    }
+  }
+
+  private void handleNodeReport(final CommandLine cmd, StrBuilder result,
+      final String nodeFormat, final String volumeFormat) {
+    String outputLine = "";
+    /*
+     * get value that identifies a DataNode from command line, it could be UUID,
+     * IP address or host name.
+     */
+    final String nodeVal = cmd.getOptionValue(DiskBalancer.NODE);
+
+    if (StringUtils.isBlank(nodeVal)) {
+      outputLine = "The value for '-node' is neither specified or empty.";
+      recordOutput(result, outputLine);
+    } else {
+      /*
+       * Reporting volume information for a specific DataNode
+       */
+      outputLine = String.format(
+          "Reporting volume information for DataNode '%s'.", nodeVal);
+      recordOutput(result, outputLine);
+
+      final String trueStr = "True";
+      final String falseStr = "False";
+      DiskBalancerDataNode dbdn = getNode(nodeVal);
+
+      if (dbdn == null) {
+        outputLine = String.format(
+            "Can't find a DataNode that matches '%s'.", nodeVal);
+        recordOutput(result, outputLine);
+      } else {
+        result.appendln(String.format(nodeFormat,
+            dbdn.getDataNodeName(),
+            dbdn.getDataNodeIP(),
+            dbdn.getDataNodePort(),
+            dbdn.getDataNodeUUID(),
+            dbdn.getVolumeCount(),
+            dbdn.getNodeDataDensity()));
+
+        for (DiskBalancerVolumeSet vset : dbdn.getVolumeSets().values()) {
+          for (DiskBalancerVolume vol : vset.getVolumes()) {
+            result.appendln(String.format(volumeFormat,
+                vol.getStorageType(),
+                vol.getPath(),
+                vol.getUsedRatio(),
+                vol.getUsed(),
+                vol.getCapacity(),
+                vol.getFreeRatio(),
+                vol.getFreeSpace(),
+                vol.getCapacity(),
+                vol.isFailed() ? trueStr : falseStr,
+                vol.isReadOnly() ? trueStr : falseStr,
+                vol.isSkip() ? trueStr : falseStr,
+                vol.isTransient() ? trueStr : falseStr));
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  protected String getHelp() {
+    return "Report volume information for a specific DataNode or top X "
+        + "one(s) benefiting from running DiskBalancer, "
+        + "top defaults to " + getDefaultTop() + ". E.g.:\n"
+        + "hdfs diskbalancer -uri http://namenode.uri -report\n"
+        + "hdfs diskbalancer -uri http://namenode.uri -report -top 5\n"
+        + "hdfs diskbalancer -uri http://namenode.uri -report "
+        + "-node {DataNodeID | IP | Hostname}";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
index 2a39609..a6a8bdc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
@@ -140,6 +140,36 @@ public class DiskBalancerVolume {
   }
 
   /**
+   * Get free space of the volume.
+   *
+   * @return long
+   */
+  @JsonIgnore
+  public long getFreeSpace() {
+    return getCapacity() - getUsed();
+  }
+
+  /**
+   * Get ratio between used space and capacity.
+   *
+   * @return double
+   */
+  @JsonIgnore
+  public double getUsedRatio() {
+    return (1.0 * getUsed()) / getCapacity();
+  }
+
+  /**
+   * Get ratio between free space and capacity.
+   *
+   * @return double
+   */
+  @JsonIgnore
+  public double getFreeRatio() {
+    return (1.0 * getFreeSpace()) / getCapacity();
+  }
+
+  /**
    * Sets the capacity of this volume.
    *
    * @param totalCapacity long

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index dde2ce4..1251e96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -29,12 +29,14 @@ import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.QueryCommand;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.ReportCommand;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.PrintStream;
 import java.net.URISyntaxException;
 
 /**
@@ -94,6 +96,22 @@ public class DiskBalancer extends Configured implements Tool {
    */
   public static final String EXECUTE = "execute";
   /**
+   * The report command prints out a disk fragmentation report about the data
+   * cluster. By default it prints the DEFAULT_TOP machines names with high
+   * nodeDataDensity {DiskBalancerDataNode#getNodeDataDensity} values. This
+   * means that these are the nodes that deviates from the ideal data
+   * distribution.
+   */
+  public static final String REPORT = "report";
+  /**
+   * specify top number of nodes to be processed.
+   */
+  public static final String TOP = "top";
+  /**
+   * specify default top number of nodes to be processed.
+   */
+  public static final int DEFAULT_TOP = 100;
+  /**
    * Name or address of the node to execute against.
    */
   public static final String NODE = "node";
@@ -157,9 +175,21 @@ public class DiskBalancer extends Configured implements Tool {
    */
   @Override
   public int run(String[] args) throws Exception {
+    return run(args, System.out);
+  }
+
+  /**
+   * Execute the command with the given arguments.
+   *
+   * @param args command specific arguments.
+   * @param out the output stream used for printing
+   * @return exit code.
+   * @throws Exception
+   */
+  public int run(String[] args, final PrintStream out) throws Exception {
     Options opts = getOpts();
     CommandLine cmd = parseArgs(args, opts);
-    return dispatch(cmd, opts);
+    return dispatch(cmd, opts, out);
   }
 
   /**
@@ -173,6 +203,7 @@ public class DiskBalancer extends Configured implements Tool {
     addExecuteCommands(opts);
     addQueryCommands(opts);
     addCancelCommands(opts);
+    addReportCommands(opts);
     return opts;
   }
 
@@ -256,6 +287,26 @@ public class DiskBalancer extends Configured implements Tool {
   }
 
   /**
+   * Adds report command options.
+   * @param opt Options
+   */
+  private void addReportCommands(Options opt) {
+    Option report = new Option(REPORT, false,
+        "Report volume information of DataNode(s)"
+            + " benefiting from running DiskBalancer. "
+            + "-report [top -X] | [-node {DataNodeID | IP | Hostname}].");
+    opt.addOption(report);
+
+    Option top = new Option(TOP, true,
+        "specify the top number of nodes to be processed.");
+    opt.addOption(top);
+
+    Option node = new Option(NODE, true,
+        "Name of the datanode in the format of DataNodeID, IP or hostname.");
+    opt.addOption(node);
+  }
+
+  /**
    * This function parses all command line arguments and returns the appropriate
    * values.
    *
@@ -272,10 +323,12 @@ public class DiskBalancer extends Configured implements Tool {
    * Dispatches calls to the right command Handler classes.
    *
    * @param cmd - CommandLine
+   * @param opts options of command line
+   * @param out the output stream used for printing
    * @throws IOException
    * @throws URISyntaxException
    */
-  private int dispatch(CommandLine cmd, Options opts)
+  private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
       throws IOException, URISyntaxException {
     Command currentCommand = null;
 
@@ -297,6 +350,10 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new CancelCommand(getConf());
       }
 
+      if (cmd.hasOption(DiskBalancer.REPORT)) {
+        currentCommand = new ReportCommand(getConf(), out);
+      }
+
       if(currentCommand == null) {
         HelpFormatter helpFormatter = new HelpFormatter();
         helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b502102b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
new file mode 100644
index 0000000..57e59f6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -0,0 +1,299 @@
+/**
+ * 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.hdfs.server.diskbalancer.command;
+
+import static org.hamcrest.CoreMatchers.allOf;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.net.URI;
+import java.util.List;
+import java.util.Scanner;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Tests various CLI commands of DiskBalancer.
+ */
+public class TestDiskBalancerCommand {
+  private MiniDFSCluster cluster;
+  private URI clusterJson;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .storagesPerDatanode(2).build();
+    cluster.waitActive();
+
+    clusterJson = getClass().getResource(
+        "/diskBalancer/data-cluster-64node-3disk.json").toURI();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      // Just make sure we can shutdown datanodes.
+      cluster.getDataNodes().get(0).shutdown();
+      cluster.shutdown();
+    }
+  }
+
+  private void testReportSimple() throws Exception {
+    final String cmdLine = String.format("hdfs diskbalancer -uri %s -report",
+        clusterJson.toString());
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(containsString("No top limit specified"),
+            containsString("using default top value"), containsString("100"))));
+    assertThat(
+        outputs.get(2),
+        is(allOf(
+            containsString("Reporting top"),
+            containsString("64"),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+    assertThat(
+        outputs.get(32),
+        is(allOf(containsString("30/64 null[null:0]"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"),
+            containsString("9 volumes with node data density 1.97"))));
+
+  }
+
+  private void testReportLessThanTotal() throws Exception {
+    final String cmdLine = String.format(
+        "hdfs diskbalancer -uri %s -report -top 32", clusterJson.toString());
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(
+            containsString("Reporting top"),
+            containsString("32"),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+    assertThat(
+        outputs.get(31),
+        is(allOf(containsString("30/32 null[null:0]"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"),
+            containsString("9 volumes with node data density 1.97"))));
+  }
+
+  private void testReportMoreThanTotal() throws Exception {
+    final String cmdLine = String.format(
+        "hdfs diskbalancer -uri %s -report -top 128", clusterJson.toString());
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(
+            containsString("Reporting top"),
+            containsString("64"),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+    assertThat(
+        outputs.get(31),
+        is(allOf(containsString("30/64 null[null:0]"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"),
+            containsString("9 volumes with node data density 1.97"))));
+
+  }
+
+  private void testReportInvalidTopLimit() throws Exception {
+    final String cmdLine = String.format(
+        "hdfs diskbalancer -uri %s -report -top xx", clusterJson.toString());
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(containsString("Top limit input is not numeric"),
+            containsString("using default top value"), containsString("100"))));
+    assertThat(
+        outputs.get(2),
+        is(allOf(
+            containsString("Reporting top"),
+            containsString("64"),
+            containsString(
+                "DataNode(s) benefiting from running DiskBalancer"))));
+    assertThat(
+        outputs.get(32),
+        is(allOf(containsString("30/64 null[null:0]"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"),
+            containsString("9 volumes with node data density 1.97"))));
+  }
+
+  private void testReportNode() throws Exception {
+    final String cmdLine = String
+        .format(
+            "hdfs diskbalancer -uri %s -report -node "
+                + "a87654a9-54c7-4693-8dd9-c9c7021dc340",
+            clusterJson.toString());
+    final List<String> outputs = runCommand(cmdLine);
+
+    assertThat(
+        outputs.get(0),
+        containsString("Processing report command"));
+    assertThat(
+        outputs.get(1),
+        is(allOf(containsString("Reporting volume information for DataNode"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"))));
+    assertThat(
+        outputs.get(2),
+        is(allOf(containsString("null[null:0]"),
+            containsString("a87654a9-54c7-4693-8dd9-c9c7021dc340"),
+            containsString("9 volumes with node data density 1.97"))));
+    assertThat(
+        outputs.get(3),
+        is(allOf(containsString("DISK"),
+            containsString("/tmp/disk/xx3j3ph3zd"),
+            containsString("0.72 used: 289544224916/400000000000"),
+            containsString("0.28 free: 110455775084/400000000000"))));
+    assertThat(
+        outputs.get(4),
+        is(allOf(containsString("DISK"),
+            containsString("/tmp/disk/Mxfcfmb24Y"),
+            containsString("0.92 used: 733099315216/800000000000"),
+            containsString("0.08 free: 66900684784/800000000000"))));
+    assertThat(
+        outputs.get(5),
+        is(allOf(containsString("DISK"),
+            containsString("DISK"),
+            containsString("/tmp/disk/KmHefYNURo"),
+            containsString("0.20 used: 39160240782/200000000000"),
+            containsString("0.80 free: 160839759218/200000000000"))));
+    assertThat(
+        outputs.get(6),
+        is(allOf(containsString("RAM_DISK"),
+            containsString("/tmp/disk/MXRyYsCz3U"),
+            containsString("0.55 used: 438102096853/800000000000"),
+            containsString("0.45 free: 361897903147/800000000000"))));
+    assertThat(
+        outputs.get(7),
+        is(allOf(containsString("RAM_DISK"),
+            containsString("/tmp/disk/DtmAygEU6f"),
+            containsString("0.34 used: 134602910470/400000000000"),
+            containsString("0.66 free: 265397089530/400000000000"))));
+    assertThat(
+        outputs.get(8),
+        is(allOf(containsString("RAM_DISK"),
+            containsString("/tmp/disk/BoBlQFxhfw"),
+            containsString("0.60 used: 477590453390/800000000000"),
+            containsString("0.40 free: 322409546610/800000000000"))));
+    assertThat(
+        outputs.get(9),
+        is(allOf(containsString("SSD"),
+            containsString("/tmp/disk/BGe09Y77dI"),
+            containsString("0.89 used: 890446265501/1000000000000"),
+            containsString("0.11 free: 109553734499/1000000000000"))));
+    assertThat(
+        outputs.get(10),
+        is(allOf(containsString("SSD"),
+            containsString("/tmp/disk/JX3H8iHggM"),
+            containsString("0.31 used: 2782614512957/9000000000000"),
+            containsString("0.69 free: 6217385487043/9000000000000"))));
+    assertThat(
+        outputs.get(11),
+        is(allOf(containsString("SSD"),
+            containsString("/tmp/disk/uLOYmVZfWV"),
+            containsString("0.75 used: 1509592146007/2000000000000"),
+            containsString("0.25 free: 490407853993/2000000000000"))));
+  }
+
+  @Test(timeout=60000)
+  public void testReportCommmand() throws Exception {
+
+    /* test basic report */
+    testReportSimple();
+
+    /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
+    testReportLessThanTotal();
+
+    /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
+    testReportMoreThanTotal();
+
+    /* test invalid top limit, e.g., -report -top xx */
+    testReportInvalidTopLimit();
+
+    /* test -report -node DataNodeID */
+    testReportNode();
+  }
+
+  @Test
+  public void testReadClusterFromJson() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+
+    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
+        conf);
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(
+        jsonConnector);
+    diskBalancerCluster.readClusterInfo();
+    assertEquals(64, diskBalancerCluster.getNodes().size());
+  }
+
+  private List<String> runCommand(final String cmdLine) throws Exception {
+
+    String[] cmds = StringUtils.split(cmdLine, ' ');
+    Configuration conf = new HdfsConfiguration();
+    org.apache.hadoop.hdfs.tools.DiskBalancer db =
+        new org.apache.hadoop.hdfs.tools.DiskBalancer(conf);
+
+    ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(bufOut);
+    db.run(cmds, out);
+
+    Scanner scanner = new Scanner(bufOut.toString());
+    List<String> outputs = Lists.newArrayList();
+    while (scanner.hasNextLine()) {
+      outputs.add(scanner.nextLine());
+    }
+    return outputs;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/49] hadoop git commit: HDFS-9709. DiskBalancer : Add tests for disk balancer using a Mock Mover class. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9709. DiskBalancer : Add tests for disk balancer using a Mock Mover class. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6c606bf5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6c606bf5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6c606bf5

Branch: refs/heads/trunk
Commit: 6c606bf5c8c1ace381ce73679c2be96d5475ba34
Parents: 9be9703
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Mar 22 16:26:49 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/DiskBalancer.java      |  14 +-
 .../DiskBalancerResultVerifier.java             |  42 ++
 .../diskbalancer/TestDiskBalancerRPC.java       |  39 +-
 .../TestDiskBalancerWithMockMover.java          | 570 +++++++++++++++++++
 4 files changed, 628 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c606bf5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index d1bc1f1..972f0fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -40,7 +40,11 @@ import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
 /**
@@ -106,6 +110,7 @@ public class DiskBalancer {
       this.isDiskBalancerEnabled = false;
       this.currentResult = Result.NO_PLAN;
       if ((this.future != null) && (!this.future.isDone())) {
+        this.currentResult = Result.PLAN_CANCELLED;
         this.blockMover.setExitFlag();
         shutdownExecutor();
       }
@@ -120,9 +125,9 @@ public class DiskBalancer {
   private void shutdownExecutor() {
     scheduler.shutdown();
     try {
-      if(!scheduler.awaitTermination(30, TimeUnit.SECONDS)) {
+      if(!scheduler.awaitTermination(10, TimeUnit.SECONDS)) {
         scheduler.shutdownNow();
-        if (!scheduler.awaitTermination(30, TimeUnit.SECONDS)) {
+        if (!scheduler.awaitTermination(10, TimeUnit.SECONDS)) {
           LOG.error("Disk Balancer : Scheduler did not terminate.");
         }
       }
@@ -218,6 +223,7 @@ public class DiskBalancer {
       if (!this.future.isDone()) {
         this.blockMover.setExitFlag();
         shutdownExecutor();
+        this.currentResult = Result.PLAN_CANCELLED;
       }
     } finally {
       lock.unlock();
@@ -537,7 +543,7 @@ public class DiskBalancer {
   /**
    * Holds references to actual volumes that we will be operating against.
    */
-  static class VolumePair {
+  public static class VolumePair {
     private final FsVolumeSpi source;
     private final FsVolumeSpi dest;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c606bf5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
new file mode 100644
index 0000000..5abb33c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
@@ -0,0 +1,42 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+
+public class DiskBalancerResultVerifier
+    extends TypeSafeMatcher<DiskBalancerException> {
+  private final DiskBalancerException.Result expectedResult;
+
+  DiskBalancerResultVerifier(DiskBalancerException.Result expectedResult) {
+    this.expectedResult = expectedResult;
+  }
+
+  @Override
+  protected boolean matchesSafely(DiskBalancerException exception) {
+    return (this.expectedResult == exception.getResult());
+  }
+
+  @Override
+  public void describeTo(Description description) {
+    description.appendText("expects Result: ")
+        .appendValue(this.expectedResult);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c606bf5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 9cd41c2..a65ed21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -90,8 +90,7 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.INVALID_PLAN_HASH));
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_HASH));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
   }
 
@@ -104,8 +103,7 @@ public class TestDiskBalancerRPC {
     planVersion++;
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.INVALID_PLAN_VERSION));
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_VERSION));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
   }
 
@@ -117,8 +115,7 @@ public class TestDiskBalancerRPC {
     int planVersion = rpcTestHelper.getPlanVersion();
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.INVALID_PLAN));
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, "");
   }
 
@@ -143,8 +140,7 @@ public class TestDiskBalancerRPC {
     planHash = String.valueOf(hashArray);
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.NO_SUCH_PLAN));
+    thrown.expect(new DiskBalancerResultVerifier(Result.NO_SUCH_PLAN));
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
@@ -155,8 +151,7 @@ public class TestDiskBalancerRPC {
     String planHash = "";
     NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.NO_SUCH_PLAN));
+    thrown.expect(new DiskBalancerResultVerifier(Result.NO_SUCH_PLAN));
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
@@ -182,8 +177,7 @@ public class TestDiskBalancerRPC {
     final String invalidSetting = "invalidSetting";
     DataNode dataNode = cluster.getDataNodes().get(dnIndex);
     thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.UNKNOWN_KEY));
+    thrown.expect(new DiskBalancerResultVerifier(Result.UNKNOWN_KEY));
     dataNode.getDiskBalancerSetting(invalidSetting);
   }
 
@@ -274,25 +268,4 @@ public class TestDiskBalancerRPC {
       return this;
     }
   }
-
-  private class ResultVerifier
-      extends TypeSafeMatcher<DiskBalancerException> {
-    private final DiskBalancerException.Result expectedResult;
-
-    ResultVerifier(DiskBalancerException.Result expectedResult){
-      this.expectedResult = expectedResult;
-    }
-
-    @Override
-    protected boolean matchesSafely(DiskBalancerException exception) {
-      return (this.expectedResult == exception.getResult());
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("expects Result: ")
-          .appendValue(this.expectedResult);
-
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c606bf5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
new file mode 100644
index 0000000..ed761ed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -0,0 +1,570 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancer;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkItem;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.apache.hadoop.util.Time;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestDiskBalancerWithMockMover {
+  static final Log LOG = LogFactory.getLog(TestDiskBalancerWithMockMover.class);
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  MiniDFSCluster cluster;
+  String sourceName;
+  String destName;
+  String sourceUUID;
+  String destUUID;
+  String nodeID;
+  DataNode dataNode;
+
+  /**
+   * Checks that we return the right error if diskbalancer is not enabled.
+   */
+  @Test
+  public void testDiskBalancerDisabled() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, false);
+    restartDataNode();
+
+    TestMover blockMover = new TestMover(cluster.getDataNodes()
+        .get(0).getFSDataset());
+
+    DiskBalancer balancer = new DiskBalancerBuilder(conf)
+        .setMover(blockMover)
+        .build();
+
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.DISK_BALANCER_NOT_ENABLED));
+
+    balancer.queryWorkStatus();
+  }
+
+  /**
+   * Checks that Enable flag works correctly.
+   *
+   * @throws DiskBalancerException
+   */
+  @Test
+  public void testDiskBalancerEnabled() throws DiskBalancerException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+
+    TestMover blockMover = new TestMover(cluster.getDataNodes()
+        .get(0).getFSDataset());
+
+    DiskBalancer balancer = new DiskBalancerBuilder(conf)
+        .setMover(blockMover)
+        .build();
+
+    DiskBalancerWorkStatus status = balancer.queryWorkStatus();
+    assertEquals(NO_PLAN, status.getResult());
+
+  }
+
+  private void executeSubmitPlan(NodePlan plan, DiskBalancer balancer,
+                                 int version) throws IOException {
+    String planJson = plan.toJson();
+    String planID = DigestUtils.sha512Hex(planJson);
+    balancer.submitPlan(planID, version, planJson, 10, false);
+  }
+
+  private void executeSubmitPlan(NodePlan plan, DiskBalancer balancer)
+      throws IOException {
+    executeSubmitPlan(plan, balancer, 1);
+  }
+
+  /**
+   * Test a second submit plan fails.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testResubmitDiskBalancerPlan() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    // ask block mover to get stuck in copy block
+    mockMoverHelper.getBlockMover().setSleep();
+    executeSubmitPlan(plan, balancer);
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.PLAN_ALREADY_IN_PROGRESS));
+    executeSubmitPlan(plan, balancer);
+
+    // Not needed but this is the cleanup step.
+    mockMoverHelper.getBlockMover().clearSleep();
+  }
+
+  @Test
+  public void testSubmitDiskBalancerPlan() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    executeSubmitPlan(plan, balancer);
+    int counter = 0;
+    while ((balancer.queryWorkStatus().getResult() != PLAN_DONE) &&
+        (counter < 3)) {
+      Thread.sleep(1000);
+      counter++;
+    }
+
+    // Asserts that submit plan caused an execution in the background.
+    assertTrue(mockMoverHelper.getBlockMover().getRunCount() == 1);
+  }
+
+  @Test
+  public void testSubmitWithOlderPlan() throws Exception {
+    final long MILLISECOND_IN_AN_HOUR = 1000 * 60 * 60L;
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    plan.setTimeStamp(Time.now() - (32 * MILLISECOND_IN_AN_HOUR));
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.OLD_PLAN_SUBMITTED));
+    executeSubmitPlan(plan, balancer);
+  }
+
+  @Test
+  public void testSubmitWithOldInvalidVersion() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.INVALID_PLAN_VERSION));
+
+    // Plan version is invalid -- there is no version 0.
+    executeSubmitPlan(plan, balancer, 0);
+  }
+
+  @Test
+  public void testSubmitWithNullPlan() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+    String planJson = plan.toJson();
+    String planID = DigestUtils.sha512Hex(planJson);
+
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.INVALID_PLAN));
+
+    balancer.submitPlan(planID, 1, null, 10, false);
+  }
+
+  @Test
+  public void testSubmitWithInvalidHash() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+
+    String planJson = plan.toJson();
+    String planID = DigestUtils.sha512Hex(planJson);
+    char repChar = planID.charAt(0);
+    repChar++;
+
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.INVALID_PLAN_HASH));
+    balancer.submitPlan(planID.replace(planID.charAt(0), repChar),
+        1, planJson, 10, false);
+
+  }
+
+  /**
+   * Test Cancel Plan.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testCancelDiskBalancerPlan() throws Exception {
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+
+    // ask block mover to delay execution
+    mockMoverHelper.getBlockMover().setSleep();
+    executeSubmitPlan(plan, balancer);
+
+
+    String planJson = plan.toJson();
+    String planID = DigestUtils.sha512Hex(planJson);
+    balancer.cancelPlan(planID);
+
+    DiskBalancerWorkStatus status = balancer.queryWorkStatus();
+    assertEquals(DiskBalancerWorkStatus.Result.PLAN_CANCELLED,
+        status.getResult());
+
+
+    executeSubmitPlan(plan, balancer);
+
+    // Send a Wrong cancellation request.
+    char first = planID.charAt(0);
+    first++;
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
+        .Result.NO_SUCH_PLAN));
+    balancer.cancelPlan(planID.replace(planID.charAt(0), first));
+
+    // Now cancel the real one
+    balancer.cancelPlan(planID);
+    mockMoverHelper.getBlockMover().clearSleep(); // unblock mover.
+
+    status = balancer.queryWorkStatus();
+    assertEquals(DiskBalancerWorkStatus.Result.PLAN_CANCELLED,
+        status.getResult());
+
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    final int NUM_STORAGES_PER_DN = 2;
+    cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(3)
+        .storagesPerDatanode(NUM_STORAGES_PER_DN)
+        .build();
+    cluster.waitActive();
+    dataNode = cluster.getDataNodes().get(0);
+    FsDatasetSpi.FsVolumeReferences references = dataNode.getFSDataset()
+        .getFsVolumeReferences();
+
+    nodeID = dataNode.getDatanodeUuid();
+    sourceName = references.get(0).getBasePath();
+    destName = references.get(1).getBasePath();
+    sourceUUID = references.get(0).getStorageID();
+    destUUID = references.get(1).getStorageID();
+    references.close();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private void restartDataNode() throws IOException {
+    if (cluster != null) {
+      cluster.restartDataNode(0);
+    }
+  }
+
+  /**
+   * Allows us to control mover class for test purposes.
+   */
+  public static class TestMover implements DiskBalancer.BlockMover {
+
+    private AtomicBoolean shouldRun;
+    private FsDatasetSpi dataset;
+    private Integer runCount;
+    private volatile boolean sleepInCopyBlocks;
+    private long delay;
+
+    public TestMover(FsDatasetSpi dataset) {
+      this.dataset = dataset;
+      this.shouldRun = new AtomicBoolean(false);
+      this.runCount = new Integer(0);
+    }
+
+    public void setSleep() {
+      sleepInCopyBlocks = true;
+    }
+
+    public void clearSleep() {
+      sleepInCopyBlocks = false;
+    }
+
+    public void setDelay(long milliseconds) {
+      this.delay = milliseconds;
+    }
+
+    /**
+     * Copies blocks from a set of volumes.
+     *
+     * @param pair - Source and Destination Volumes.
+     * @param item - Number of bytes to move from volumes.
+     */
+    @Override
+    public void copyBlocks(DiskBalancer.VolumePair pair,
+                           DiskBalancerWorkItem item) {
+
+      try {
+        // get stuck if we are asked to sleep.
+        while (sleepInCopyBlocks) {
+          if (!this.shouldRun()) {
+            return;
+          }
+          Thread.sleep(10);
+        }
+        if (delay > 0) {
+          Thread.sleep(delay);
+        }
+        synchronized (runCount) {
+          if (shouldRun()) {
+            runCount++;
+          }
+        }
+      } catch (InterruptedException ex) {
+        // A failure here can be safely ignored with no impact for tests.
+        LOG.error(ex.toString());
+      }
+    }
+
+    /**
+     * Sets copyblocks into runnable state.
+     */
+    @Override
+    public void setRunnable() {
+      this.shouldRun.set(true);
+    }
+
+    /**
+     * Signals copy block to exit.
+     */
+    @Override
+    public void setExitFlag() {
+      this.shouldRun.set(false);
+    }
+
+    /**
+     * Returns the shouldRun boolean flag.
+     */
+    public boolean shouldRun() {
+      return this.shouldRun.get();
+    }
+
+    @Override
+    public FsDatasetSpi getDataset() {
+      return this.dataset;
+    }
+
+    public int getRunCount() {
+      synchronized (runCount) {
+        LOG.info("Run count : " + runCount.intValue());
+        return runCount.intValue();
+      }
+    }
+  }
+
+  private class MockMoverHelper {
+    private DiskBalancer balancer;
+    private NodePlan plan;
+    private TestMover blockMover;
+
+    public DiskBalancer getBalancer() {
+      return balancer;
+    }
+
+    public NodePlan getPlan() {
+      return plan;
+    }
+
+    public TestMover getBlockMover() {
+      return blockMover;
+    }
+
+    public MockMoverHelper invoke() throws Exception {
+      Configuration conf = new HdfsConfiguration();
+      conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+      restartDataNode();
+
+      blockMover = new TestMover(dataNode.getFSDataset());
+      blockMover.setRunnable();
+
+      balancer = new DiskBalancerBuilder(conf)
+          .setMover(blockMover)
+          .setNodeID(nodeID)
+          .build();
+
+      DiskBalancerCluster diskBalancerCluster = new DiskBalancerClusterBuilder()
+          .setClusterSource("/diskBalancer/data-cluster-3node-3disk.json")
+          .build();
+
+      plan = new PlanBuilder(diskBalancerCluster, nodeID)
+          .setPathMap(sourceName, destName)
+          .setUUIDMap(sourceUUID, destUUID)
+          .build();
+      return this;
+    }
+  }
+
+  private class DiskBalancerBuilder {
+    private TestMover blockMover;
+    private Configuration conf;
+    private String nodeID;
+
+    public DiskBalancerBuilder(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public DiskBalancerBuilder setNodeID(String nodeID) {
+      this.nodeID = nodeID;
+      return this;
+    }
+
+    public DiskBalancerBuilder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    public DiskBalancerBuilder setMover(TestMover mover) {
+      this.blockMover = mover;
+      return this;
+    }
+
+    public DiskBalancerBuilder setRunnable() {
+      blockMover.setRunnable();
+      return this;
+    }
+
+    public DiskBalancer build() {
+      Preconditions.checkNotNull(blockMover);
+      return new DiskBalancer(nodeID, conf,
+          blockMover);
+    }
+  }
+
+  private class DiskBalancerClusterBuilder {
+    private String jsonFilePath;
+    private Configuration conf;
+
+    public DiskBalancerClusterBuilder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    public DiskBalancerClusterBuilder setClusterSource(String jsonFilePath)
+        throws Exception {
+      this.jsonFilePath = jsonFilePath;
+      return this;
+    }
+
+    public DiskBalancerCluster build() throws Exception {
+      DiskBalancerCluster diskBalancerCluster;
+      URI clusterJson = getClass().getResource(jsonFilePath).toURI();
+      ClusterConnector jsonConnector =
+          ConnectorFactory.getCluster(clusterJson, conf);
+      diskBalancerCluster = new DiskBalancerCluster(jsonConnector);
+      diskBalancerCluster.readClusterInfo();
+      diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+      return diskBalancerCluster;
+    }
+  }
+
+  private class PlanBuilder {
+    private String sourcePath;
+    private String destPath;
+    private String sourceUUID;
+    private String destUUID;
+    private DiskBalancerCluster balancerCluster;
+    private String nodeID;
+
+    public PlanBuilder(DiskBalancerCluster balancerCluster, String nodeID) {
+      this.balancerCluster = balancerCluster;
+      this.nodeID = nodeID;
+    }
+
+    public PlanBuilder setPathMap(String sourcePath, String destPath) {
+      this.sourcePath = sourcePath;
+      this.destPath = destPath;
+      return this;
+    }
+
+    public PlanBuilder setUUIDMap(String sourceUUID, String destUUID) {
+      this.sourceUUID = sourceUUID;
+      this.destUUID = destUUID;
+      return this;
+    }
+
+    public NodePlan build() throws Exception {
+      final int dnIndex = 0;
+      Preconditions.checkNotNull(balancerCluster);
+      Preconditions.checkState(nodeID.length() > 0);
+
+      DiskBalancerDataNode node = balancerCluster.getNodes().get(dnIndex);
+      node.setDataNodeUUID(nodeID);
+      GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+      NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+          ());
+      planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
+      setVolumeNames(plan);
+      return plan;
+    }
+
+    private void setVolumeNames(NodePlan plan) {
+      Iterator<Step> iter = plan.getVolumeSetPlans().iterator();
+      while (iter.hasNext()) {
+        MoveStep nextStep = (MoveStep) iter.next();
+        nextStep.getSourceVolume().setPath(sourcePath);
+        nextStep.getSourceVolume().setUuid(sourceUUID);
+        nextStep.getDestinationVolume().setPath(destPath);
+        nextStep.getDestinationVolume().setUuid(destUUID);
+      }
+    }
+
+  }
+}
+


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/49] hadoop git commit: HDFS-10476. DiskBalancer: Plan command output directory should be a sub-directory. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10476. DiskBalancer: Plan command output directory should be a sub-directory. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/47dcb0f9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47dcb0f9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47dcb0f9

Branch: refs/heads/trunk
Commit: 47dcb0f95288a5e6f05480d274f1ebd8cc873ef8
Parents: 06a9799
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 7 10:18:05 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/diskbalancer/command/Command.java  | 4 ++--
 .../src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java | 4 ++++
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47dcb0f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index feee977..fb975a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -163,10 +163,10 @@ public abstract class Command extends Configured {
       if (getClusterURI().getScheme().startsWith("file")) {
         diskBalancerLogs = new Path(
             System.getProperty("user.dir") + DEFAULT_LOG_DIR.toString() +
-                format.format(now));
+                Path.SEPARATOR + format.format(now));
       } else {
         diskBalancerLogs = new Path(DEFAULT_LOG_DIR.toString() +
-            format.format(now));
+            Path.SEPARATOR + format.format(now));
       }
     } else {
       diskBalancerLogs = new Path(path);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47dcb0f9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 4005652..dde2ce4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -216,6 +216,10 @@ public class DiskBalancer extends Configured implements Tool {
     Option help =
         new Option(HELP, true, "Help about a command or this message");
     opt.addOption(help);
+
+    Option verbose = new Option(VERBOSE, "Print out the summary of the plan");
+    opt.addOption(verbose);
+
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/49] hadoop git commit: HDFS-9420. Add DataModels for DiskBalancer. Contributed by Anu Engineer

Posted by ar...@apache.org.
HDFS-9420. Add DataModels for DiskBalancer. Contributed by Anu Engineer


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/91a5c481
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/91a5c481
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/91a5c481

Branch: refs/heads/trunk
Commit: 91a5c4814381a4d4c3ce9b29a1f85299e03be835
Parents: 0b9edf6
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Nov 23 19:07:42 2015 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   5 +
 .../connectors/ClusterConnector.java            |  44 +++
 .../diskbalancer/connectors/package-info.java   |  29 ++
 .../datamodel/DiskBalancerCluster.java          | 249 ++++++++++++++
 .../datamodel/DiskBalancerDataNode.java         | 269 +++++++++++++++
 .../datamodel/DiskBalancerVolume.java           | 330 +++++++++++++++++++
 .../datamodel/DiskBalancerVolumeSet.java        | 325 ++++++++++++++++++
 .../diskbalancer/datamodel/package-info.java    |  31 ++
 .../hdfs/server/diskbalancer/package-info.java  |  36 ++
 .../diskbalancer/DiskBalancerTestUtil.java      | 227 +++++++++++++
 .../server/diskbalancer/TestDataModels.java     | 224 +++++++++++++
 .../diskbalancer/connectors/NullConnector.java  |  59 ++++
 12 files changed, 1828 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
new file mode 100644
index 0000000..5a71032
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -0,0 +1,5 @@
+HDFS-1312 Change Log
+
+  NEW FEATURES
+
+    HDFS-9420. Add DataModels for DiskBalancer. (Anu Engineer via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ClusterConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ClusterConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ClusterConnector.java
new file mode 100644
index 0000000..3dbfec2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ClusterConnector.java
@@ -0,0 +1,44 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+
+import java.util.List;
+
+/**
+ * ClusterConnector interface hides all specifics about how we communicate to
+ * the HDFS cluster. This interface returns data in classes that diskbalancer
+ * understands.
+ */
+public interface ClusterConnector {
+
+  /**
+   * getNodes function returns a list of DiskBalancerDataNodes.
+   *
+   * @return Array of DiskBalancerDataNodes
+   */
+  List<DiskBalancerDataNode> getNodes() throws Exception;
+
+  /**
+   * Returns info about the connector.
+   *
+   * @return String.
+   */
+  String getConnectorInfo();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
new file mode 100644
index 0000000..8164804
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
@@ -0,0 +1,29 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+/**
+ * Connectors package is a set of logical connectors that connect
+ * to various data sources to read the hadoop cluster information.
+ *
+ * We currently have 1 connector in this package. it is
+ *
+ * NullConnector - This is an in-memory connector that is useful in testing.
+ * we can crate dataNodes on the fly and attach to this connector and
+ * ask the diskBalancer Cluster to read data from this source.
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
new file mode 100644
index 0000000..91f7eaa
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -0,0 +1,249 @@
+/**
+ * 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.hdfs.server.diskbalancer.datamodel;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * DiskBalancerCluster represents the nodes that we are working against.
+ * <p/>
+ * Please Note :
+ * <p/>
+ * Semantics of inclusionList and exclusionLists.
+ * <p/>
+ * If a non-empty inclusionList is specified then the diskBalancer assumes that
+ * the user is only interested in processing that list of nodes. This node list
+ * is checked against the exclusionList and only the nodes in inclusionList but
+ * not in exclusionList is processed.
+ * <p/>
+ * if inclusionList is empty, then we assume that all live nodes in the nodes is
+ * to be processed by diskBalancer. In that case diskBalancer will avoid any
+ * nodes specified in the exclusionList but will process all nodes in the
+ * cluster.
+ * <p/>
+ * In other words, an empty inclusionList is means all the nodes otherwise
+ * only a given list is processed and ExclusionList is always honored.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class DiskBalancerCluster {
+
+  static final Log LOG = LogFactory.getLog(DiskBalancerCluster.class);
+  private final Set<String> exclusionList;
+  private final Set<String> inclusionList;
+  private ClusterConnector clusterConnector;
+  private List<DiskBalancerDataNode> nodes;
+  private String outputpath;
+
+  @JsonIgnore
+  private List<DiskBalancerDataNode> nodesToProcess;
+  private float threshold;
+
+  /**
+   * Empty Constructor needed by Jackson.
+   */
+  public DiskBalancerCluster() {
+    nodes = new LinkedList<>();
+    exclusionList = new TreeSet<>();
+    inclusionList = new TreeSet<>();
+
+  }
+
+  /**
+   * Constructs a DiskBalancerCluster.
+   *
+   * @param connector - ClusterConnector
+   * @throws IOException
+   */
+  public DiskBalancerCluster(ClusterConnector connector) throws IOException {
+    Preconditions.checkNotNull(connector);
+    clusterConnector = connector;
+    exclusionList = new TreeSet<>();
+    inclusionList = new TreeSet<>();
+  }
+
+  /**
+   * Parses a Json string and converts to DiskBalancerCluster.
+   *
+   * @param json - Json String
+   * @return DiskBalancerCluster
+   * @throws IOException
+   */
+  public static DiskBalancerCluster parseJson(String json) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(json, DiskBalancerCluster.class);
+  }
+
+  /**
+   * readClusterInfo connects to the cluster and reads the node's data.  This
+   * data is used as basis of rest of computation in DiskBalancerCluster
+   */
+  public void readClusterInfo() throws Exception {
+    Preconditions.checkNotNull(clusterConnector);
+    LOG.info("Using connector : " + clusterConnector.getConnectorInfo());
+    nodes = clusterConnector.getNodes();
+  }
+
+  /**
+   * Gets all DataNodes in the Cluster.
+   *
+   * @return Array of DisKBalancerDataNodes
+   */
+  public List<DiskBalancerDataNode> getNodes() {
+    return nodes;
+  }
+
+  /**
+   * Sets the list of nodes of this cluster.
+   *
+   * @param clusterNodes List of Nodes
+   */
+  public void setNodes(List<DiskBalancerDataNode> clusterNodes) {
+    this.nodes = clusterNodes;
+  }
+
+  /**
+   * Returns the current ExclusionList.
+   *
+   * @return List of Nodes that are excluded from diskBalancer right now.
+   */
+  public Set<String> getExclusionList() {
+    return exclusionList;
+  }
+
+  /**
+   * sets the list of nodes to exclude from process of diskBalancer.
+   *
+   * @param excludedNodes - exclusionList of nodes.
+   */
+  public void setExclusionList(Set<String> excludedNodes) {
+    this.exclusionList.addAll(excludedNodes);
+  }
+
+  /**
+   * Returns the threshold value. This is used for indicating how much skew is
+   * acceptable, This is expressed as a percentage. For example to say 20% skew
+   * between volumes is acceptable set this value to 20.
+   *
+   * @return float
+   */
+  public float getThreshold() {
+    return threshold;
+  }
+
+  /**
+   * Sets the threshold value.
+   *
+   * @param thresholdPercent - float - in percentage
+   */
+  public void setThreshold(float thresholdPercent) {
+    Preconditions.checkState((thresholdPercent >= 0.0f) &&
+        (thresholdPercent <= 100.0f),  "A percentage value expected.");
+    this.threshold = thresholdPercent;
+  }
+
+  /**
+   * Gets the Inclusion list.
+   *
+   * @return List of machine to be processed by diskBalancer.
+   */
+  public Set<String> getInclusionList() {
+    return inclusionList;
+  }
+
+  /**
+   * Sets the inclusionList.
+   *
+   * @param includeNodes - set of machines to be processed by diskBalancer.
+   */
+  public void setInclusionList(Set<String> includeNodes) {
+    this.inclusionList.addAll(includeNodes);
+  }
+
+  /**
+   * returns a serialized json string.
+   *
+   * @return String - json
+   * @throws IOException
+   */
+  public String toJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(this);
+  }
+
+  /**
+   * Returns the Nodes to Process which is the real list of nodes processed by
+   * diskBalancer.
+   *
+   * @return List of DiskBalancerDataNodes
+   */
+  @JsonIgnore
+  public List<DiskBalancerDataNode> getNodesToProcess() {
+    return nodesToProcess;
+  }
+
+  /**
+   * Sets the nodes to process.
+   *
+   * @param dnNodesToProcess - List of DataNodes to process
+   */
+  @JsonIgnore
+  public void setNodesToProcess(List<DiskBalancerDataNode> dnNodesToProcess) {
+    this.nodesToProcess = dnNodesToProcess;
+  }
+
+  /**
+   * Returns th output path for this cluster.
+   */
+  public String getOutput() {
+    return outputpath;
+  }
+
+  /**
+   * Sets the output path for this run.
+   *
+   * @param output - Path
+   */
+  public void setOutput(String output) {
+    this.outputpath = output;
+  }
+
+  /**
+   * Writes a snapshot of the cluster to the specified directory.
+   *
+   * @param snapShotName - name of the snapshot
+   */
+  public void createSnapshot(String snapShotName) throws IOException {
+    String json = this.toJson();
+    File outFile = new File(getOutput() + "/" + snapShotName);
+    FileUtils.writeStringToFile(outFile, json);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
new file mode 100644
index 0000000..87030db
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
@@ -0,0 +1,269 @@
+/**
+ * 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.hdfs.server.diskbalancer.datamodel;
+
+import com.google.common.base.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * DiskBalancerDataNode represents a DataNode that exists in the cluster. It
+ * also contains a metric called nodeDataDensity which allows us to compare
+ * between a set of Nodes.
+ */
+public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
+  private float nodeDataDensity;
+  private Map<String, DiskBalancerVolumeSet> volumeSets;
+  private String dataNodeUUID;
+  private String dataNodeIP;
+  private int dataNodePort;
+  private String dataNodeName;
+  private int volumeCount;
+
+  /**
+   * Constructs an Empty Data Node.
+   */
+  public DiskBalancerDataNode() {
+  }
+
+  /**
+   * Constructs a DataNode.
+   *
+   * @param dataNodeID - Node ID
+   */
+  public DiskBalancerDataNode(String dataNodeID) {
+    this.dataNodeUUID = dataNodeID;
+    volumeSets = new HashMap<>();
+  }
+
+  /**
+   * Returns the IP address of this Node.
+   *
+   * @return IP Address string
+   */
+  public String getDataNodeIP() {
+    return dataNodeIP;
+  }
+
+  /**
+   * Sets the IP address of this Node.
+   *
+   * @param ipaddress - IP Address
+   */
+  public void setDataNodeIP(String ipaddress) {
+    this.dataNodeIP = ipaddress;
+  }
+
+  /**
+   * Returns the Port of this DataNode.
+   *
+   * @return Port Number
+   */
+  public int getDataNodePort() {
+    return dataNodePort;
+  }
+
+  /**
+   * Sets the DataNode Port number.
+   *
+   * @param port - Datanode Port Number
+   */
+  public void setDataNodePort(int port) {
+    this.dataNodePort = port;
+  }
+
+  /**
+   * Get DataNode DNS name.
+   *
+   * @return name of the node
+   */
+  public String getDataNodeName() {
+    return dataNodeName;
+  }
+
+  /**
+   * Sets node's DNS name.
+   *
+   * @param name - Data node name
+   */
+  public void setDataNodeName(String name) {
+    this.dataNodeName = name;
+  }
+
+  /**
+   * Returns the Volume sets on this node.
+   *
+   * @return a Map of VolumeSets
+   */
+  public Map<String, DiskBalancerVolumeSet> getVolumeSets() {
+    return volumeSets;
+  }
+
+  /**
+   * Returns datanode ID.
+   **/
+  public String getDataNodeUUID() {
+    return dataNodeUUID;
+  }
+
+  /**
+   * Sets Datanode UUID.
+   *
+   * @param nodeID - Node ID.
+   */
+  public void setDataNodeUUID(String nodeID) {
+    this.dataNodeUUID = nodeID;
+  }
+
+  /**
+   * Indicates whether some other object is "equal to" this one.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if ((obj == null) || (obj.getClass() != getClass())) {
+      return false;
+    }
+    DiskBalancerDataNode that = (DiskBalancerDataNode) obj;
+    return dataNodeUUID.equals(that.getDataNodeUUID());
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less than,
+   * equal to, or greater than the specified object.
+   *
+   * @param that the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object is
+   * less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(DiskBalancerDataNode that) {
+    Preconditions.checkNotNull(that);
+
+    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+        < 0) {
+      return -1;
+    }
+
+    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+        == 0) {
+      return 0;
+    }
+
+    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+        > 0) {
+      return 1;
+    }
+    return 0;
+  }
+
+  /**
+   * Returns a hash code value for the object. This method is supported for the
+   * benefit of hash tables such as those provided by {@link HashMap}.
+   */
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  /**
+   * returns NodeDataDensity Metric.
+   *
+   * @return float
+   */
+  public float getNodeDataDensity() {
+    return nodeDataDensity;
+  }
+
+  /**
+   * computes nodes data density.
+   * <p/>
+   * This metric allows us to compare different  nodes and how well the data is
+   * spread across a set of volumes inside the node.
+   */
+  public void computeNodeDensity() {
+    float sum = 0;
+    int volcount = 0;
+    for (DiskBalancerVolumeSet vset : volumeSets.values()) {
+      for (DiskBalancerVolume vol : vset.getVolumes()) {
+        sum += Math.abs(vol.getVolumeDataDensity());
+        volcount++;
+      }
+    }
+    nodeDataDensity = sum;
+    this.volumeCount = volcount;
+
+  }
+
+  /**
+   * Computes if this node needs balancing at all.
+   *
+   * @param threshold - Percentage
+   * @return true or false
+   */
+  public boolean isBalancingNeeded(float threshold) {
+    for (DiskBalancerVolumeSet vSet : getVolumeSets().values()) {
+      if (vSet.isBalancingNeeded(threshold)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Adds a volume to the DataNode.
+   * <p/>
+   * it is assumed that we have one thread per node hence this call is not
+   * synchronised neither is the map is protected.
+   *
+   * @param volume - volume
+   */
+  public void addVolume(DiskBalancerVolume volume) throws Exception {
+    Preconditions.checkNotNull(volume, "volume cannot be null");
+    Preconditions.checkNotNull(volumeSets, "volume sets cannot be null");
+    Preconditions
+        .checkNotNull(volume.getStorageType(), "storage type cannot be null");
+
+    String volumeSetKey = volume.getStorageType();
+    DiskBalancerVolumeSet vSet;
+    if (volumeSets.containsKey(volumeSetKey)) {
+      vSet = volumeSets.get(volumeSetKey);
+    } else {
+      vSet = new DiskBalancerVolumeSet(volume.isTransient());
+      volumeSets.put(volumeSetKey, vSet);
+    }
+
+    vSet.addVolume(volume);
+    computeNodeDensity();
+  }
+
+  /**
+   * Returns how many volumes are in the DataNode.
+   *
+   * @return int
+   */
+  public int getVolumeCount() {
+    return volumeCount;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
new file mode 100644
index 0000000..a608248
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
@@ -0,0 +1,330 @@
+/**
+ * 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.hdfs.server.diskbalancer.datamodel;
+
+import com.google.common.base.Preconditions;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+
+/**
+ * DiskBalancerVolume represents a volume in the DataNode.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class DiskBalancerVolume {
+  private String path;
+  private long capacity;
+  private String storageType;
+  private long used;
+  private long reserved;
+  private String uuid;
+  private boolean failed;
+  private boolean isTransient;
+  private float volumeDataDensity;
+  private boolean skip = false;
+  private boolean isReadOnly;
+
+  /**
+   * Constructs DiskBalancerVolume.
+   */
+  public DiskBalancerVolume() {
+  }
+
+  /**
+   * Parses a Json string and converts to DiskBalancerVolume.
+   *
+   * @param json - Json String
+   *
+   * @return DiskBalancerCluster
+   *
+   * @throws IOException
+   */
+  public static DiskBalancerVolume parseJson(String json) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(json, DiskBalancerVolume.class);
+  }
+
+  /**
+   * Get this volume Data Density
+   * Please see DiskBalancerVolumeSet#computeVolumeDataDensity to see how
+   * this is computed.
+   *
+   * @return float.
+   */
+  public float getVolumeDataDensity() {
+    return volumeDataDensity;
+  }
+
+  /**
+   * Sets this volume's data density.
+   *
+   * @param volDataDensity - density
+   */
+  public void setVolumeDataDensity(float volDataDensity) {
+    this.volumeDataDensity = volDataDensity;
+  }
+
+  /**
+   * Indicates if the volume is Transient in nature.
+   *
+   * @return true or false.
+   */
+  public boolean isTransient() {
+    return isTransient;
+  }
+
+  /**
+   * Sets volumes transient nature.
+   *
+   * @param aTransient - bool
+   */
+  public void setTransient(boolean aTransient) {
+    this.isTransient = aTransient;
+  }
+
+  /**
+   * Compares two volumes and decides if it is the same volume.
+   *
+   * @param o Volume Object
+   *
+   * @return boolean
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DiskBalancerVolume that = (DiskBalancerVolume) o;
+    return uuid.equals(that.uuid);
+  }
+
+  /**
+   * Computes hash code for a diskBalancerVolume.
+   *
+   * @return int
+   */
+  @Override
+  public int hashCode() {
+    return uuid.hashCode();
+  }
+
+  /**
+   * Capacity of this volume.
+   *
+   * @return long
+   */
+  public long getCapacity() {
+    return capacity;
+  }
+
+  /**
+   * Sets the capacity of this volume.
+   *
+   * @param totalCapacity long
+   */
+  public void setCapacity(long totalCapacity) {
+    this.capacity = totalCapacity;
+  }
+
+  /**
+   * Indicates if this is a failed volume.
+   *
+   * @return boolean
+   */
+  public boolean isFailed() {
+    return failed;
+  }
+
+  /**
+   * Sets the failed flag for this volume.
+   *
+   * @param fail boolean
+   */
+  public void setFailed(boolean fail) {
+    this.failed = fail;
+  }
+
+  /**
+   * Returns the path for this volume.
+   *
+   * @return String
+   */
+  public String getPath() {
+    return path;
+  }
+
+  /**
+   * Sets the path for this volume.
+   *
+   * @param volPath Path
+   */
+  public void setPath(String volPath) {
+    this.path = volPath;
+  }
+
+  /**
+   * Gets the reserved size for this volume.
+   *
+   * @return Long - Reserved size.
+   */
+  public long getReserved() {
+    return reserved;
+  }
+
+  /**
+   * Sets the reserved size.
+   *
+   * @param reservedSize -- Sets the reserved.
+   */
+  public void setReserved(long reservedSize) {
+    this.reserved = reservedSize;
+  }
+
+  /**
+   * Gets the StorageType.
+   *
+   * @return String StorageType.
+   */
+  public String getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Sets the StorageType.
+   *
+   * @param typeOfStorage - Storage Type String.
+   */
+  public void setStorageType(String typeOfStorage) {
+    this.storageType = typeOfStorage;
+  }
+
+  /**
+   * Gets the dfsUsed Size.
+   *
+   * @return - long - used space
+   */
+  public long getUsed() {
+    return used;
+  }
+
+  /**
+   * Sets the used Space for Long.
+   *
+   * @param dfsUsedSpace - dfsUsedSpace for this volume.
+   */
+  public void setUsed(long dfsUsedSpace) {
+    Preconditions.checkArgument(dfsUsedSpace < this.getCapacity());
+    this.used = dfsUsedSpace;
+  }
+
+  /**
+   * Gets the uuid for this volume.
+   *
+   * @return String - uuid of th volume
+   */
+  public String getUuid() {
+    return uuid;
+  }
+
+  /**
+   * Sets the uuid for this volume.
+   *
+   * @param id - String
+   */
+  public void setUuid(String id) {
+    this.uuid = id;
+  }
+
+  /**
+   * Returns effective capacity of a volume.
+   *
+   * @return float - fraction that represents used capacity.
+   */
+  @JsonIgnore
+  public long computeEffectiveCapacity() {
+    return getCapacity() - getReserved();
+  }
+
+  /**
+   * returns a Json String.
+   *
+   * @return String
+   *
+   * @throws IOException
+   */
+  public String toJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(this);
+  }
+
+  /**
+   * returns if we should skip this volume.
+   * @return true / false
+   */
+  public boolean isSkip() {
+    return skip;
+  }
+
+  /**
+   * Sets the Skip value for this volume.
+   * @param skipValue bool
+   */
+  public void setSkip(boolean skipValue) {
+    this.skip = skipValue;
+  }
+
+  /**
+   * Returns the usedPercentage of a disk.
+   * This is useful in debugging disk usage
+   * @return float
+   */
+  public float computeUsedPercentage() {
+    return (float) (getUsed()) / (float) (getCapacity());
+  }
+
+  /**
+   * Tells us if a volume is transient.
+   * @param transientValue
+   */
+  public void setIsTransient(boolean transientValue) {
+    this.isTransient = transientValue;
+  }
+
+  /**
+   * Tells us if this volume is read-only.
+   * @return true / false
+   */
+  public boolean isReadOnly() {
+    return isReadOnly;
+  }
+
+  /**
+   * Sets this volume as read only.
+   * @param readOnly - boolean
+   */
+  public void setReadOnly(boolean readOnly) {
+    isReadOnly = readOnly;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
new file mode 100644
index 0000000..15c21ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
@@ -0,0 +1,325 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.datamodel;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.htrace.fasterxml.jackson.annotation.JsonProperty;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+
+/**
+ * DiskBalancerVolumeSet is a collection of storage devices on the
+ * data node which are of similar StorageType.
+ */
+@JsonIgnoreProperties({"sortedQueue", "volumeCount", "idealUsed"})
+public class DiskBalancerVolumeSet {
+  static final Log LOG = LogFactory.getLog(DiskBalancerVolumeSet.class);
+  private final int maxDisks = 256;
+
+  @JsonProperty("transient")
+  private boolean isTransient;
+  private Set<DiskBalancerVolume> volumes;
+
+  @JsonIgnore
+  private TreeSet<DiskBalancerVolume> sortedQueue;
+  private String storageType;
+  private String setID;
+
+  private float idealUsed;
+
+
+  /**
+   * Constructs Empty DiskNBalanceVolumeSet.
+   * This is needed by jackson
+   */
+  public DiskBalancerVolumeSet() {
+    setID = UUID.randomUUID().toString();
+  }
+
+  /**
+   * Constructs a DiskBalancerVolumeSet.
+   *
+   * @param isTransient - boolean
+   */
+  public DiskBalancerVolumeSet(boolean isTransient) {
+    this.isTransient = isTransient;
+    volumes = new HashSet<>(maxDisks);
+    sortedQueue = new TreeSet<>(new MinHeap());
+    this.storageType = null;
+    setID = UUID.randomUUID().toString();
+  }
+
+  /**
+   * Constructs a new DiskBalancerVolumeSet.
+   */
+  public DiskBalancerVolumeSet(DiskBalancerVolumeSet volumeSet) {
+    this.isTransient = volumeSet.isTransient();
+    this.storageType = volumeSet.storageType;
+    this.volumes = new HashSet<>(volumeSet.volumes);
+    sortedQueue = new TreeSet<>(new MinHeap());
+    setID = UUID.randomUUID().toString();
+  }
+
+  /**
+   * Tells us if this volumeSet is transient.
+   *
+   * @return - true or false
+   */
+  @JsonProperty("transient")
+  public boolean isTransient() {
+    return isTransient;
+  }
+
+  /**
+   * Set the transient properties for this volumeSet.
+   *
+   * @param transientValue - Boolean
+   */
+  @JsonProperty("transient")
+  public void setTransient(boolean transientValue) {
+    this.isTransient = transientValue;
+  }
+
+  /**
+   * Computes Volume Data Density. Adding a new volume changes
+   * the volumeDataDensity for all volumes. So we throw away
+   * our priority queue and recompute everything.
+   *
+   * we discard failed volumes from this computation.
+   *
+   * totalCapacity = totalCapacity of this volumeSet
+   * totalUsed = totalDfsUsed for this volumeSet
+   * idealUsed = totalUsed / totalCapacity
+   * dfsUsedRatio = dfsUsedOnAVolume / Capacity On that Volume
+   * volumeDataDensity = idealUsed - dfsUsedRatio
+   */
+  public void computeVolumeDataDensity() {
+    long totalCapacity = 0;
+    long totalUsed = 0;
+    sortedQueue.clear();
+
+    // when we plan to re-distribute data we need to make
+    // sure that we skip failed volumes.
+    for (DiskBalancerVolume volume : volumes) {
+      if (!volume.isFailed() && !volume.isSkip()) {
+
+        if (volume.computeEffectiveCapacity() < 0) {
+          skipMisConfiguredVolume(volume);
+          continue;
+        }
+
+        totalCapacity += volume.computeEffectiveCapacity();
+        totalUsed += volume.getUsed();
+      }
+    }
+
+    if (totalCapacity != 0) {
+      this.idealUsed = totalUsed / (float) totalCapacity;
+    }
+
+    for (DiskBalancerVolume volume : volumes) {
+      if (!volume.isFailed() && !volume.isSkip()) {
+        float dfsUsedRatio =
+            volume.getUsed() / (float) volume.computeEffectiveCapacity();
+        volume.setVolumeDataDensity(this.idealUsed - dfsUsedRatio);
+        sortedQueue.add(volume);
+      }
+    }
+  }
+
+  private void skipMisConfiguredVolume(DiskBalancerVolume volume) {
+    //probably points to some sort of mis-configuration. Log this and skip
+    // processing this volume.
+    String errMessage = String.format("Real capacity is negative." +
+                                          "This usually points to some " +
+                                          "kind of mis-configuration.%n" +
+                                          "Capacity : %d Reserved : %d " +
+                                          "realCap = capacity - " +
+                                          "reserved = %d.%n" +
+                                          "Skipping this volume from " +
+                                          "all processing. type : %s id" +
+                                          " :%s",
+                                      volume.getCapacity(),
+                                      volume.getReserved(),
+                                      volume.computeEffectiveCapacity(),
+                                      volume.getStorageType(),
+                                      volume.getUuid());
+
+    LOG.fatal(errMessage);
+    volume.setSkip(true);
+  }
+
+  /**
+   * Returns the number of volumes in the Volume Set.
+   *
+   * @return int
+   */
+  @JsonIgnore
+  public int getVolumeCount() {
+    return volumes.size();
+  }
+
+  /**
+   * Get Storage Type.
+   *
+   * @return String
+   */
+  public String getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Set Storage Type.
+   * @param typeOfStorage -- StorageType
+   */
+  public void setStorageType(String typeOfStorage) {
+    this.storageType = typeOfStorage;
+  }
+
+  /**
+   * adds a given volume into this volume set.
+   *
+   * @param volume - volume to add.
+   *
+   * @throws Exception
+   */
+  public void addVolume(DiskBalancerVolume volume) throws Exception {
+    Preconditions.checkNotNull(volume, "volume cannot be null");
+    Preconditions.checkState(isTransient() == volume.isTransient(),
+                             "Mismatch in volumeSet and volume's transient " +
+                                 "properties.");
+
+
+    if (this.storageType == null) {
+      Preconditions.checkState(volumes.size() == 0L, "Storage Type is Null but"
+          + " volume size is " + volumes.size());
+      this.storageType = volume.getStorageType();
+    } else {
+      Preconditions.checkState(this.storageType.equals(volume.getStorageType()),
+                               "Adding wrong type of disk to this volume set");
+    }
+    volumes.add(volume);
+    computeVolumeDataDensity();
+
+  }
+
+  /**
+   * Returns a list diskVolumes that are part of this volume set.
+   *
+   * @return List
+   */
+  public List<DiskBalancerVolume> getVolumes() {
+    return new ArrayList<>(volumes);
+  }
+
+
+  @JsonIgnore
+  public TreeSet<DiskBalancerVolume> getSortedQueue() {
+    return sortedQueue;
+  }
+
+  /**
+   * Computes whether we need to do any balancing on this volume Set at all.
+   * It checks if any disks are out of threshold value
+   *
+   * @param thresholdPercentage - threshold - in percentage
+   *
+   * @return true if balancing is needed false otherwise.
+   */
+  public boolean isBalancingNeeded(float thresholdPercentage) {
+    float threshold = thresholdPercentage / 100.0f;
+
+    if(volumes == null || volumes.size() <= 1) {
+      // there is nothing we can do with a single volume.
+      // so no planning needed.
+      return false;
+    }
+
+    for (DiskBalancerVolume vol : volumes) {
+      boolean notSkip = !vol.isFailed() && !vol.isTransient() && !vol.isSkip();
+      if ((Math.abs(vol.getVolumeDataDensity()) > threshold) && notSkip) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Remove a volume from the current set.
+   *
+   * This call does not recompute the volumeDataDensity. It has to be
+   * done manually after this call.
+   *
+   * @param volume - Volume to remove
+   */
+  public void removeVolume(DiskBalancerVolume volume) {
+    volumes.remove(volume);
+    sortedQueue.remove(volume);
+  }
+
+  /**
+   * Get Volume Set ID.
+   * @return String
+   */
+  public String getSetID() {
+    return setID;
+  }
+
+  /**
+   * Set VolumeSet ID.
+   * @param volID String
+   */
+  public void setSetID(String volID) {
+    this.setID = volID;
+  }
+
+  /**
+   * Gets the idealUsed for this volume set.
+   */
+
+  @JsonIgnore
+  public float getIdealUsed() {
+    return this.idealUsed;
+  }
+
+  static class MinHeap implements Comparator<DiskBalancerVolume>, Serializable {
+
+    /**
+     * Compares its two arguments for order.  Returns a negative integer,
+     * zero, or a positive integer as the first argument is less than, equal
+     * to, or greater than the second.
+     */
+    @Override
+    public int compare(DiskBalancerVolume first, DiskBalancerVolume second) {
+      return Float
+          .compare(second.getVolumeDataDensity(), first.getVolumeDataDensity());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/package-info.java
new file mode 100644
index 0000000..f72e283
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/package-info.java
@@ -0,0 +1,31 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.datamodel;
+/**
+ * Disk Balancer Data Model is the Data Model for the cluster that
+ * Disk Balancer is working against. This information is read
+ * directly from NameNode or from a user supplied json model file.
+ *
+ * Here is the overview of the model maintained by diskBalancer.
+ *
+ * DiskBalancerCluster is a list of DiskBalancerDataNodes.
+ * DiskBalancerDataNodes is a collection of DiskBalancerVolumeSets
+ * DiskBalancerVolumeSets is a collection of DiskBalancerVolumes
+ * DiskBalancerVolumes represents actual volumes on DataNodes.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/package-info.java
new file mode 100644
index 0000000..4bec98f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/package-info.java
@@ -0,0 +1,36 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer;
+/**
+ * Disk Balancer connects to a {@link org.apache.hadoop.hdfs.server.datanode
+ * .DataNode} and attempts to spread data across all volumes evenly.
+ *
+ * This is achieved by :
+ *
+ * 1) Calculating the average data that should be on a set of volumes grouped
+ * by the type. For example, how much data should be on each volume of SSDs on a
+ * machine.
+ *
+ * 2) Once we know the average data that is expected to be on a volume we
+ * move data from volumes with higher than average load to volumes with
+ * less than average load.
+ *
+ * 3) Disk Balancer operates against data nodes which are live and operational.
+ *
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
new file mode 100644
index 0000000..5e3f4bf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -0,0 +1,227 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.apache.hadoop.util.Time;
+
+import java.util.Random;
+import java.util.UUID;
+
+/**
+ * Helper class to create various cluster configrations at run time.
+ */
+public class DiskBalancerTestUtil {
+  // we modeling disks here, hence HDD style units
+  public static final long GB = 1000000000L;
+  public static final long TB = 1000000000000L;
+  private static int[] diskSizes =
+      {1, 2, 3, 4, 5, 6, 7, 8, 9, 100, 200, 300, 400, 500, 600, 700, 800, 900};
+  Random rand;
+  private String stringTable =
+      "ABCDEDFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0987654321";
+
+  /**
+   * Constructs a util class.
+   */
+  public DiskBalancerTestUtil() {
+    this.rand = new Random(Time.monotonicNow());
+  }
+
+  /**
+   * Returns a random string.
+   *
+   * @param length - Number of chars in the string
+   *
+   * @return random String
+   */
+  private String getRandomName(int length) {
+    StringBuilder name = new StringBuilder();
+    for (int x = 0; x < length; x++) {
+      name.append(stringTable.charAt(rand.nextInt(stringTable.length())));
+    }
+    return name.toString();
+  }
+
+  /**
+   * Returns a Random Storage Type.
+   *
+   * @return - StorageType
+   */
+  private StorageType getRandomStorageType() {
+    return StorageType.parseStorageType(rand.nextInt(3));
+  }
+
+  /**
+   * Returns random capacity, if the size is smaller than 10
+   * they are TBs otherwise the size is assigned to GB range.
+   *
+   * @return Long - Disk Size
+   */
+  private long getRandomCapacity() {
+    int size = diskSizes[rand.nextInt(diskSizes.length)];
+    if (size < 10) {
+      return size * TB;
+    } else {
+      return size * GB;
+    }
+  }
+
+  /**
+   * Some value under 20% in these tests.
+   */
+  private long getRandomReserved(long capacity) {
+    double rcap = capacity * 0.2d;
+    double randDouble = rand.nextDouble();
+    double temp = randDouble * rcap;
+    return (new Double(temp)).longValue();
+
+  }
+
+  /**
+   * Some value less that capacity - reserved.
+   */
+  private long getRandomDfsUsed(long capacity, long reserved) {
+    double rcap = capacity - reserved;
+    double randDouble = rand.nextDouble();
+    double temp = randDouble * rcap;
+    return (new Double(temp)).longValue();
+  }
+
+  /**
+   * Creates a Random Volume of a specific storageType.
+   *
+   * @return Volume
+   */
+  public DiskBalancerVolume createRandomVolume() {
+    return createRandomVolume(getRandomStorageType());
+  }
+
+  /**
+   * Creates a Random Volume for testing purpose.
+   *
+   * @param type - StorageType
+   *
+   * @return DiskBalancerVolume
+   */
+  public DiskBalancerVolume createRandomVolume(StorageType type) {
+    DiskBalancerVolume volume = new DiskBalancerVolume();
+    volume.setPath("/tmp/disk/" + getRandomName(10));
+    volume.setStorageType(type.toString());
+    volume.setTransient(type.isTransient());
+
+    volume.setCapacity(getRandomCapacity());
+    volume.setReserved(getRandomReserved(volume.getCapacity()));
+    volume
+        .setUsed(getRandomDfsUsed(volume.getCapacity(), volume.getReserved()));
+    volume.setUuid(UUID.randomUUID().toString());
+    return volume;
+  }
+
+  /**
+   * Creates a RandomVolumeSet.
+   *
+   * @param type -Storage Type
+   * @param diskCount - How many disks you need.
+   *
+   * @return volumeSet
+   *
+   * @throws Exception
+   */
+  public DiskBalancerVolumeSet createRandomVolumeSet(StorageType type,
+                                                     int diskCount)
+      throws Exception {
+
+    Preconditions.checkState(diskCount > 0);
+    DiskBalancerVolumeSet volumeSet =
+        new DiskBalancerVolumeSet(type.isTransient());
+    for (int x = 0; x < diskCount; x++) {
+      volumeSet.addVolume(createRandomVolume(type));
+    }
+    assert (volumeSet.getVolumeCount() == diskCount);
+    return volumeSet;
+  }
+
+  /**
+   * Creates a RandomDataNode.
+   *
+   * @param diskTypes - Storage types needed in the Node
+   * @param diskCount - Disk count - that many disks of each type is created
+   *
+   * @return DataNode
+   *
+   * @throws Exception
+   */
+  public DiskBalancerDataNode createRandomDataNode(StorageType[] diskTypes,
+                                                   int diskCount)
+      throws Exception {
+    Preconditions.checkState(diskTypes.length > 0);
+    Preconditions.checkState(diskCount > 0);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    for (StorageType t : diskTypes) {
+      DiskBalancerVolumeSet vSet = createRandomVolumeSet(t, diskCount);
+      for (DiskBalancerVolume v : vSet.getVolumes()) {
+        node.addVolume(v);
+      }
+    }
+    return node;
+  }
+
+  /**
+   * Creates a RandomCluster.
+   *
+   * @param dataNodeCount - How many nodes you need
+   * @param diskTypes - StorageTypes you need in each node
+   * @param diskCount - How many disks you need of each type.
+   *
+   * @return Cluster
+   *
+   * @throws Exception
+   */
+  public DiskBalancerCluster createRandCluster(int dataNodeCount,
+                                               StorageType[] diskTypes,
+                                               int diskCount)
+
+      throws Exception {
+    Preconditions.checkState(diskTypes.length > 0);
+    Preconditions.checkState(diskCount > 0);
+    Preconditions.checkState(dataNodeCount > 0);
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    // once we add these nodes into the connector, cluster will read them
+    // from the connector.
+    for (int x = 0; x < dataNodeCount; x++) {
+      nullConnector.addNode(createRandomDataNode(diskTypes, diskCount));
+    }
+
+    // with this call we have populated the cluster info
+    cluster.readClusterInfo();
+    return cluster;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
new file mode 100644
index 0000000..3507c96
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
@@ -0,0 +1,224 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.UUID;
+
+public class TestDataModels {
+  @Test
+  public void TestCreateRandomVolume() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolume vol = util.createRandomVolume(StorageType.DISK);
+    Assert.assertNotNull(vol.getUuid());
+    Assert.assertNotNull(vol.getPath());
+    Assert.assertNotNull(vol.getStorageType());
+    Assert.assertFalse(vol.isFailed());
+    Assert.assertFalse(vol.isTransient());
+    Assert.assertTrue(vol.getCapacity() > 0);
+    Assert.assertTrue((vol.getCapacity() - vol.getReserved()) > 0);
+    Assert.assertTrue((vol.getReserved() + vol.getUsed()) < vol.getCapacity());
+  }
+
+  @Test
+  public void TestCreateRandomVolumeSet() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolumeSet vSet =
+        util.createRandomVolumeSet(StorageType.SSD, 10);
+    Assert.assertEquals(10, vSet.getVolumeCount());
+    Assert.assertEquals(StorageType.SSD.toString(),
+        vSet.getVolumes().get(0).getStorageType());
+
+  }
+
+  @Test
+  public void TestCreateRandomDataNode() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node = util.createRandomDataNode(
+        new StorageType[]{StorageType.DISK, StorageType.RAM_DISK}, 10);
+    Assert.assertNotNull(node.getNodeDataDensity());
+  }
+
+  @Test
+  public void TestDiskQueues() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node = util.createRandomDataNode(
+        new StorageType[]{StorageType.DISK, StorageType.RAM_DISK}, 3);
+    TreeSet<DiskBalancerVolume> sortedQueue =
+        node.getVolumeSets().get(StorageType.DISK.toString()).getSortedQueue();
+
+    List<DiskBalancerVolume> reverseList = new LinkedList<>();
+    List<DiskBalancerVolume> highList = new LinkedList<>();
+    int queueSize = sortedQueue.size();
+    for (int x = 0; x < queueSize; x++) {
+      reverseList.add(sortedQueue.first());
+      highList.add(sortedQueue.first());
+    }
+    Collections.reverse(reverseList);
+
+    for (int x = 0; x < queueSize; x++) {
+
+      Assert.assertEquals(reverseList.get(x).getCapacity(),
+          highList.get(x).getCapacity());
+      Assert.assertEquals(reverseList.get(x).getReserved(),
+          highList.get(x).getReserved());
+      Assert.assertEquals(reverseList.get(x).getUsed(),
+          highList.get(x).getUsed());
+    }
+  }
+
+  @Test
+  public void TestNoBalancingNeededEvenDataSpread() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // create two disks which have exactly same data and isBalancing should
+    // say we don't need to balance.
+    DiskBalancerVolume v1 = util.createRandomVolume(StorageType.SSD);
+    v1.setCapacity(DiskBalancerTestUtil.TB);
+    v1.setReserved(100 * DiskBalancerTestUtil.GB);
+    v1.setUsed(500 * DiskBalancerTestUtil.GB);
+
+    DiskBalancerVolume v2 = util.createRandomVolume(StorageType.SSD);
+    v2.setCapacity(DiskBalancerTestUtil.TB);
+    v2.setReserved(100 * DiskBalancerTestUtil.GB);
+    v2.setUsed(500 * DiskBalancerTestUtil.GB);
+
+    node.addVolume(v1);
+    node.addVolume(v2);
+
+    for (DiskBalancerVolumeSet vsets : node.getVolumeSets().values()) {
+      Assert.assertFalse(vsets.isBalancingNeeded(10.0f));
+    }
+  }
+
+  @Test
+  public void TestNoBalancingNeededTransientDisks() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // create two disks which have different data sizes, but
+    // transient. isBalancing should say no balancing needed.
+    DiskBalancerVolume v1 = util.createRandomVolume(StorageType.RAM_DISK);
+    v1.setCapacity(DiskBalancerTestUtil.TB);
+    v1.setReserved(100 * DiskBalancerTestUtil.GB);
+    v1.setUsed(1 * DiskBalancerTestUtil.GB);
+
+    DiskBalancerVolume v2 = util.createRandomVolume(StorageType.RAM_DISK);
+    v2.setCapacity(DiskBalancerTestUtil.TB);
+    v2.setReserved(100 * DiskBalancerTestUtil.GB);
+    v2.setUsed(500 * DiskBalancerTestUtil.GB);
+
+    node.addVolume(v1);
+    node.addVolume(v2);
+
+    for (DiskBalancerVolumeSet vsets : node.getVolumeSets().values()) {
+      Assert.assertFalse(vsets.isBalancingNeeded(10.0f));
+    }
+  }
+
+  @Test
+  public void TestNoBalancingNeededFailedDisks() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // create two disks which have which are normal disks, but fail
+    // one of them. VolumeSet should say no balancing needed.
+    DiskBalancerVolume v1 = util.createRandomVolume(StorageType.SSD);
+    v1.setCapacity(DiskBalancerTestUtil.TB);
+    v1.setReserved(100 * DiskBalancerTestUtil.GB);
+    v1.setUsed(1 * DiskBalancerTestUtil.GB);
+    v1.setFailed(true);
+
+    DiskBalancerVolume v2 = util.createRandomVolume(StorageType.SSD);
+    v2.setCapacity(DiskBalancerTestUtil.TB);
+    v2.setReserved(100 * DiskBalancerTestUtil.GB);
+    v2.setUsed(500 * DiskBalancerTestUtil.GB);
+
+    node.addVolume(v1);
+    node.addVolume(v2);
+
+    for (DiskBalancerVolumeSet vsets : node.getVolumeSets().values()) {
+      Assert.assertFalse(vsets.isBalancingNeeded(10.0f));
+    }
+  }
+
+  @Test
+  public void TestNeedBalancingUnevenDataSpread() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    DiskBalancerVolume v1 = util.createRandomVolume(StorageType.SSD);
+    v1.setCapacity(DiskBalancerTestUtil.TB);
+    v1.setReserved(100 * DiskBalancerTestUtil.GB);
+    v1.setUsed(0);
+
+    DiskBalancerVolume v2 = util.createRandomVolume(StorageType.SSD);
+    v2.setCapacity(DiskBalancerTestUtil.TB);
+    v2.setReserved(100 * DiskBalancerTestUtil.GB);
+    v2.setUsed(500 * DiskBalancerTestUtil.GB);
+
+    node.addVolume(v1);
+    node.addVolume(v2);
+
+    for (DiskBalancerVolumeSet vsets : node.getVolumeSets().values()) {
+      Assert.assertTrue(vsets.isBalancingNeeded(10.0f));
+    }
+  }
+
+  @Test
+  public void TestVolumeSerialize() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+    DiskBalancerVolume volume = util.createRandomVolume(StorageType.DISK);
+    String originalString = volume.toJson();
+    DiskBalancerVolume parsedVolume =
+        DiskBalancerVolume.parseJson(originalString);
+    String parsedString = parsedVolume.toJson();
+    Assert.assertEquals(originalString, parsedString);
+  }
+
+  @Test
+  public void TestClusterSerialize() throws Exception {
+    DiskBalancerTestUtil util = new DiskBalancerTestUtil();
+
+    // Create a Cluster with 3 datanodes, 3 disk types and 3 disks in each type
+    // that is 9 disks in each machine.
+    DiskBalancerCluster cluster = util.createRandCluster(3, new StorageType[]{
+        StorageType.DISK, StorageType.RAM_DISK, StorageType.SSD}, 3);
+
+    DiskBalancerCluster newCluster =
+        DiskBalancerCluster.parseJson(cluster.toJson());
+    Assert.assertEquals(cluster.getNodes(), newCluster.getNodes());
+    Assert
+        .assertEquals(cluster.getNodes().size(), newCluster.getNodes().size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a5c481/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/NullConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/NullConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/NullConnector.java
new file mode 100644
index 0000000..3f78530
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/NullConnector.java
@@ -0,0 +1,59 @@
+/**
+ * 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.hdfs.server.diskbalancer.connectors;
+
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * This connector allows user to create an in-memory cluster
+ * and is useful in testing.
+ */
+public class NullConnector implements ClusterConnector {
+  private final List<DiskBalancerDataNode> nodes = new LinkedList<>();
+
+  /**
+   * getNodes function returns a list of DiskBalancerDataNodes.
+   *
+   * @return Array of DiskBalancerDataNodes
+   */
+  @Override
+  public List<DiskBalancerDataNode> getNodes() throws Exception {
+    return nodes;
+  }
+
+  /**
+   * Returns info about the connector.
+   *
+   * @return String.
+   */
+  @Override
+  public String getConnectorInfo() {
+    return "Null Connector : No persistence, in-memory connector";
+  }
+
+  /**
+   * Allows user to add nodes into this connector.
+   *
+   * @param node - Node to add
+   */
+  public void addNode(DiskBalancerDataNode node) {
+    nodes.add(node);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/49] hadoop git commit: HDFS-10402. DiskBalancer: Add QueryStatus command. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-10402. DiskBalancer: Add QueryStatus command. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9e5fcb5e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9e5fcb5e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9e5fcb5e

Branch: refs/heads/trunk
Commit: 9e5fcb5e40bb370e4579e6040c02e923c1a90427
Parents: 5df2d2b
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 20 14:09:58 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/QueryCommand.java      | 82 ++++++++++++++++++++
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 22 ++++++
 2 files changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e5fcb5e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
new file mode 100644
index 0000000..36448b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -0,0 +1,82 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
+/**
+ * Gets the current status of disk balancer command.
+ */
+public class QueryCommand extends Command {
+
+  /**
+   * Constructs QueryCommand.
+   *
+   * @param conf - Configuration.
+   */
+  public QueryCommand(Configuration conf) {
+    super(conf);
+    addValidCommandParameters(DiskBalancer.QUERY, "Queries the status of disk" +
+        " plan running on a given datanode.");
+    addValidCommandParameters(DiskBalancer.VERBOSE, "Prints verbose results.");
+  }
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.info("Executing \"query plan\" command.");
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.QUERY));
+    verifyCommandOptions(DiskBalancer.QUERY, cmd);
+    String nodeName = cmd.getOptionValue(DiskBalancer.QUERY);
+    Preconditions.checkNotNull(nodeName);
+    ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeName);
+    try {
+      DiskBalancerWorkStatus workStatus = dataNode.queryDiskBalancerPlan();
+      System.out.printf("Plan ID: %s Result: %s%n", workStatus.getPlanID(),
+          workStatus.getResult().toString());
+
+      if(cmd.hasOption(DiskBalancer.VERBOSE)) {
+        System.out.printf("%s", workStatus.currentStateString());
+      }
+    } catch (DiskBalancerException ex) {
+      LOG.error("Query plan failed. ex: {}", ex);
+      throw ex;
+    }
+  }
+
+  /**
+   * Gets extended help for this command.
+   *
+   * @return Help Message
+   */
+  @Override
+  protected String getHelp() {
+    return "Gets the status of disk balancing on a given node";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e5fcb5e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 87fbf4a..f5dbe4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.QueryCommand;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
@@ -101,6 +102,11 @@ public class DiskBalancer extends Configured implements Tool {
   public static final String VERBOSE = "v";
   public static final int PLAN_VERSION = 1;
   /**
+   * Reports the status of disk balancer operation.
+   */
+  public static final String QUERY = "query";
+
+  /**
    * Template for the Before File. It is node.before.json.
    */
   public static final String BEFORE_TEMPLATE = "%s.before.json";
@@ -160,6 +166,8 @@ public class DiskBalancer extends Configured implements Tool {
   private Options getOpts() {
     Options opts = new Options();
     addPlanCommands(opts);
+    addExecuteCommands(opts);
+    addQueryCommands(opts);
     return opts;
   }
 
@@ -216,6 +224,16 @@ public class DiskBalancer extends Configured implements Tool {
   }
 
   /**
+   * Adds query command options.
+   * @param opt Options
+   */
+  private void addQueryCommands(Options opt) {
+    Option query = new Option(QUERY, true, "Queries the disk balancer " +
+        "status of a given datanode. e.g. -query <nodename>");
+    opt.addOption(query);
+  }
+
+  /**
    * This function parses all command line arguments and returns the appropriate
    * values.
    *
@@ -249,6 +267,10 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new ExecuteCommand(getConf());
       }
 
+      if(cmd.hasOption(DiskBalancer.QUERY)) {
+        currentCommand = new QueryCommand(getConf());
+      }
+
       if(currentCommand == null) {
         HelpFormatter helpFormatter = new HelpFormatter();
         helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/49] hadoop git commit: HDFS-10500. Diskbalancer: Print out information when a plan is not generated. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10500. Diskbalancer: Print out information when a plan is not generated. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/78a1032b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78a1032b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78a1032b

Branch: refs/heads/trunk
Commit: 78a1032b71af7672840da98808e2bebac3cc11d1
Parents: d2ff793
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Jun 9 13:43:19 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/CancelCommand.java     |  4 +-
 .../server/diskbalancer/command/Command.java    | 19 +-----
 .../diskbalancer/command/ExecuteCommand.java    |  2 +-
 .../diskbalancer/command/PlanCommand.java       | 61 ++++++++++----------
 4 files changed, 37 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78a1032b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
index f395802..3834d9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
@@ -70,7 +70,7 @@ public class CancelCommand extends Command {
       // points us to the plan file, we can compute the hash as well as read
       // the address of the datanode from the plan file.
       String planFile = cmd.getOptionValue(DiskBalancer.CANCEL);
-      Preconditions.checkArgument(planFile == null || planFile.isEmpty(),
+      Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
           "Invalid plan file specified.");
       String planData = null;
       try (FSDataInputStream plan = open(planFile)) {
@@ -88,7 +88,7 @@ public class CancelCommand extends Command {
    */
   private void cancelPlan(String planData) throws IOException {
     Preconditions.checkNotNull(planData);
-    NodePlan plan = readPlan(planData);
+    NodePlan plan = NodePlan.parseJson(planData);
     String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
     Preconditions.checkNotNull(dataNodeAddress);
     ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78a1032b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index fb975a8..94a21d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -31,16 +31,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
-
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,10 +56,10 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.List;
 
 /**
  * Common interface for command handling.
@@ -394,16 +391,4 @@ public abstract class Command extends Configured {
   protected DiskBalancerCluster getCluster() {
     return cluster;
   }
-
-  /**
-   * Returns a plan from the Json Data.
-   *
-   * @param planData - Json String
-   * @return NodePlan
-   * @throws IOException
-   */
-  protected NodePlan readPlan(String planData) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.readValue(planData, NodePlan.class);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78a1032b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index 6d30e86..85f2a86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -79,7 +79,7 @@ public class ExecuteCommand extends Command {
    */
   private void submitPlan(String planData) throws IOException {
     Preconditions.checkNotNull(planData);
-    NodePlan plan = readPlan(planData);
+    NodePlan plan = NodePlan.parseJson(planData);
     String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
     Preconditions.checkNotNull(dataNodeAddress);
     ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78a1032b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 7cf0df1..91ab7fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -25,13 +25,12 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
-    .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;
@@ -122,11 +121,14 @@ public class PlanCommand extends Command {
     setNodesToProcess(node);
     populatePathNames(node);
 
+    NodePlan plan = null;
     List<NodePlan> plans = getCluster().computePlan(this.thresholdPercentage);
     setPlanParams(plans);
 
-    LOG.info("Writing plan to : {}", getOutputPath());
-    System.out.printf("Writing plan to : %s%n", getOutputPath());
+    if (plans.size() > 0) {
+      plan = plans.get(0);
+    }
+
 
     try (FSDataOutputStream beforeStream = create(String.format(
         DiskBalancer.BEFORE_TEMPLATE,
@@ -135,18 +137,24 @@ public class PlanCommand extends Command {
           .getBytes(StandardCharsets.UTF_8));
     }
 
-    try (FSDataOutputStream planStream = create(String.format(
-        DiskBalancer.PLAN_TEMPLATE,
-        cmd.getOptionValue(DiskBalancer.PLAN)))) {
-      planStream.write(getPlan(plans).getBytes(StandardCharsets.UTF_8));
+    if (plan != null) {
+      LOG.info("Writing plan to : {}", getOutputPath());
+      try (FSDataOutputStream planStream = create(String.format(
+          DiskBalancer.PLAN_TEMPLATE,
+          cmd.getOptionValue(DiskBalancer.PLAN)))) {
+        planStream.write(plan.toJson().getBytes(StandardCharsets.UTF_8));
+      }
+    } else {
+      LOG.info("No plan generated. DiskBalancing not needed for node: {} " +
+              "threshold used: {}", cmd.getOptionValue(DiskBalancer.PLAN),
+          this.thresholdPercentage);
     }
 
-    if (cmd.hasOption(DiskBalancer.VERBOSE)) {
+    if (cmd.hasOption(DiskBalancer.VERBOSE) && plans.size() > 0) {
       printToScreen(plans);
     }
   }
 
-
   /**
    * Reads the Physical path of the disks we are balancing. This is needed to
    * make the disk balancer human friendly and not used in balancing.
@@ -210,14 +218,21 @@ public class PlanCommand extends Command {
   static private void printToScreen(List<NodePlan> plans) {
     System.out.println("\nPlan :\n");
     System.out.println(StringUtils.repeat("=", 80));
-    System.out.println("Source Disk\t\t Dest.Disk\t\t Move Size\t Type\n ");
+
+    System.out.println(
+        StringUtils.center("Source Disk", 30) +
+        StringUtils.center("Dest.Disk", 30) +
+        StringUtils.center("Size", 10) +
+        StringUtils.center("Type", 10));
+
     for (NodePlan plan : plans) {
       for (Step step : plan.getVolumeSetPlans()) {
-        System.out.println(String.format("%s\t%s\t%s\t%s",
-            step.getSourceVolume().getPath(),
-            step.getDestinationVolume().getPath(),
-            step.getSizeString(step.getBytesToMove()),
-            step.getDestinationVolume().getStorageType()));
+        System.out.println(String.format("%s %s %s %s",
+            StringUtils.center(step.getSourceVolume().getPath(), 30),
+            StringUtils.center(step.getDestinationVolume().getPath(), 30),
+            StringUtils.center(step.getSizeString(step.getBytesToMove()), 10),
+            StringUtils.center(step.getDestinationVolume().getStorageType(),
+                10)));
       }
     }
 
@@ -243,16 +258,4 @@ public class PlanCommand extends Command {
       }
     }
   }
-
-  /**
-   * Returns a Json represenation of the plans.
-   *
-   * @param plan - List of plans.
-   * @return String.
-   * @throws IOException
-   */
-  private String getPlan(List<NodePlan> plan) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.writeValueAsString(plan);
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/49] hadoop git commit: HDFS-10552. DiskBalancer "-query" results in NPE if no plan for the node. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10552. DiskBalancer "-query" results in NPE if no plan for the node. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8a6e3541
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8a6e3541
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8a6e3541

Branch: refs/heads/trunk
Commit: 8a6e3541226fb1b6798cedecc56f1f160012becf
Parents: e8de281
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Jun 22 17:35:55 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:02 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/DiskBalancer.java      |  5 +++-
 .../diskbalancer/command/QueryCommand.java      |  2 +-
 .../command/TestDiskBalancerCommand.java        | 25 ++++++++++++++++++++
 3 files changed, 30 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a6e3541/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index b31b997..5a1fb9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -104,6 +104,7 @@ public class DiskBalancer {
     scheduler = Executors.newSingleThreadExecutor();
     lock = new ReentrantLock();
     workMap = new ConcurrentHashMap<>();
+    this.planID = "";  // to keep protobuf happy.
     this.isDiskBalancerEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_DISK_BALANCER_ENABLED,
         DFSConfigKeys.DFS_DISK_BALANCER_ENABLED_DEFAULT);
@@ -223,7 +224,9 @@ public class DiskBalancer {
     lock.lock();
     try {
       checkDiskBalancerEnabled();
-      if ((this.planID == null) || (!this.planID.equals(planID))) {
+      if (this.planID == null ||
+          !this.planID.equals(planID) ||
+          this.planID.isEmpty()) {
         LOG.error("Disk Balancer - No such plan. Cancel plan failed. PlanID: " +
             planID);
         throw new DiskBalancerException("No such plan.",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a6e3541/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index 6c759e2..fac1e51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -63,7 +63,7 @@ public class QueryCommand extends Command {
     String nodeAddress = nodeName;
 
     // if the string is not name:port format use the default port.
-    if (!nodeName.matches("^.*:\\d$")) {
+    if (!nodeName.matches("[^\\:]+:[0-9]{2,5}")) {
       int defaultIPC = NetUtils.createSocketAddr(
           getConf().getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
               DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a6e3541/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index ceb762f..b0821e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
@@ -351,4 +352,28 @@ public class TestDiskBalancerCommand {
     }
     return outputs;
   }
+
+  /**
+   * Making sure that we can query the node without having done a submit.
+   * @throws Exception
+   */
+  @Test
+  public void testDiskBalancerQueryWithoutSubmit() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    final int numDatanodes = 2;
+    MiniDFSCluster miniDFSCluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes).build();
+    try {
+      miniDFSCluster.waitActive();
+      DataNode dataNode = miniDFSCluster.getDataNodes().get(0);
+      final String queryArg = String.format("-query localhost:%d", dataNode
+          .getIpcPort());
+      final String cmdLine = String.format("hdfs diskbalancer %s",
+          queryArg);
+      runCommand(cmdLine);
+    } finally {
+      miniDFSCluster.shutdown();
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/49] hadoop git commit: HDFS-9543. DiskBalancer: Add Data mover. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9543. DiskBalancer: Add Data mover. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1594b472
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1594b472
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1594b472

Branch: refs/heads/trunk
Commit: 1594b472bb9df7537dbc001411c99058cc11ba41
Parents: 7820737
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Apr 28 16:12:04 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:20:24 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DataNode.java   |   2 -
 .../hdfs/server/datanode/DiskBalancer.java      | 365 +++++++++++++++++--
 .../datamodel/DiskBalancerDataNode.java         |  13 +-
 .../datamodel/DiskBalancerVolume.java           |   6 +-
 .../datamodel/DiskBalancerVolumeSet.java        |  34 +-
 .../server/diskbalancer/planner/MoveStep.java   |  14 +-
 .../hdfs/server/diskbalancer/planner/Step.java  |  20 +-
 .../hdfs/server/balancer/TestBalancer.java      |   3 +-
 .../server/diskbalancer/TestDiskBalancer.java   | 247 +++++++++++++
 .../hdfs/server/diskbalancer/TestPlanner.java   |  28 +-
 10 files changed, 666 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 776da3a..d6be2e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3324,8 +3324,6 @@ public class DataNode extends ReconfigurableBase
    * @param planID  - Hash value of the plan.
    * @param planVersion - Plan version, reserved for future use. We have only
    *                    version 1 now.
-   * @param bandwidth - Max disk bandwidth to use, 0 means use value defined
-   *                  in the configration.
    * @param plan - Actual plan
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index b62a4fc..7f768ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -23,7 +23,9 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.DiskBalancerWorkEntry;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus
+    .DiskBalancerWorkEntry;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -39,6 +41,8 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.HashMap;
+import java.util.List;
+import java.util.LinkedList;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
@@ -48,18 +52,21 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
 /**
  * Worker class for Disk Balancer.
- * <p/>
+ * <p>
  * Here is the high level logic executed by this class. Users can submit disk
  * balancing plans using submitPlan calls. After a set of sanity checks the plan
  * is admitted and put into workMap.
- * <p/>
+ * <p>
  * The executePlan launches a thread that picks up work from workMap and hands
  * it over to the BlockMover#copyBlocks function.
- * <p/>
+ * <p>
  * Constraints :
- * <p/>
+ * <p>
  * Only one plan can be executing in a datanode at any given time. This is
  * ensured by checking the future handle of the worker thread in submitPlan.
  */
@@ -127,11 +134,12 @@ public class DiskBalancer {
    * Shutdown the executor.
    */
   private void shutdownExecutor() {
+    final int secondsTowait = 10;
     scheduler.shutdown();
     try {
-      if(!scheduler.awaitTermination(10, TimeUnit.SECONDS)) {
+      if (!scheduler.awaitTermination(secondsTowait, TimeUnit.SECONDS)) {
         scheduler.shutdownNow();
-        if (!scheduler.awaitTermination(10, TimeUnit.SECONDS)) {
+        if (!scheduler.awaitTermination(secondsTowait, TimeUnit.SECONDS)) {
           LOG.error("Disk Balancer : Scheduler did not terminate.");
         }
       }
@@ -207,6 +215,7 @@ public class DiskBalancer {
 
   /**
    * Cancels a running plan.
+   *
    * @param planID - Hash of the plan to cancel.
    * @throws DiskBalancerException
    */
@@ -297,7 +306,7 @@ public class DiskBalancer {
    * @throws DiskBalancerException
    */
   private NodePlan verifyPlan(String planID, long planVersion, String plan,
-                               boolean force) throws DiskBalancerException {
+                              boolean force) throws DiskBalancerException {
 
     Preconditions.checkState(lock.isHeldByCurrentThread());
     verifyPlanVersion(planVersion);
@@ -372,8 +381,8 @@ public class DiskBalancer {
         (TimeUnit.HOURS.toMillis(
             DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS))) < now) {
       String hourString = "Plan was generated more than " +
-           Integer.toString(DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS)
-          +  " hours ago.";
+          Integer.toString(DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS)
+          + " hours ago.";
       LOG.error("Disk Balancer - " + hourString);
       throw new DiskBalancerException(hourString,
           DiskBalancerException.Result.OLD_PLAN_SUBMITTED);
@@ -484,14 +493,14 @@ public class DiskBalancer {
   /**
    * Insert work items to work map.
    *
-   * @param source      - Source vol
-   * @param dest        - destination volume
-   * @param step        - Move Step
+   * @param source - Source vol
+   * @param dest   - destination volume
+   * @param step   - Move Step
    */
   private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest,
                               Step step) throws DiskBalancerException {
 
-    if(source.getStorageID().equals(dest.getStorageID())) {
+    if (source.getStorageID().equals(dest.getStorageID())) {
       LOG.info("Disk Balancer - source & destination volumes are same.");
       throw new DiskBalancerException("source and destination volumes are " +
           "same.", DiskBalancerException.Result.INVALID_MOVE);
@@ -604,13 +613,15 @@ public class DiskBalancer {
 
   /**
    * Actual DataMover class for DiskBalancer.
-   * <p/>
+   * <p>
    */
   public static class DiskBalancerMover implements BlockMover {
     private final FsDatasetSpi dataset;
     private long diskBandwidth;
     private long blockTolerance;
     private long maxDiskErrors;
+    private int poolIndex;
+    private AtomicBoolean shouldRun;
 
     /**
      * Constructs diskBalancerMover.
@@ -620,6 +631,7 @@ public class DiskBalancer {
      */
     public DiskBalancerMover(FsDatasetSpi dataset, Configuration conf) {
       this.dataset = dataset;
+      shouldRun = new AtomicBoolean(false);
 
       this.diskBandwidth = conf.getLong(
           DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT,
@@ -659,30 +671,333 @@ public class DiskBalancer {
     }
 
     /**
-     * Copies blocks from a set of volumes.
-     *
-     * @param pair - Source and Destination Volumes.
-     * @param item - Number of bytes to move from volumes.
+     * Sets Diskmover copyblocks into runnable state.
      */
     @Override
-    public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
-
+    public void setRunnable() {
+      this.shouldRun.set(true);
     }
 
     /**
-     * Begin the actual copy operations. This is useful in testing.
+     * Signals copy block to exit.
      */
     @Override
-    public void setRunnable() {
+    public void setExitFlag() {
+      this.shouldRun.set(false);
+    }
 
+    /**
+     * Returns the shouldRun boolean flag.
+     */
+    public boolean shouldRun() {
+      return this.shouldRun.get();
     }
 
     /**
-     * Tells copyBlocks to exit from the copy routine.
+     * Checks if a given block is less than needed size to meet our goal.
+     *
+     * @param blockSize - block len
+     * @param item      - Work item
+     * @return true if this block meets our criteria, false otherwise.
+     */
+    private boolean isLessThanNeeded(long blockSize,
+                                     DiskBalancerWorkItem item) {
+      long bytesToCopy = item.getBytesToCopy() - item.getBytesCopied();
+      bytesToCopy = bytesToCopy +
+          ((bytesToCopy * getBlockTolerancePercentage(item)) / 100);
+      return (blockSize <= bytesToCopy) ? true : false;
+    }
+
+    /**
+     * Returns the default block tolerance if the plan does not have value of
+     * tolerance specified.
+     *
+     * @param item - DiskBalancerWorkItem
+     * @return long
+     */
+    private long getBlockTolerancePercentage(DiskBalancerWorkItem item) {
+      return item.getTolerancePercent() <= 0 ? this.blockTolerance :
+          item.getTolerancePercent();
+    }
+
+    /**
+     * Inflates bytesCopied and returns true or false. This allows us to stop
+     * copying if we have reached close enough.
+     *
+     * @param item DiskBalancerWorkItem
+     * @return -- false if we need to copy more, true if we are done
+     */
+    private boolean isCloseEnough(DiskBalancerWorkItem item) {
+      long temp = item.getBytesCopied() +
+          ((item.getBytesCopied() * getBlockTolerancePercentage(item)) / 100);
+      return (item.getBytesToCopy() >= temp) ? false : true;
+    }
+
+    /**
+     * Returns disk bandwidth associated with this plan, if none is specified
+     * returns the global default.
+     *
+     * @param item DiskBalancerWorkItem.
+     * @return MB/s - long
+     */
+    private long getDiskBandwidth(DiskBalancerWorkItem item) {
+      return item.getBandwidth() <= 0 ? this.diskBandwidth : item
+          .getBandwidth();
+    }
+
+    /**
+     * Computes sleep delay needed based on the block that just got copied. we
+     * copy using a burst mode, that is we let the copy proceed in full
+     * throttle. Once a copy is done, we compute how many bytes have been
+     * transferred and try to average it over the user specified bandwidth. In
+     * other words, This code implements a poor man's token bucket algorithm for
+     * traffic shaping.
+     *
+     * @param bytesCopied - byteCopied.
+     * @param timeUsed    in milliseconds
+     * @param item        DiskBalancerWorkItem
+     * @return sleep delay in Milliseconds.
+     */
+    private long computeDelay(long bytesCopied, long timeUsed,
+                              DiskBalancerWorkItem item) {
+
+      // we had an overflow, ignore this reading and continue.
+      if (timeUsed == 0) {
+        return 0;
+      }
+      final int megaByte = 1024 * 1024;
+      long bytesInMB = bytesCopied / megaByte;
+      long lastThroughput = bytesInMB / SECONDS.convert(timeUsed,
+          TimeUnit.MILLISECONDS);
+      long delay = (bytesInMB / getDiskBandwidth(item)) - lastThroughput;
+      return (delay <= 0) ? 0 : MILLISECONDS.convert(delay, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Returns maximum errors to tolerate for the specific plan or the default.
+     *
+     * @param item - DiskBalancerWorkItem
+     * @return maximum error counts to tolerate.
+     */
+    private long getMaxError(DiskBalancerWorkItem item) {
+      return item.getMaxDiskErrors() <= 0 ? this.maxDiskErrors :
+          item.getMaxDiskErrors();
+    }
+
+    /**
+     * Gets the next block that we can copy, returns null if we cannot find a
+     * block that fits our parameters or if have run out of blocks.
+     *
+     * @param iter Block Iter
+     * @param item - Work item
+     * @return Extended block or null if no copyable block is found.
+     */
+    private ExtendedBlock getBlockToCopy(FsVolumeSpi.BlockIterator iter,
+                                         DiskBalancerWorkItem item) {
+      while (!iter.atEnd() && item.getErrorCount() < getMaxError(item)) {
+        try {
+          ExtendedBlock block = iter.nextBlock();
+
+          // A valid block is a finalized block, we iterate until we get
+          // finalized blocks
+          if (!this.dataset.isValidBlock(block)) {
+            continue;
+          }
+
+          // We don't look for the best, we just do first fit
+          if (isLessThanNeeded(block.getNumBytes(), item)) {
+            return block;
+          }
+
+        } catch (IOException e) {
+          item.incErrorCount();
+        }
+      }
+
+      if (item.getErrorCount() >= getMaxError(item)) {
+        item.setErrMsg("Error count exceeded.");
+        LOG.info("Maximum error count exceeded. Error count: {} Max error:{} "
+            , item.getErrorCount(), item.getMaxDiskErrors());
+      }
+
+      return null;
+    }
+
+    /**
+     * Opens all Block pools on a given volume.
+     *
+     * @param source    Source
+     * @param poolIters List of PoolIters to maintain.
+     */
+    private void openPoolIters(FsVolumeSpi source, List<FsVolumeSpi
+        .BlockIterator> poolIters) {
+      Preconditions.checkNotNull(source);
+      Preconditions.checkNotNull(poolIters);
+
+      for (String blockPoolID : source.getBlockPoolList()) {
+        poolIters.add(source.newBlockIterator(blockPoolID,
+            "DiskBalancerSource"));
+      }
+    }
+
+    /**
+     * Returns the next block that we copy from all the block pools. This
+     * function looks across all block pools to find the next block to copy.
+     *
+     * @param poolIters - List of BlockIterators
+     * @return ExtendedBlock.
+     */
+    ExtendedBlock getNextBlock(List<FsVolumeSpi.BlockIterator> poolIters,
+                               DiskBalancerWorkItem item) {
+      Preconditions.checkNotNull(poolIters);
+      int currentCount = 0;
+      ExtendedBlock block = null;
+      while (block == null && currentCount < poolIters.size()) {
+        currentCount++;
+        poolIndex = poolIndex++ % poolIters.size();
+        FsVolumeSpi.BlockIterator currentPoolIter = poolIters.get(poolIndex);
+        block = getBlockToCopy(currentPoolIter, item);
+      }
+
+      if (block == null) {
+        try {
+          item.setErrMsg("No source blocks found to move.");
+          LOG.error("No movable source blocks found. {}", item.toJson());
+        } catch (IOException e) {
+          LOG.error("Unable to get json from Item.");
+        }
+      }
+      return block;
+    }
+
+    /**
+     * Close all Pool Iters.
+     *
+     * @param poolIters List of BlockIters
+     */
+    private void closePoolIters(List<FsVolumeSpi.BlockIterator> poolIters) {
+      Preconditions.checkNotNull(poolIters);
+      for (FsVolumeSpi.BlockIterator iter : poolIters) {
+        try {
+          iter.close();
+        } catch (IOException ex) {
+          LOG.error("Error closing a block pool iter. ex: {}", ex);
+        }
+      }
+    }
+
+    /**
+     * Copies blocks from a set of volumes.
+     *
+     * @param pair - Source and Destination Volumes.
+     * @param item - Number of bytes to move from volumes.
      */
     @Override
-    public void setExitFlag() {
+    public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
+      FsVolumeSpi source = pair.getSource();
+      FsVolumeSpi dest = pair.getDest();
+      List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>();
 
+      if (source.isTransientStorage() || dest.isTransientStorage()) {
+        return;
+      }
+
+      try {
+        openPoolIters(source, poolIters);
+        if (poolIters.size() == 0) {
+          LOG.error("No block pools found on volume. volume : {}. Exiting.",
+              source.getBasePath());
+          return;
+        }
+
+        while (shouldRun()) {
+          try {
+
+            // Check for the max error count constraint.
+            if (item.getErrorCount() > getMaxError(item)) {
+              LOG.error("Exceeded the max error count. source {}, dest: {} " +
+                      "error count: {}", source.getBasePath(),
+                  dest.getBasePath(), item.getErrorCount());
+              this.setExitFlag();
+              continue;
+            }
+
+            // Check for the block tolerance constraint.
+            if (isCloseEnough(item)) {
+              LOG.info("Copy from {} to {} done. copied {} bytes and {} " +
+                      "blocks.",
+                  source.getBasePath(), dest.getBasePath(),
+                  item.getBytesCopied(), item.getBlocksCopied());
+              this.setExitFlag();
+              continue;
+            }
+
+            ExtendedBlock block = getNextBlock(poolIters, item);
+            // we are not able to find any blocks to copy.
+            if (block == null) {
+              this.setExitFlag();
+              LOG.error("No source blocks, exiting the copy. Source: {}, " +
+                      "dest:{}", source.getBasePath(), dest.getBasePath());
+              continue;
+            }
+
+            // check if someone told us exit, treat this as an interruption
+            // point
+            // for the thread, since both getNextBlock and moveBlocAcrossVolume
+            // can take some time.
+            if (!shouldRun()) {
+              continue;
+            }
+
+            long timeUsed;
+            // There is a race condition here, but we will get an IOException
+            // if dest has no space, which we handle anyway.
+            if (dest.getAvailable() > item.getBytesToCopy()) {
+              long begin = System.nanoTime();
+              this.dataset.moveBlockAcrossVolumes(block, dest);
+              long now = System.nanoTime();
+              timeUsed = (now - begin) > 0 ? now - begin : 0;
+            } else {
+
+              // Technically it is possible for us to find a smaller block and
+              // make another copy, but opting for the safer choice of just
+              // exiting here.
+              LOG.error("Destination volume: {} does not have enough space to" +
+                  " accommodate a block. Block Size: {} Exiting from" +
+                  " copyBlocks.", dest.getBasePath(), block.getNumBytes());
+              this.setExitFlag();
+              continue;
+            }
+
+            LOG.debug("Moved block with size {} from  {} to {}",
+                block.getNumBytes(), source.getBasePath(),
+                dest.getBasePath());
+
+            item.incCopiedSoFar(block.getNumBytes());
+            item.incBlocksCopied();
+
+            // Check for the max throughput constraint.
+            // We sleep here to keep the promise that we will not
+            // copy more than Max MB/sec. we sleep enough time
+            // to make sure that our promise is good on average.
+            // Because we sleep, if a shutdown or cancel call comes in
+            // we exit via Thread Interrupted exception.
+            Thread.sleep(computeDelay(block.getNumBytes(), timeUsed, item));
+
+          } catch (IOException ex) {
+            LOG.error("Exception while trying to copy blocks. error: {}", ex);
+            item.incErrorCount();
+          } catch (InterruptedException e) {
+            LOG.error("Copy Block Thread interrupted, exiting the copy.");
+            Thread.currentThread().interrupt();
+            item.incErrorCount();
+            this.setExitFlag();
+          }
+        }
+      } finally {
+        // Close all Iters.
+        closePoolIters(poolIters);
+      }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
index 87030db..f70a983 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
@@ -28,7 +28,7 @@ import java.util.Map;
  * between a set of Nodes.
  */
 public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
-  private float nodeDataDensity;
+  private double nodeDataDensity;
   private Map<String, DiskBalancerVolumeSet> volumeSets;
   private String dataNodeUUID;
   private String dataNodeIP;
@@ -159,17 +159,17 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
   public int compareTo(DiskBalancerDataNode that) {
     Preconditions.checkNotNull(that);
 
-    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+    if (Double.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
         < 0) {
       return -1;
     }
 
-    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+    if (Double.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
         == 0) {
       return 0;
     }
 
-    if (Float.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+    if (Double.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
         > 0) {
       return 1;
     }
@@ -190,7 +190,7 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
    *
    * @return float
    */
-  public float getNodeDataDensity() {
+  public double getNodeDataDensity() {
     return nodeDataDensity;
   }
 
@@ -201,7 +201,7 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
    * spread across a set of volumes inside the node.
    */
   public void computeNodeDensity() {
-    float sum = 0;
+    double sum = 0;
     int volcount = 0;
     for (DiskBalancerVolumeSet vset : volumeSets.values()) {
       for (DiskBalancerVolume vol : vset.getVolumes()) {
@@ -249,6 +249,7 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
       vSet = volumeSets.get(volumeSetKey);
     } else {
       vSet = new DiskBalancerVolumeSet(volume.isTransient());
+      vSet.setStorageType(volumeSetKey);
       volumeSets.put(volumeSetKey, vSet);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
index 24e891f..2a39609 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
@@ -38,7 +38,7 @@ public class DiskBalancerVolume {
   private String uuid;
   private boolean failed;
   private boolean isTransient;
-  private float volumeDataDensity;
+  private double volumeDataDensity;
   private boolean skip = false;
   private boolean isReadOnly;
 
@@ -69,7 +69,7 @@ public class DiskBalancerVolume {
    *
    * @return float.
    */
-  public float getVolumeDataDensity() {
+  public double getVolumeDataDensity() {
     return volumeDataDensity;
   }
 
@@ -78,7 +78,7 @@ public class DiskBalancerVolume {
    *
    * @param volDataDensity - density
    */
-  public void setVolumeDataDensity(float volDataDensity) {
+  public void setVolumeDataDensity(double volDataDensity) {
     this.volumeDataDensity = volDataDensity;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
index 2faf249..70d7536 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
@@ -53,7 +53,7 @@ public class DiskBalancerVolumeSet {
   private String storageType;
   private String setID;
 
-  private float idealUsed;
+  private double idealUsed;
 
 
   /**
@@ -142,19 +142,32 @@ public class DiskBalancerVolumeSet {
     }
 
     if (totalCapacity != 0) {
-      this.idealUsed = totalUsed / (float) totalCapacity;
+      this.idealUsed = truncateDecimals(totalUsed /
+          (double) totalCapacity);
     }
 
     for (DiskBalancerVolume volume : volumes) {
       if (!volume.isFailed() && !volume.isSkip()) {
-        float dfsUsedRatio =
-            volume.getUsed() / (float) volume.computeEffectiveCapacity();
+        double dfsUsedRatio =
+            truncateDecimals(volume.getUsed() /
+                (double) volume.computeEffectiveCapacity());
+
         volume.setVolumeDataDensity(this.idealUsed - dfsUsedRatio);
         sortedQueue.add(volume);
       }
     }
   }
 
+  /**
+   * Truncate to 4 digits since uncontrolled precision is some times
+   * counter intitive to what users expect.
+   * @param value - double.
+   * @return double.
+   */
+  private double truncateDecimals(double value) {
+    final int multiplier = 10000;
+    return (double) ((long) (value * multiplier)) / multiplier;
+  }
   private void skipMisConfiguredVolume(DiskBalancerVolume volume) {
     //probably points to some sort of mis-configuration. Log this and skip
     // processing this volume.
@@ -255,7 +268,7 @@ public class DiskBalancerVolumeSet {
    * @return true if balancing is needed false otherwise.
    */
   public boolean isBalancingNeeded(float thresholdPercentage) {
-    float threshold = thresholdPercentage / 100.0f;
+    double threshold = thresholdPercentage / 100.0d;
 
     if(volumes == null || volumes.size() <= 1) {
       // there is nothing we can do with a single volume.
@@ -265,7 +278,10 @@ public class DiskBalancerVolumeSet {
 
     for (DiskBalancerVolume vol : volumes) {
       boolean notSkip = !vol.isFailed() && !vol.isTransient() && !vol.isSkip();
-      if ((Math.abs(vol.getVolumeDataDensity()) > threshold) && notSkip) {
+      Double absDensity =
+          truncateDecimals(Math.abs(vol.getVolumeDataDensity()));
+
+      if ((absDensity > threshold) && notSkip) {
         return true;
       }
     }
@@ -306,7 +322,7 @@ public class DiskBalancerVolumeSet {
    */
 
   @JsonIgnore
-  public float getIdealUsed() {
+  public double getIdealUsed() {
     return this.idealUsed;
   }
 
@@ -319,8 +335,8 @@ public class DiskBalancerVolumeSet {
      */
     @Override
     public int compare(DiskBalancerVolume first, DiskBalancerVolume second) {
-      return Float
-          .compare(second.getVolumeDataDensity(), first.getVolumeDataDensity());
+      return Double.compare(second.getVolumeDataDensity(),
+          first.getVolumeDataDensity());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
index 9a493a5..b5f68fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
@@ -38,7 +38,7 @@ import org.apache.htrace.fasterxml.jackson.annotation.JsonInclude;
 public class MoveStep implements Step {
   private DiskBalancerVolume sourceVolume;
   private DiskBalancerVolume destinationVolume;
-  private float idealStorage;
+  private double idealStorage;
   private long bytesToMove;
   private String volumeSetID;
 
@@ -55,7 +55,7 @@ public class MoveStep implements Step {
    * @param bytesToMove       - number of bytes to move
    * @param volumeSetID       - a diskBalancer generated id.
    */
-  public MoveStep(DiskBalancerVolume sourceVolume, float idealStorage,
+  public MoveStep(DiskBalancerVolume sourceVolume, double idealStorage,
                   DiskBalancerVolume destinationVolume, long bytesToMove,
                   String volumeSetID) {
     this.destinationVolume = destinationVolume;
@@ -98,7 +98,7 @@ public class MoveStep implements Step {
    * @return float
    */
   @Override
-  public float getIdealStorage() {
+  public double getIdealStorage() {
     return idealStorage;
   }
 
@@ -146,7 +146,7 @@ public class MoveStep implements Step {
    *
    * @param idealStorage - ideal Storage
    */
-  public void setIdealStorage(float idealStorage) {
+  public void setIdealStorage(double idealStorage) {
     this.idealStorage = idealStorage;
   }
 
@@ -199,6 +199,7 @@ public class MoveStep implements Step {
    * move operation is aborted.
    * @return  long.
    */
+  @Override
   public long getMaxDiskErrors() {
     return maxDiskErrors;
   }
@@ -208,6 +209,7 @@ public class MoveStep implements Step {
    * step is aborted.
    * @param maxDiskErrors - long
    */
+  @Override
   public void setMaxDiskErrors(long maxDiskErrors) {
     this.maxDiskErrors = maxDiskErrors;
   }
@@ -223,6 +225,7 @@ public class MoveStep implements Step {
    *
    * @return tolerance percentage.
    */
+  @Override
   public long getTolerancePercent() {
     return tolerancePercent;
   }
@@ -231,6 +234,7 @@ public class MoveStep implements Step {
    * Sets the tolerance percentage.
    * @param tolerancePercent  - long
    */
+  @Override
   public void setTolerancePercent(long tolerancePercent) {
     this.tolerancePercent = tolerancePercent;
   }
@@ -241,6 +245,7 @@ public class MoveStep implements Step {
    * datanode while data node is in use.
    * @return  long.
    */
+  @Override
   public long getBandwidth() {
     return bandwidth;
   }
@@ -250,6 +255,7 @@ public class MoveStep implements Step {
    * @param bandwidth  - Long, MB / Sec of data to be moved between
    *                   source and destinatin volume.
    */
+  @Override
   public void setBandwidth(long bandwidth) {
     this.bandwidth = bandwidth;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
index f13909f..8f69653 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
@@ -42,7 +42,7 @@ public interface Step {
    *
    * @return idealStorage
    */
-  float getIdealStorage();
+  double getIdealStorage();
 
   /**
    * Gets Source Volume.
@@ -87,5 +87,23 @@ public interface Step {
    */
   long getBandwidth();
 
+  /**
+   * Sets Tolerance percent on a specific step.
+   * @param tolerancePercent - tolerance in percentage.
+   */
+  void setTolerancePercent(long tolerancePercent);
+
+  /**
+   * Set Bandwidth on a specific step.
+   * @param bandwidth - in MB/s
+   */
+  void setBandwidth(long bandwidth);
+
+  /**
+   * Set maximum errors to tolerate before disk balancer step fails.
+   * @param maxDiskErrors - error count.
+   */
+  void setMaxDiskErrors(long maxDiskErrors);
+
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 738cfe6..1cfd488 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -264,7 +264,8 @@ public class TestBalancer {
   }
 
   /* create a file with a length of <code>fileLen</code> */
-  static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
+  public static void createFile(MiniDFSCluster cluster, Path filePath, long
+      fileLen,
       short replicationFactor, int nnIndex)
   throws IOException, InterruptedException, TimeoutException {
     FileSystem fs = cluster.getFileSystem(nnIndex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
new file mode 100644
index 0000000..f50637c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
@@ -0,0 +1,247 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestDiskBalancer {
+
+  @Test
+  public void TestDiskBalancerNameNodeConnectivity() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    final int numDatanodes = 2;
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes).build();
+    try {
+      cluster.waitActive();
+      ClusterConnector nameNodeConnector =
+          ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+      DiskBalancerCluster DiskBalancerCluster = new DiskBalancerCluster
+          (nameNodeConnector);
+      DiskBalancerCluster.readClusterInfo();
+      assertEquals(DiskBalancerCluster.getNodes().size(), numDatanodes);
+      DataNode dnNode = cluster.getDataNodes().get(0);
+      DiskBalancerDataNode dbDnNode =
+          DiskBalancerCluster.getNodeByUUID(dnNode.getDatanodeUuid());
+      assertEquals(dnNode.getDatanodeUuid(), dbDnNode.getDataNodeUUID());
+      assertEquals(dnNode.getDatanodeId().getIpAddr(),
+          dbDnNode.getDataNodeIP());
+      assertEquals(dnNode.getDatanodeId().getHostName(),
+          dbDnNode.getDataNodeName());
+      try (FsDatasetSpi.FsVolumeReferences ref = dnNode.getFSDataset()
+          .getFsVolumeReferences()) {
+        assertEquals(ref.size(), dbDnNode.getVolumeCount());
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * This test simulates a real Data node working with DiskBalancer.
+   *
+   * Here is the overview of this test.
+   *
+   * 1. Write a bunch of blocks and move them to one disk to create imbalance.
+   * 2. Rewrite  the capacity of the disks in DiskBalancer Model so that
+   * planner will produce a move plan.
+   * 3. Execute the move plan and wait unitl the plan is done.
+   * 4. Verify the source disk has blocks now.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void TestDiskBalancerEndToEnd() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    final int DEFAULT_BLOCK_SIZE = 100;
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    final int numDatanodes = 1;
+    final String fileName = "/tmp.txt";
+    final Path filePath = new Path(fileName);
+    final int blocks = 100;
+    final int blocksSize = 1024;
+    final int fileLen = blocks * blocksSize;
+
+
+    // Write a file and restart the cluster
+    long [] capacities = new long[]{ DEFAULT_BLOCK_SIZE * 2 * fileLen,
+        DEFAULT_BLOCK_SIZE * 2 * fileLen };
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(numDatanodes)
+        .storageCapacities(capacities)
+        .storageTypes(new StorageType[] {StorageType.DISK, StorageType.DISK})
+        .storagesPerDatanode(2)
+        .build();
+    FsVolumeImpl source = null;
+    FsVolumeImpl dest = null;
+    try {
+      cluster.waitActive();
+      Random r = new Random();
+      FileSystem fs = cluster.getFileSystem(0);
+      TestBalancer.createFile(cluster, filePath, fileLen, (short) 1,
+          numDatanodes - 1);
+
+      DFSTestUtil.waitReplication(fs, filePath, (short) 1);
+      cluster.restartDataNodes();
+      cluster.waitActive();
+
+      // Get the data node and move all data to one disk.
+      DataNode dnNode = cluster.getDataNodes().get(numDatanodes - 1);
+      try (FsDatasetSpi.FsVolumeReferences refs =
+               dnNode.getFSDataset().getFsVolumeReferences()) {
+        source = (FsVolumeImpl) refs.get(0);
+        dest = (FsVolumeImpl) refs.get(1);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) > 0);
+        DiskBalancerTestUtil.moveAllDataToDestVolume(
+            dnNode.getFSDataset(), source, dest);
+       assertTrue(DiskBalancerTestUtil.getBlockCount(source) == 0);
+      }
+
+      cluster.restartDataNodes();
+      cluster.waitActive();
+
+      // Start up a disk balancer and read the cluster info.
+      final DataNode newDN = cluster.getDataNodes().get(numDatanodes - 1);
+      ClusterConnector nameNodeConnector =
+          ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+      DiskBalancerCluster diskBalancerCluster =
+          new DiskBalancerCluster(nameNodeConnector);
+      diskBalancerCluster.readClusterInfo();
+      List<DiskBalancerDataNode> nodesToProcess = new LinkedList<>();
+
+      // Rewrite the capacity in the model to show that disks need
+      // re-balancing.
+      setVolumeCapacity(diskBalancerCluster, DEFAULT_BLOCK_SIZE * 2 * fileLen, "DISK");
+      // Pick a node to process.
+      nodesToProcess.add(diskBalancerCluster.getNodeByUUID(dnNode
+          .getDatanodeUuid()));
+      diskBalancerCluster.setNodesToProcess(nodesToProcess);
+
+      // Compute a plan.
+      List<NodePlan> clusterplan = diskBalancerCluster.computePlan(0.0f);
+
+      // Now we must have a plan,since the node is imbalanced and we
+      // asked the disk balancer to create a plan.
+      assertTrue(clusterplan.size() == 1);
+
+      NodePlan plan = clusterplan.get(0);
+      plan.setNodeUUID(dnNode.getDatanodeUuid());
+      plan.setTimeStamp(Time.now());
+      String planJson = plan.toJson();
+      String planID = DigestUtils.sha512Hex(planJson);
+      assertNotNull(plan.getVolumeSetPlans());
+      assertTrue(plan.getVolumeSetPlans().size() > 0);
+      plan.getVolumeSetPlans().get(0).setTolerancePercent(10);
+
+
+      // Submit the plan and wait till the execution is done.
+      newDN.submitDiskBalancerPlan(planID, 1, planJson, false);
+
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            return newDN.queryDiskBalancerPlan().getResult() ==
+                DiskBalancerWorkStatus.Result.PLAN_DONE;
+          } catch (IOException ex) {
+            return false;
+          }
+        }
+      }, 1000, 100000);
+
+
+      //verify that it worked.
+      dnNode = cluster.getDataNodes().get(numDatanodes - 1);
+      assertEquals(dnNode.queryDiskBalancerPlan().getResult(),
+          DiskBalancerWorkStatus.Result.PLAN_DONE);
+      try (FsDatasetSpi.FsVolumeReferences refs =
+               dnNode.getFSDataset().getFsVolumeReferences()) {
+        source = (FsVolumeImpl) refs.get(0);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) > 0);
+      }
+
+
+
+      // Tolerance
+      long delta = (plan.getVolumeSetPlans().get(0).getBytesToMove()
+          * 10) / 100;
+      assertTrue(
+          (DiskBalancerTestUtil.getBlockCount(source) *
+              DEFAULT_BLOCK_SIZE + delta) >=
+              plan.getVolumeSetPlans().get(0).getBytesToMove());
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Sets alll Disks capacity to size specified.
+   * @param cluster   - DiskBalancerCluster
+   * @param size   - new size of the disk
+   */
+  private void setVolumeCapacity(DiskBalancerCluster cluster, long size,
+                                 String diskType) {
+    Preconditions.checkNotNull(cluster);
+    for(DiskBalancerDataNode node : cluster.getNodes()) {
+      for (DiskBalancerVolume vol :
+          node.getVolumeSets().get(diskType).getVolumes()) {
+        vol.setCapacity(size);
+      }
+      node.getVolumeSets().get(diskType).computeVolumeDataDensity();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1594b472/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
index ad18075..77c2aa3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
@@ -21,11 +21,9 @@ import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
-    .DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
-    .DiskBalancerVolumeSet;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
@@ -143,9 +141,9 @@ public class TestPlanner {
     cluster.readClusterInfo();
     Assert.assertEquals(1, cluster.getNodes().size());
 
-    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    GreedyPlanner planner = new GreedyPlanner(5.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeUUID(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     // We should have only one planned move from
@@ -184,8 +182,8 @@ public class TestPlanner {
     cluster.readClusterInfo();
     Assert.assertEquals(1, cluster.getNodes().size());
 
-    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+    GreedyPlanner planner = new GreedyPlanner(5.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeUUID(), node.getDataNodePort
         ());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
@@ -262,11 +260,10 @@ public class TestPlanner {
     assertEquals(2, plan.getVolumeSetPlans().size());
     Step step = plan.getVolumeSetPlans().get(0);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
-
+    assertTrue(step.getSizeString(step.getBytesToMove()).matches("33.[2|3|4] G"));
     step = plan.getVolumeSetPlans().get(1);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
+    assertTrue(step.getSizeString(step.getBytesToMove()).matches("33.[2|3|4] G"));
   }
 
   @Test
@@ -318,11 +315,12 @@ public class TestPlanner {
 
     Step step = newPlan.getVolumeSetPlans().get(0);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
+    assertTrue(step.getSizeString(step.getBytesToMove()).matches("18.[6|7|8] G"));
 
     step = newPlan.getVolumeSetPlans().get(1);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
+    assertTrue(step.getSizeString(step.getBytesToMove()).matches("18.[6|7|8] G"));
+
   }
 
   @Test
@@ -364,7 +362,7 @@ public class TestPlanner {
 
       if (step.getDestinationVolume().getPath().equals("volume0-1")) {
         assertEquals("volume100", step.getSourceVolume().getPath());
-        assertEquals("28.6 G",
+        assertEquals("28.5 G",
             step.getSizeString(step.getBytesToMove()));
       }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/49] hadoop git commit: HDFS-9681. DiskBalancer: Add QueryPlan implementation. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9681. DiskBalancer: Add QueryPlan implementation. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e646c2eb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e646c2eb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e646c2eb

Branch: refs/heads/trunk
Commit: e646c2eb50b9ae2b0b084d78a4ea68e106804321
Parents: 2b1b2fa
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Feb 24 16:49:30 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../ClientDatanodeProtocolTranslatorPB.java     |  11 +-
 .../server/datanode/DiskBalancerWorkStatus.java | 194 +++++++++++++++++--
 .../src/main/proto/ClientDatanodeProtocol.proto |   5 +-
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   5 +-
 ...tDatanodeProtocolServerSideTranslatorPB.java |   5 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   8 +-
 .../hdfs/server/datanode/DiskBalancer.java      |  39 ++++
 .../diskbalancer/TestDiskBalancerRPC.java       |  26 ++-
 8 files changed, 249 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 786d834..7076026 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBa
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
@@ -392,10 +393,14 @@ public class ClientDatanodeProtocolTranslatorPB implements
           QueryPlanStatusRequestProto.newBuilder().build();
       QueryPlanStatusResponseProto response =
           rpcProxy.queryDiskBalancerPlan(NULL_CONTROLLER, request);
-      return new DiskBalancerWorkStatus(response.hasResult() ?
-          response.getResult() : 0,
+      DiskBalancerWorkStatus.Result result = Result.NO_PLAN;
+      if(response.hasResult()) {
+        result = DiskBalancerWorkStatus.Result.values()[
+            response.getResult()];
+      }
+
+      return new DiskBalancerWorkStatus(result,
           response.hasPlanID() ? response.getPlanID() : null,
-          response.hasStatus() ? response.getStatus() : null,
           response.hasCurrentStatus() ? response.getCurrentStatus() : null);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
index 6b29ce8..d6943cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
@@ -19,8 +19,17 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import static org.codehaus.jackson.map.type.TypeFactory.defaultInstance;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.LinkedList;
 
 /**
  * Helper class that reports how much work has has been done by the node.
@@ -28,33 +37,69 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class DiskBalancerWorkStatus {
-  private final int result;
-  private final String planID;
-  private final String status;
-  private final String currentState;
+
+  private final List<DiskBalancerWorkEntry> currentState;
+  private Result result;
+  private String planID;
+
+  /**
+   * Constructs a default workStatus Object.
+   */
+  public DiskBalancerWorkStatus() {
+    this.currentState = new LinkedList<>();
+  }
+
+  /**
+   * Constructs a workStatus Object.
+   *
+   * @param result - int
+   * @param planID - Plan ID
+   */
+  public DiskBalancerWorkStatus(Result result, String planID) {
+    this();
+    this.result = result;
+    this.planID = planID;
+  }
 
   /**
    * Constructs a workStatus Object.
    *
    * @param result       - int
    * @param planID       - Plan ID
-   * @param status       - Current Status
    * @param currentState - Current State
    */
-  public DiskBalancerWorkStatus(int result, String planID, String status,
-                                String currentState) {
+  public DiskBalancerWorkStatus(Result result, String planID,
+                                List<DiskBalancerWorkEntry> currentState) {
     this.result = result;
     this.planID = planID;
-    this.status = status;
     this.currentState = currentState;
   }
 
+
+  /**
+   * Constructs a workStatus Object.
+   *
+   * @param result       - int
+   * @param planID       - Plan ID
+   * @param currentState - List of WorkEntries.
+   */
+  public DiskBalancerWorkStatus(Result result, String planID,
+                                String currentState) throws IOException {
+    this.result = result;
+    this.planID = planID;
+    ObjectMapper mapper = new ObjectMapper();
+    this.currentState = mapper.readValue(currentState,
+        defaultInstance().constructCollectionType(
+            List.class, DiskBalancerWorkEntry.class));
+  }
+
+
   /**
    * Returns result.
    *
    * @return long
    */
-  public int getResult() {
+  public Result getResult() {
     return result;
   }
 
@@ -68,20 +113,135 @@ public class DiskBalancerWorkStatus {
   }
 
   /**
-   * Returns Status.
+   * Gets current Status.
    *
-   * @return String
+   * @return - Json String
    */
-  public String getStatus() {
-    return status;
+  public List<DiskBalancerWorkEntry> getCurrentState() {
+    return currentState;
   }
 
   /**
-   * Gets current Status.
+   * Return current state as a string.
    *
-   * @return - Json String
+   * @throws IOException
+   **/
+  public String getCurrentStateString() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(currentState);
+  }
+
+
+  /**
+   * Adds a new work entry to the list.
+   *
+   * @param entry - DiskBalancerWorkEntry
    */
-  public String getCurrentState() {
-    return currentState;
+
+  public void addWorkEntry(DiskBalancerWorkEntry entry) {
+    Preconditions.checkNotNull(entry);
+    currentState.add(entry);
+  }
+
+  /** Various result values. **/
+  public enum Result {
+    NO_PLAN(0),
+    PLAN_UNDER_PROGRESS(1),
+    PLAN_DONE(2),
+    PLAN_CANCELLED(3);
+    private int result;
+
+    private Result(int result) {
+      this.result = result;
+    }
+
+    /**
+     * Get int value of result.
+     *
+     * @return int
+     */
+    public int getIntResult() {
+      return result;
+    }
+  }
+
+  /**
+   * A class that is used to report each work item that we are working on. This
+   * class describes the Source, Destination and how much data has been already
+   * moved, errors encountered etc. This is useful for the disk balancer stats
+   * as well as the queryStatus RPC.
+   */
+  public static class DiskBalancerWorkEntry {
+    private String sourcePath;
+    private String destPath;
+    private DiskBalancerWorkItem workItem;
+
+    /**
+     * Constructs a Work Entry class.
+     *
+     * @param sourcePath - Source Path where we are moving data from.
+     * @param destPath   - Destination path to where we are moving data to.
+     * @param workItem   - Current work status of this move.
+     */
+    public DiskBalancerWorkEntry(String sourcePath, String destPath,
+                                 DiskBalancerWorkItem workItem) {
+      this.sourcePath = sourcePath;
+      this.destPath = destPath;
+      this.workItem = workItem;
+    }
+
+    /**
+     * Returns the source path.
+     *
+     * @return - Source path
+     */
+    public String getSourcePath() {
+      return sourcePath;
+    }
+
+    /**
+     * Sets the Source Path.
+     *
+     * @param sourcePath - Volume Path.
+     */
+    public void setSourcePath(String sourcePath) {
+      this.sourcePath = sourcePath;
+    }
+
+    /**
+     * Gets the Destination path.
+     *
+     * @return - Path
+     */
+    public String getDestPath() {
+      return destPath;
+    }
+
+    /**
+     * Sets the destination path.
+     *
+     * @param destPath - Path
+     */
+    public void setDestPath(String destPath) {
+      this.destPath = destPath;
+    }
+
+    /**
+     * Gets the current status of work for these volumes.
+     *
+     * @return - Work Item
+     */
+    public DiskBalancerWorkItem getWorkItem() {
+      return workItem;
+    }
+
+    /**
+     * Sets the work item.
+     *
+     * @param workItem - sets the work item information
+     */
+    public void setWorkItem(DiskBalancerWorkItem workItem) {
+      this.workItem = workItem;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index c61c700..c7cd4fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -194,9 +194,8 @@ message QueryPlanStatusRequestProto {
  */
 message QueryPlanStatusResponseProto {
   optional uint32 result = 1;
-  optional string status = 2;
-  optional string planID = 3;
-  optional string currentStatus = 4;
+  optional string planID = 2;
+  optional string currentStatus = 3;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 27de7d0..07403cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -24,6 +24,9 @@ HDFS-1312 Change Log
     HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer via
     Arpit Agarwal)
 
-    HDFS-9671skBalancer: SubmitPlan implementation. (Anu Engineer via
+    HDFS-9671. DiskBalancer: SubmitPlan implementation. (Anu Engineer via
+    Arpit Agarwal)
+
+    HDFS-9681. DiskBalancer: Add QueryPlan implementation. (Anu Engineer via
     Arpit Agarwal)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 692fca3..d72a060 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -296,10 +296,9 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       DiskBalancerWorkStatus result = impl.queryDiskBalancerPlan();
       return QueryPlanStatusResponseProto
           .newBuilder()
-          .setResult(result.getResult())
+          .setResult(result.getResult().getIntResult())
           .setPlanID(result.getPlanID())
-          .setStatus(result.getStatus())
-          .setCurrentStatus(result.getCurrentState())
+          .setCurrentStatus(result.getCurrentStateString())
           .build();
     } catch (Exception e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 8d805a1..56585a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3345,11 +3345,15 @@ public class DataNode extends ReconfigurableBase
         DiskBalancerException.Result.INTERNAL_ERROR);
   }
 
+  /**
+   * Returns the status of current or last executed work plan.
+   * @return DiskBalancerWorkStatus.
+   * @throws IOException
+   */
   @Override
   public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException {
     checkSuperuserPrivilege();
-    throw new DiskBalancerException("Not Implemented",
-        DiskBalancerException.Result.INTERNAL_ERROR);
+    return this.diskBalancer.queryWorkStatus();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 1c8ba4cf..c01fb4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -25,6 +25,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.DiskBalancerWorkEntry;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
@@ -68,6 +70,7 @@ public class DiskBalancer {
   private ExecutorService scheduler;
   private Future future;
   private String planID;
+  private DiskBalancerWorkStatus.Result currentResult;
 
   /**
    * Constructs a Disk Balancer object. This object takes care of reading a
@@ -79,6 +82,7 @@ public class DiskBalancer {
    */
   public DiskBalancer(String dataNodeUUID,
                       Configuration conf, BlockMover blockMover) {
+    this.currentResult = Result.NO_PLAN;
     this.blockMover = blockMover;
     this.dataset = this.blockMover.getDataset();
     this.dataNodeUUID = dataNodeUUID;
@@ -97,6 +101,7 @@ public class DiskBalancer {
     lock.lock();
     try {
       this.isDiskBalancerEnabled = false;
+      this.currentResult = Result.NO_PLAN;
       if ((this.future != null) && (!this.future.isDone())) {
         this.blockMover.setExitFlag();
         shutdownExecutor();
@@ -151,6 +156,7 @@ public class DiskBalancer {
           verifyPlan(planID, planVersion, plan, bandwidth, force);
       createWorkPlan(nodePlan);
       this.planID = planID;
+      this.currentResult = Result.PLAN_UNDER_PROGRESS;
       executePlan();
     } finally {
       lock.unlock();
@@ -158,6 +164,39 @@ public class DiskBalancer {
   }
 
   /**
+   * Returns the Current Work Status of a submitted Plan.
+   *
+   * @return DiskBalancerWorkStatus.
+   * @throws DiskBalancerException
+   */
+  public DiskBalancerWorkStatus queryWorkStatus() throws DiskBalancerException {
+    lock.lock();
+    try {
+      checkDiskBalancerEnabled();
+      // if we had a plan in progress, check if it is finished.
+      if (this.currentResult == Result.PLAN_UNDER_PROGRESS &&
+          this.future != null &&
+          this.future.isDone()) {
+        this.currentResult = Result.PLAN_DONE;
+      }
+
+      DiskBalancerWorkStatus status =
+          new DiskBalancerWorkStatus(this.currentResult, this.planID);
+      for (Map.Entry<VolumePair, DiskBalancerWorkItem> entry :
+          workMap.entrySet()) {
+        DiskBalancerWorkEntry workEntry = new DiskBalancerWorkEntry(
+            entry.getKey().getSource().getBasePath(),
+            entry.getKey().getDest().getBasePath(),
+            entry.getValue());
+        status.addWorkEntry(workEntry);
+      }
+      return status;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
    * Throws if Disk balancer is disabled.
    *
    * @throws DiskBalancerException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e646c2eb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index dc24787..974e973 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
@@ -36,6 +37,9 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
+
 public class TestDiskBalancerRPC {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
@@ -134,28 +138,20 @@ public class TestDiskBalancerRPC {
     Assert.assertEquals(cluster.getDataNodes().size(),
         diskBalancerCluster.getNodes().size());
     diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
-    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
+        dataNode.getDatanodeUuid());
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
     NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
         ());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
 
-    final int planVersion = 0; // So far we support only one version.
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    final int planVersion = 1; // So far we support only one version.
     String planHash = DigestUtils.sha512Hex(plan.toJson());
-
-    // Since submitDiskBalancerPlan is not implemented yet, it throws an
-    // Exception, this will be modified with the actual implementation.
-    try {
       dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-    } catch (DiskBalancerException ex) {
-      // Let us ignore this for time being.
-    }
-
-    // TODO : This will be fixed when we have implementation for this
-    // function in server side.
-    thrown.expect(DiskBalancerException.class);
-    dataNode.queryDiskBalancerPlan();
+    DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
+    Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
+        status.getResult() == PLAN_DONE);
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/49] hadoop git commit: HDFS-9545: DiskBalancer: Add Plan Command. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9545: DiskBalancer: Add Plan Command. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/75882ec0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/75882ec0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/75882ec0

Branch: refs/heads/trunk
Commit: 75882ec0b096da862b8b373b70a091c19f281b2a
Parents: 1594b47
Author: Anu Engineer <ae...@apache.org>
Authored: Mon May 9 10:17:56 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/hdfs               |   6 +
 .../server/diskbalancer/command/Command.java    | 381 +++++++++++++++++++
 .../diskbalancer/command/PlanCommand.java       | 217 +++++++++++
 .../diskbalancer/command/package-info.java      |  22 ++
 .../datamodel/DiskBalancerCluster.java          |  89 +++--
 .../datamodel/DiskBalancerDataNode.java         |   2 +-
 .../datamodel/DiskBalancerVolumeSet.java        |   2 +-
 .../diskbalancer/planner/GreedyPlanner.java     |   4 +-
 .../diskbalancer/planner/PlannerFactory.java    |   6 +-
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 260 +++++++++++++
 .../TestDiskBalancerWithMockMover.java          |   2 -
 11 files changed, 945 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 2a29d17..7952560 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -39,6 +39,7 @@ function hadoop_usage
   hadoop_add_subcommand "debug" "run a Debug Admin to execute HDFS debug commands"
   hadoop_add_subcommand "dfs" "run a filesystem command on the file system"
   hadoop_add_subcommand "dfsadmin" "run a DFS admin client"
+  hadoop_add_subcommand "diskbalancer" "Distributes data evenly among disks on a given node"
   hadoop_add_subcommand "envvars" "display computed Hadoop environment variables"
   hadoop_add_subcommand "erasurecode" "run a HDFS ErasureCoding CLI"
   hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode"
@@ -125,6 +126,11 @@ function hdfscmd_case
       hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
       HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
     ;;
+    diskbalancer)
+      HADOOP_CLASSNAME=org.apache.hadoop.hdfs.tools.DiskBalancer
+      hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS"
+      HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}"
+    ;;
     envvars)
       echo "JAVA_HOME='${JAVA_HOME}'"
       echo "HADOOP_HDFS_HOME='${HADOOP_HDFS_HOME}'"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
new file mode 100644
index 0000000..6522434
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -0,0 +1,381 @@
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.text.SimpleDateFormat;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.List;
+
+/**
+ * Common interface for command handling.
+ */
+public abstract class Command extends Configured {
+  static final Logger LOG = LoggerFactory.getLogger(Command.class);
+  private Map<String, String> validArgs = new HashMap<>();
+  private URI clusterURI;
+  private FileSystem fs = null;
+  private DiskBalancerCluster cluster = null;
+
+  private static final Path DEFAULT_LOG_DIR = new Path("/system/diskbalancer");
+
+  private Path diskBalancerLogs;
+
+  /**
+   * Constructs a command.
+   */
+  public Command(Configuration conf) {
+    super(conf);
+    // These arguments are valid for all commands.
+    addValidCommandParameters(DiskBalancer.NAMENODEURI, "Name Node URI or " +
+        "file URI for cluster");
+    addValidCommandParameters(DiskBalancer.HELP, "Help for this command");
+    addValidCommandParameters("arg", "");
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  public abstract void execute(CommandLine cmd) throws Exception;
+
+  /**
+   * Gets extended help for this command.
+   *
+   * @return Help Message
+   */
+  protected abstract String getHelp();
+
+  /**
+   * verifies user provided URL.
+   *
+   * @param uri - UrlString
+   * @return URL
+   * @throws URISyntaxException, MalformedURLException
+   */
+  protected URI verifyURI(String uri)
+      throws URISyntaxException, MalformedURLException {
+    if ((uri == null) || uri.isEmpty()) {
+      throw new MalformedURLException(
+          "A valid URI is needed to execute this command.");
+    }
+    return new URI(uri);
+  }
+
+  /**
+   * Process the URI and return the cluster with nodes setup. This is used in
+   * all commands.
+   *
+   * @param cmd - CommandLine
+   * @return DiskBalancerCluster
+   * @throws Exception
+   */
+  protected DiskBalancerCluster readClusterInfo(CommandLine cmd) throws
+      Exception {
+    Preconditions.checkNotNull(cmd);
+    Preconditions
+        .checkState(cmd.getOptionValue(DiskBalancer.NAMENODEURI) != null,
+            "Required argument missing : uri");
+
+    setClusterURI(verifyURI(cmd.getOptionValue(DiskBalancer.NAMENODEURI)));
+    LOG.debug("using name node URI : {}", this.getClusterURI());
+    ClusterConnector connector = ConnectorFactory.getCluster(this.clusterURI,
+        getConf());
+
+    cluster = new DiskBalancerCluster(connector);
+
+    LOG.debug("Reading cluster info");
+    cluster.readClusterInfo();
+    return cluster;
+  }
+
+  /**
+   * Setup the outpath.
+   *
+   * @param path - Path or null to use default path.
+   * @throws IOException
+   */
+  protected void setOutputPath(String path) throws IOException {
+
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MMM-dd-HH-mm-ss");
+    Date now = new Date();
+
+    fs = FileSystem.get(getClusterURI(), getConf());
+    if (path == null || path.isEmpty()) {
+      if (getClusterURI().getScheme().startsWith("file")) {
+        diskBalancerLogs = new Path(
+            System.getProperty("user.dir") + DEFAULT_LOG_DIR.toString() +
+                format.format(now));
+      } else {
+        diskBalancerLogs = new Path(DEFAULT_LOG_DIR.toString() +
+            format.format(now));
+      }
+    } else {
+      diskBalancerLogs = new Path(path);
+    }
+    if (fs.exists(diskBalancerLogs)) {
+      LOG.error("Another Diskbalancer instance is running ? - Target " +
+          "Directory already exists. {}", diskBalancerLogs);
+      throw new IOException("Another DiskBalancer files already exist at the " +
+          "target location. " + diskBalancerLogs.toString());
+    }
+    fs.mkdirs(diskBalancerLogs);
+  }
+
+  /**
+   * Sets the nodes to process.
+   *
+   * @param node - Node
+   */
+  protected void setNodesToProcess(DiskBalancerDataNode node) {
+    List<DiskBalancerDataNode> nodelist = new LinkedList<>();
+    nodelist.add(node);
+    setNodesToProcess(nodelist);
+  }
+
+  /**
+   * Sets the list of Nodes to process.
+   *
+   * @param nodes Nodes.
+   */
+  protected void setNodesToProcess(List<DiskBalancerDataNode> nodes) {
+    if (cluster == null) {
+      throw new IllegalStateException("Set nodes to process invoked before " +
+          "initializing cluster. Illegal usage.");
+    }
+    cluster.setNodesToProcess(nodes);
+  }
+
+  /**
+   * Returns a DiskBalancer Node from the Cluster or null if not found.
+   *
+   * @param nodeName - can the hostname, IP address or UUID of the node.
+   * @return - DataNode if found.
+   */
+  DiskBalancerDataNode getNode(String nodeName) {
+    DiskBalancerDataNode node = null;
+    if (nodeName == null || nodeName.isEmpty()) {
+      return node;
+    }
+    if (cluster.getNodes().size() == 0) {
+      return node;
+    }
+
+    node = cluster.getNodeByName(nodeName);
+    if (node != null) {
+      return node;
+    }
+
+    node = cluster.getNodeByIPAddress(nodeName);
+    if (node != null) {
+      return node;
+    }
+    node = cluster.getNodeByUUID(nodeName);
+    return node;
+  }
+
+  /**
+   * Gets the node set from a file or a string.
+   *
+   * @param listArg - String File URL or a comma separated list of node names.
+   * @return Set of node names
+   * @throws IOException
+   */
+  private Set<String> getNodeList(String listArg) throws IOException {
+    URL listURL;
+    String nodeData;
+    Set<String> resultSet = new TreeSet<>();
+
+    if ((listArg == null) || listArg.isEmpty()) {
+      return resultSet;
+    }
+    if (listArg.startsWith("file://")) {
+      listURL = new URL(listArg);
+      byte[] data = Files.readAllBytes(Paths.get(listURL.getPath()));
+      nodeData = new String(data, Charset.forName("UTF-8"));
+    } else {
+      nodeData = listArg;
+    }
+
+    String[] nodes = nodeData.split(",");
+    Collections.addAll(resultSet, nodes);
+    return resultSet;
+  }
+
+  /**
+   * Verifies if the command line options are sane.
+   *
+   * @param commandName - Name of the command
+   * @param cmd         - Parsed Command Line
+   */
+  protected void verifyCommandOptions(String commandName, CommandLine cmd) {
+    @SuppressWarnings("unchecked")
+    Iterator<Option> iter = cmd.iterator();
+    while (iter.hasNext()) {
+      Option opt = iter.next();
+      if (!validArgs.containsKey(opt.getArgName())) {
+        String errMessage = String
+            .format("%nInvalid argument found for command %s : %s%n",
+                commandName, opt.getArgName());
+        StringBuilder validArguments = new StringBuilder();
+        validArguments.append("Valid arguments are : %n");
+        for (Map.Entry<String, String> args : validArgs.entrySet()) {
+          String key = args.getKey();
+          String desc = args.getValue();
+          String s = String.format("\t %s : %s %n", key, desc);
+          validArguments.append(s);
+        }
+        LOG.error(errMessage + validArguments.toString());
+        throw new IllegalArgumentException("Invalid Arguments found.");
+      }
+    }
+  }
+
+  /**
+   * Gets cluster URL.
+   *
+   * @return - URL
+   */
+  public URI getClusterURI() {
+    return clusterURI;
+  }
+
+  /**
+   * Set cluster URL.
+   *
+   * @param clusterURI - URL
+   */
+  public void setClusterURI(URI clusterURI) {
+    this.clusterURI = clusterURI;
+  }
+
+  /**
+   * Copied from DFSAdmin.java. -- Creates a connection to dataNode.
+   *
+   * @param datanode - dataNode.
+   * @return ClientDataNodeProtocol
+   * @throws IOException
+   */
+  public ClientDatanodeProtocol getDataNodeProxy(String datanode)
+      throws IOException {
+    InetSocketAddress datanodeAddr = NetUtils.createSocketAddr(datanode);
+
+    // For datanode proxy the server principal should be DN's one.
+    getConf().set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
+        getConf().get(DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, ""));
+
+    // Create the client
+    ClientDatanodeProtocol dnProtocol =
+        DFSUtilClient.createClientDatanodeProtocolProxy(datanodeAddr, getUGI(),
+            getConf(), NetUtils.getSocketFactory(getConf(),
+                ClientDatanodeProtocol
+                    .class));
+    return dnProtocol;
+  }
+
+  /**
+   * Returns UGI.
+   *
+   * @return UserGroupInformation.
+   * @throws IOException
+   */
+  private static UserGroupInformation getUGI()
+      throws IOException {
+    return UserGroupInformation.getCurrentUser();
+  }
+
+  /**
+   * Returns a file created in the cluster.
+   *
+   * @param fileName - fileName to open.
+   * @return OutputStream.
+   */
+  protected FSDataOutputStream create(String fileName) throws IOException {
+    return fs.create(new Path(this.diskBalancerLogs, fileName));
+  }
+
+  /**
+   * Returns the output path where the plan and snapshot gets written.
+   *
+   * @return Path
+   */
+  protected Path getOutputPath() {
+    return diskBalancerLogs;
+  }
+
+  /**
+   * Adds valid params to the valid args table.
+   *
+   * @param key
+   * @param desc
+   */
+  protected void addValidCommandParameters(String key, String desc) {
+    validArgs.put(key, desc);
+  }
+
+  /**
+   * Returns the cluster.
+   *
+   * @return Cluster.
+   */
+  protected DiskBalancerCluster getCluster() {
+    return cluster;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
new file mode 100644
index 0000000..2422215
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -0,0 +1,217 @@
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * Class that implements Plan Command.
+ * <p>
+ * Plan command reads the Cluster Info and creates a plan for specified data
+ * node or a set of Data nodes.
+ * <p>
+ * It writes the output to a default location unless changed by the user.
+ */
+public class PlanCommand extends Command {
+  private double thresholdPercentage;
+  private int bandwidth;
+  private int maxError;
+
+  /**
+   * Constructs a plan command.
+   */
+  public PlanCommand(Configuration conf) {
+    super(conf);
+    this.thresholdPercentage = 1;
+    this.bandwidth = 0;
+    this.maxError = 0;
+    addValidCommandParameters(DiskBalancer.OUTFILE, "Output file");
+    addValidCommandParameters(DiskBalancer.BANDWIDTH, "Maximum Bandwidth to " +
+        "be used while copying.");
+    addValidCommandParameters(DiskBalancer.THRESHOLD, "Percentage skew that " +
+        "we tolerate before diskbalancer starts working.");
+    addValidCommandParameters(DiskBalancer.MAXERROR, "Max errors to tolerate " +
+        "between 2 disks");
+    addValidCommandParameters(DiskBalancer.NODE, "Name / Address of the node.");
+    addValidCommandParameters(DiskBalancer.VERBOSE, "Run plan command in " +
+        "verbose mode.");
+  }
+
+  /**
+   * Runs the plan command. This command can be run with various options like
+   * <p>
+   * -plan -node IP -plan -node hostName -plan -node DatanodeUUID
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.debug("Processing Plan Command.");
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.PLAN));
+    verifyCommandOptions(DiskBalancer.PLAN, cmd);
+
+    if (!cmd.hasOption(DiskBalancer.NODE)) {
+      throw new IllegalArgumentException("A node name is required to create a" +
+          " plan.");
+    }
+
+    if (cmd.hasOption(DiskBalancer.BANDWIDTH)) {
+      this.bandwidth = Integer.parseInt(cmd.getOptionValue(DiskBalancer
+          .BANDWIDTH));
+    }
+
+    if (cmd.hasOption(DiskBalancer.MAXERROR)) {
+      this.maxError = Integer.parseInt(cmd.getOptionValue(DiskBalancer
+          .MAXERROR));
+    }
+
+    readClusterInfo(cmd);
+    String output = null;
+    if (cmd.hasOption(DiskBalancer.OUTFILE)) {
+      output = cmd.getOptionValue(DiskBalancer.OUTFILE);
+    }
+    setOutputPath(output);
+
+    DiskBalancerDataNode node = getNode(cmd.getOptionValue(DiskBalancer.NODE));
+    if (node == null) {
+      throw new IllegalArgumentException("Unable to find the specified node. " +
+          cmd.getOptionValue(DiskBalancer.NODE));
+    }
+    this.thresholdPercentage = getThresholdPercentage(cmd);
+    setNodesToProcess(node);
+
+    List<NodePlan> plans = getCluster().computePlan(this.thresholdPercentage);
+    setPlanParams(plans);
+
+    LOG.info("Writing plan to : {}", getOutputPath());
+    System.out.printf("Writing plan to : %s%n", getOutputPath());
+
+    try(FSDataOutputStream beforeStream = create(String.format(
+        DiskBalancer.BEFORE_TEMPLATE,
+        cmd.getOptionValue(DiskBalancer.NODE)))) {
+      beforeStream.write(getCluster().toJson()
+          .getBytes(StandardCharsets.UTF_8));
+    }
+
+    try(FSDataOutputStream planStream = create(String.format(
+        DiskBalancer.PLAN_TEMPLATE,
+        cmd.getOptionValue(DiskBalancer.NODE)))) {
+      planStream.write(getPlan(plans).getBytes(StandardCharsets.UTF_8));
+    }
+
+    if (cmd.hasOption(DiskBalancer.VERBOSE)) {
+      printToScreen(plans);
+    }
+  }
+
+  /**
+   * Gets extended help for this command.
+   *
+   * @return Help Message
+   */
+  @Override
+  protected String getHelp() {
+    return "This commands creates a disk balancer plan for given datanode";
+  }
+
+  /**
+   * Get Threshold for planning purpose.
+   *
+   * @param cmd - Command Line Argument.
+   * @return double
+   */
+  private double getThresholdPercentage(CommandLine cmd) {
+    Double value = 0.0;
+    if (cmd.hasOption(DiskBalancer.THRESHOLD)) {
+      value = Double.parseDouble(cmd.getOptionValue(DiskBalancer.THRESHOLD));
+    }
+
+    if ((value <= 0.0) || (value > 100.0)) {
+      value = getConf().getDouble(
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT,
+          DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT);
+    }
+    return value;
+  }
+
+  /**
+   * Prints a quick summary of the plan to screen.
+   *
+   * @param plans - List of NodePlans.
+   */
+  static private void printToScreen(List<NodePlan> plans) {
+    System.out.println("\nPlan :\n");
+    System.out.println(StringUtils.repeat("=", 80));
+    System.out.println("Source Disk\t\t Dest.Disk\t\t Move Size\t Type\n ");
+    for (NodePlan plan : plans) {
+      for (Step step : plan.getVolumeSetPlans()) {
+        System.out.println(String.format("%s\t%s\t%s\t%s",
+            step.getSourceVolume().getPath(),
+            step.getDestinationVolume().getPath(),
+            step.getSizeString(step.getBytesToMove()),
+            step.getDestinationVolume().getStorageType()));
+      }
+    }
+
+    System.out.println(StringUtils.repeat("=", 80));
+  }
+
+  /**
+   * Sets user specified plan parameters.
+   *
+   * @param plans - list of plans.
+   */
+  private void setPlanParams(List<NodePlan> plans) {
+    for (NodePlan plan : plans) {
+      for (Step step : plan.getVolumeSetPlans()) {
+        if (this.bandwidth > 0) {
+          step.setBandwidth(this.bandwidth);
+        }
+        if (this.maxError > 0) {
+          step.setMaxDiskErrors(this.maxError);
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns a Json represenation of the plans.
+   *
+   * @param plan - List of plans.
+   * @return String.
+   * @throws IOException
+   */
+  private String getPlan(List<NodePlan> plan) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(plan);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/package-info.java
new file mode 100644
index 0000000..e5494ef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  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.
+ */
+
+/**
+ Commands for disk balancer command line tool.
+ */
+package org.apache.hadoop.hdfs.server.diskbalancer.command;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
index 7b82278..17a6ebb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -31,12 +31,13 @@ import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.Map;
+import java.util.HashMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -76,6 +77,13 @@ public class DiskBalancerCluster {
 
   @JsonIgnore
   private List<DiskBalancerDataNode> nodesToProcess;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode> ipList;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode> hostNames;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode>  hostUUID;
+
   private float threshold;
 
   /**
@@ -85,7 +93,9 @@ public class DiskBalancerCluster {
     nodes = new LinkedList<>();
     exclusionList = new TreeSet<>();
     inclusionList = new TreeSet<>();
-
+    ipList = new HashMap<>();
+    hostNames = new HashMap<>();
+    hostUUID = new HashMap<>();
   }
 
   /**
@@ -95,10 +105,9 @@ public class DiskBalancerCluster {
    * @throws IOException
    */
   public DiskBalancerCluster(ClusterConnector connector) throws IOException {
+    this();
     Preconditions.checkNotNull(connector);
     clusterConnector = connector;
-    exclusionList = new TreeSet<>();
-    inclusionList = new TreeSet<>();
   }
 
   /**
@@ -119,8 +128,25 @@ public class DiskBalancerCluster {
    */
   public void readClusterInfo() throws Exception {
     Preconditions.checkNotNull(clusterConnector);
-    LOG.info("Using connector : " + clusterConnector.getConnectorInfo());
+    LOG.debug("Using connector : {}" , clusterConnector.getConnectorInfo());
     nodes = clusterConnector.getNodes();
+    for(DiskBalancerDataNode node : nodes) {
+
+      if(node.getDataNodeIP()!= null && !node.getDataNodeIP().isEmpty()) {
+        ipList.put(node.getDataNodeIP(), node);
+      }
+
+      if(node.getDataNodeName() != null && !node.getDataNodeName().isEmpty()) {
+        // TODO : should we support Internationalized Domain Names ?
+        // Disk balancer assumes that host names are ascii. If not
+        // end user can always balance the node via IP address or DataNode UUID.
+        hostNames.put(node.getDataNodeName().toLowerCase(Locale.US), node);
+      }
+
+      if(node.getDataNodeUUID() != null && !node.getDataNodeUUID().isEmpty()) {
+        hostUUID.put(node.getDataNodeUUID(), node);
+      }
+    }
   }
 
   /**
@@ -259,30 +285,6 @@ public class DiskBalancerCluster {
   }
 
   /**
-   * Creates an Output directory for the cluster output.
-   *
-   * @throws IOException - On failure to create an new directory
-   */
-  public void createOutPutDirectory() throws IOException {
-    if (Files.exists(Paths.get(this.getOutput()))) {
-      LOG.error("An output directory already exists at this location. Path : " +
-          this.getOutput());
-      throw new IOException(
-          "An output directory already exists at this location. Path : " +
-              this.getOutput());
-    }
-
-    File f = new File(this.getOutput());
-    if (!f.mkdirs()) {
-      LOG.error("Unable to create the output directory. Path : " + this
-          .getOutput());
-      throw new IOException(
-          "Unable to create the output directory. Path : " + this.getOutput());
-    }
-    LOG.info("Output directory created. Path : " + this.getOutput());
-  }
-
-  /**
    * Compute plan takes a node and constructs a planner that creates a plan that
    * we would like to follow.
    * <p/>
@@ -294,7 +296,7 @@ public class DiskBalancerCluster {
    * @param thresholdPercent - in percentage
    * @return list of NodePlans
    */
-  public List<NodePlan> computePlan(float thresholdPercent) {
+  public List<NodePlan> computePlan(double thresholdPercent) {
     List<NodePlan> planList = new LinkedList<>();
 
     if (nodesToProcess == null) {
@@ -366,11 +368,24 @@ public class DiskBalancerCluster {
    * @return DiskBalancerDataNode.
    */
   public DiskBalancerDataNode getNodeByUUID(String uuid) {
-    for(DiskBalancerDataNode node : this.getNodes()) {
-      if(node.getDataNodeUUID().equals(uuid)) {
-        return node;
-      }
-    }
-    return null;
+    return hostUUID.get(uuid);
+  }
+
+  /**
+   * Returns a node by IP Address.
+   * @param ipAddresss - IP address String.
+   * @return DiskBalancerDataNode.
+   */
+  public DiskBalancerDataNode getNodeByIPAddress(String ipAddresss) {
+    return ipList.get(ipAddresss);
+  }
+
+  /**
+   * Returns a node by hostName.
+   * @param hostName - HostName.
+   * @return DiskBalancerDataNode.
+   */
+  public DiskBalancerDataNode getNodeByName(String hostName) {
+    return hostNames.get(hostName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
index f70a983..a200f4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
@@ -220,7 +220,7 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
    * @param threshold - Percentage
    * @return true or false
    */
-  public boolean isBalancingNeeded(float threshold) {
+  public boolean isBalancingNeeded(double threshold) {
     for (DiskBalancerVolumeSet vSet : getVolumeSets().values()) {
       if (vSet.isBalancingNeeded(threshold)) {
         return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
index 70d7536..97d8e28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
@@ -267,7 +267,7 @@ public class DiskBalancerVolumeSet {
    *
    * @return true if balancing is needed false otherwise.
    */
-  public boolean isBalancingNeeded(float thresholdPercentage) {
+  public boolean isBalancingNeeded(double thresholdPercentage) {
     double threshold = thresholdPercentage / 100.0d;
 
     if(volumes == null || volumes.size() <= 1) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
index 88ddca4..b3d51c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
@@ -44,7 +44,7 @@ public class GreedyPlanner implements Planner {
   public static final long TB = GB * 1024L;
   private static final Logger LOG =
       LoggerFactory.getLogger(GreedyPlanner.class);
-  private final float threshold;
+  private final double threshold;
 
   /**
    * Constructs a greedy planner.
@@ -52,7 +52,7 @@ public class GreedyPlanner implements Planner {
    * @param threshold - Disk tolerance that we are ok with
    * @param node      - node on which this planner is operating upon
    */
-  public GreedyPlanner(float threshold, DiskBalancerDataNode node) {
+  public GreedyPlanner(double threshold, DiskBalancerDataNode node) {
     this.threshold = threshold;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
index 24f2970..3566438 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
@@ -5,9 +5,9 @@
  * 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/>
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ * <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
@@ -38,7 +38,7 @@ public final class PlannerFactory {
    * @return Planner
    */
   public static Planner getPlanner(String plannerName,
-                                   DiskBalancerDataNode node, float threshold) {
+      DiskBalancerDataNode node, double threshold) {
     if (plannerName.equals(GREEDY_PLANNER)) {
       if (LOG.isDebugEnabled()) {
         String message = String

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
new file mode 100644
index 0000000..e44d3dc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -0,0 +1,260 @@
+/**
+ * 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.hdfs.tools;
+
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+/**
+ * DiskBalancer is a tool that can be used to ensure that data is spread evenly
+ * across volumes of same storage type.
+ * <p>
+ * For example, if you have 3 disks, with 100 GB , 600 GB and 200 GB on each
+ * disk, this tool will ensure that each disk will have 300 GB.
+ * <p>
+ * This tool can be run while data nodes are fully functional.
+ * <p>
+ * At very high level diskbalancer computes a set of moves that will make disk
+ * utilization equal and then those moves are executed by the datanode.
+ */
+public class DiskBalancer extends Configured implements Tool {
+  /**
+   * Construct a DiskBalancer.
+   *
+   * @param conf
+   */
+  public DiskBalancer(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * NameNodeURI can point to either a real namenode, or a json file that
+   * contains the diskBalancer data in json form, that jsonNodeConnector knows
+   * how to deserialize.
+   * <p>
+   * Expected formats are :
+   * <p>
+   * hdfs://namenode.uri or file:///data/myCluster.json
+   */
+  public static final String NAMENODEURI = "uri";
+
+  /**
+   * Computes a plan for a given set of nodes.
+   */
+  public static final String PLAN = "plan";
+
+  /**
+   * Output file name, for commands like report, plan etc. This is an optional
+   * argument, by default diskbalancer will write all its output to
+   * /system/reports/diskbalancer of the current cluster it is operating
+   * against.
+   */
+  public static final String OUTFILE = "out";
+
+  /**
+   * Help for the program.
+   */
+  public static final String HELP = "help";
+
+  /**
+   * Percentage of data unevenness that we are willing to live with. For example
+   * - a value like 10 indicates that we are okay with 10 % +/- from
+   * idealStorage Target.
+   */
+  public static final String THRESHOLD = "thresholdPercentage";
+
+  /**
+   * Specifies the maximum disk bandwidth to use per second.
+   */
+  public static final String BANDWIDTH = "bandwidth";
+
+  /**
+   * Specifies the maximum errors to tolerate.
+   */
+  public static final String MAXERROR = "maxerror";
+
+  /**
+   * Node name or IP against which Disk Balancer is being run.
+   */
+  public static final String NODE = "node";
+
+  /**
+   * Runs the command in verbose mode.
+   */
+  public static final String VERBOSE = "v";
+
+  /**
+   * Template for the Before File. It is node.before.json.
+   */
+  public static final String BEFORE_TEMPLATE = "%s.before.json";
+
+  /**
+   * Template for the plan file. it is node.plan.json.
+   */
+  public static final String PLAN_TEMPLATE = "%s.plan.json";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DiskBalancer.class);
+
+  /**
+   * Main for the  DiskBalancer Command handling.
+   *
+   * @param argv - System Args Strings[]
+   * @throws Exception
+   */
+  public static void main(String[] argv) throws Exception {
+    DiskBalancer shell = new DiskBalancer(new HdfsConfiguration());
+    int res = 0;
+    try {
+      res = ToolRunner.run(shell, argv);
+    } catch (Exception ex) {
+      LOG.error(ex.toString());
+      System.exit(1);
+    }
+    System.exit(res);
+  }
+
+  /**
+   * Execute the command with the given arguments.
+   *
+   * @param args command specific arguments.
+   * @return exit code.
+   * @throws Exception
+   */
+  @Override
+  public int run(String[] args) throws Exception {
+    Options opts = getOpts();
+    CommandLine cmd = parseArgs(args, opts);
+    return dispatch(cmd, opts);
+  }
+
+  /**
+   * returns the Command Line Options.
+   *
+   * @return Options
+   */
+  private Options getOpts() {
+    Options opts = new Options();
+    addCommands(opts);
+    return opts;
+  }
+
+  /**
+   * Adds commands that we handle to opts.
+   *
+   * @param opt - Optins
+   */
+  private void addCommands(Options opt) {
+
+    Option nameNodeUri =
+        new Option(NAMENODEURI, true, "NameNode URI. e.g http://namenode" +
+            ".mycluster.com or file:///myCluster" +
+            ".json");
+    opt.addOption(nameNodeUri);
+
+    Option outFile =
+        new Option(OUTFILE, true, "File to write output to, if not specified " +
+            "defaults will be used." +
+            "e.g -out outfile.txt");
+    opt.addOption(outFile);
+
+    Option plan = new Option(PLAN, false, "write plan to the default file");
+    opt.addOption(plan);
+
+    Option bandwidth = new Option(BANDWIDTH, true, "Maximum disk bandwidth to" +
+        " be consumed by diskBalancer. " +
+        "Expressed as MBs per second.");
+    opt.addOption(bandwidth);
+
+    Option threshold = new Option(THRESHOLD, true, "Percentage skew that we " +
+        "tolerate before diskbalancer starts working or stops when reaching " +
+        "that range.");
+    opt.addOption(threshold);
+
+    Option maxErrors = new Option(MAXERROR, true, "Describes how many errors " +
+        "can be tolerated while copying between a pair of disks.");
+    opt.addOption(maxErrors);
+
+    Option node = new Option(NODE, true, "Node Name or IP");
+    opt.addOption(node);
+
+    Option help =
+        new Option(HELP, true, "Help about a command or this message");
+    opt.addOption(help);
+
+  }
+
+  /**
+   * This function parses all command line arguments and returns the appropriate
+   * values.
+   *
+   * @param argv - Argv from main
+   * @return CommandLine
+   */
+  private CommandLine parseArgs(String[] argv, Options opts)
+      throws org.apache.commons.cli.ParseException {
+    BasicParser parser = new BasicParser();
+    return parser.parse(opts, argv);
+  }
+
+  /**
+   * Dispatches calls to the right command Handler classes.
+   *
+   * @param cmd - CommandLine
+   * @throws IOException
+   * @throws URISyntaxException
+   */
+  private int dispatch(CommandLine cmd, Options opts)
+      throws IOException, URISyntaxException {
+    Command currentCommand = null;
+
+    try {
+      if (cmd.hasOption(DiskBalancer.PLAN)) {
+        currentCommand = new PlanCommand(getConf());
+      } else {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",
+            "disk balancer commands", opts,
+            "Please correct your command and try again.");
+        return 1;
+      }
+
+      currentCommand.execute(cmd);
+
+    } catch (Exception ex) {
+      System.err.printf(ex.getMessage());
+      return 1;
+    }
+    return 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75882ec0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
index 5032611..1cc90e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -50,8 +50,6 @@ import org.junit.rules.ExpectedException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/49] hadoop git commit: Merge branch 'HDFS-1312' into trunk

Posted by ar...@apache.org.
Merge branch 'HDFS-1312' into trunk


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/63148438
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/63148438
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/63148438

Branch: refs/heads/trunk
Commit: 6314843881b4c67d08215e60293f8b33242b9416
Parents: 0b9edf6 b2584be
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 23 23:00:56 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 23:00:56 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |   30 +
 .../ClientDatanodeProtocolTranslatorPB.java     |   91 +
 .../server/datanode/DiskBalancerWorkItem.java   |  287 +
 .../server/datanode/DiskBalancerWorkStatus.java |  277 +
 .../src/main/proto/ClientDatanodeProtocol.proto |   86 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |    1 +
 .../hadoop-hdfs/src/main/bin/hdfs               |    6 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   20 +
 ...tDatanodeProtocolServerSideTranslatorPB.java |   91 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  108 +-
 .../hdfs/server/datanode/DataNodeMXBean.java    |    8 +
 .../hdfs/server/datanode/DiskBalancer.java      | 1063 ++
 .../server/datanode/fsdataset/FsDatasetSpi.java |   11 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   95 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |   13 +
 .../diskbalancer/DiskBalancerConstants.java     |   44 +
 .../diskbalancer/DiskBalancerException.java     |  101 +
 .../diskbalancer/command/CancelCommand.java     |  147 +
 .../server/diskbalancer/command/Command.java    |  437 +
 .../diskbalancer/command/ExecuteCommand.java    |  115 +
 .../diskbalancer/command/HelpCommand.java       |  109 +
 .../diskbalancer/command/PlanCommand.java       |  271 +
 .../diskbalancer/command/QueryCommand.java      |  104 +
 .../diskbalancer/command/ReportCommand.java     |  214 +
 .../diskbalancer/command/package-info.java      |   22 +
 .../connectors/ClusterConnector.java            |   44 +
 .../connectors/ConnectorFactory.java            |   55 +
 .../connectors/DBNameNodeConnector.java         |  162 +
 .../connectors/JsonNodeConnector.java           |   78 +
 .../diskbalancer/connectors/package-info.java   |   37 +
 .../datamodel/DiskBalancerCluster.java          |  391 +
 .../datamodel/DiskBalancerDataNode.java         |  270 +
 .../datamodel/DiskBalancerVolume.java           |  361 +
 .../datamodel/DiskBalancerVolumeSet.java        |  342 +
 .../diskbalancer/datamodel/package-info.java    |   31 +
 .../hdfs/server/diskbalancer/package-info.java  |   36 +
 .../diskbalancer/planner/GreedyPlanner.java     |  264 +
 .../server/diskbalancer/planner/MoveStep.java   |  262 +
 .../server/diskbalancer/planner/NodePlan.java   |  190 +
 .../server/diskbalancer/planner/Planner.java    |   28 +
 .../diskbalancer/planner/PlannerFactory.java    |   60 +
 .../hdfs/server/diskbalancer/planner/Step.java  |  109 +
 .../diskbalancer/planner/package-info.java      |   46 +
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  |  475 +
 .../src/main/resources/hdfs-default.xml         |   40 +
 .../src/site/markdown/HDFSCommands.md           |   22 +
 .../src/site/markdown/HDFSDiskbalancer.md       |  120 +
 .../hdfs/server/balancer/TestBalancer.java      |    3 +-
 .../server/datanode/SimulatedFSDataset.java     |    7 +
 .../extdataset/ExternalDatasetImpl.java         |    8 +
 .../DiskBalancerResultVerifier.java             |   45 +
 .../diskbalancer/DiskBalancerTestUtil.java      |  267 +
 .../server/diskbalancer/TestConnectors.java     |   85 +
 .../server/diskbalancer/TestDataModels.java     |  227 +
 .../server/diskbalancer/TestDiskBalancer.java   |  257 +
 .../diskbalancer/TestDiskBalancerRPC.java       |  323 +
 .../TestDiskBalancerWithMockMover.java          |  628 ++
 .../hdfs/server/diskbalancer/TestPlanner.java   |  513 +
 .../command/TestDiskBalancerCommand.java        |  390 +
 .../diskbalancer/connectors/NullConnector.java  |   59 +
 .../diskBalancer/data-cluster-3node-3disk.json  |  380 +
 .../diskBalancer/data-cluster-64node-3disk.json | 9484 ++++++++++++++++++
 62 files changed, 19824 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/49] hadoop git commit: HDFS-9645. DiskBalancer: Add Query RPC. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9645. DiskBalancer: Add Query RPC. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/96fe685b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/96fe685b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/96fe685b

Branch: refs/heads/trunk
Commit: 96fe685b7a4d8db63caabae9fae4987627f08231
Parents: 0501d43
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Jan 20 10:47:30 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |  6 ++
 .../ClientDatanodeProtocolTranslatorPB.java     | 22 +++++
 .../hadoop/hdfs/server/datanode/WorkStatus.java | 85 ++++++++++++++++++++
 .../src/main/proto/ClientDatanodeProtocol.proto | 26 ++++++
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |  2 +
 ...tDatanodeProtocolServerSideTranslatorPB.java | 23 ++++++
 .../hadoop/hdfs/server/datanode/DataNode.java   |  5 ++
 .../diskbalancer/TestDiskBalancerRPC.java       | 48 ++++++++++-
 8 files changed, 214 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index 125a3c1..705c98f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
 
 /** An client-datanode protocol for block recovery
  */
@@ -177,4 +178,9 @@ public interface ClientDatanodeProtocol {
    */
   void cancelDiskBalancePlan(String planID) throws IOException;
 
+
+  /**
+   * Gets the status of an executing diskbalancer Plan.
+   */
+  WorkStatus queryDiskBalancerPlan() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index e037fcf..59f2fd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -54,7 +54,10 @@ import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.Start
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
@@ -377,4 +380,23 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  /**
+   * Gets the status of an executing diskbalancer Plan.
+   */
+  @Override
+  public WorkStatus queryDiskBalancerPlan() throws IOException {
+    try {
+      QueryPlanStatusRequestProto request =
+          QueryPlanStatusRequestProto.newBuilder().build();
+      QueryPlanStatusResponseProto response =
+          rpcProxy.queryDiskBalancerPlan(NULL_CONTROLLER, request);
+      return new WorkStatus(response.hasResult() ? response.getResult() : 0,
+          response.hasPlanID() ? response.getPlanID() : null,
+          response.hasStatus() ? response.getStatus() : null,
+          response.hasCurrentStatus() ? response.getCurrentStatus() : null);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
new file mode 100644
index 0000000..259a311
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
@@ -0,0 +1,85 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.datanode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Helper class that reports how much work has has been done by the node.
+ */
+@InterfaceAudience.Private
+public class WorkStatus {
+  private int result;
+  private String planID;
+  private String status;
+  private String currentState;
+
+  /**
+   * Constructs a workStatus Object.
+   *
+   * @param result       - int
+   * @param planID       - Plan ID
+   * @param status       - Current Status
+   * @param currentState - Current State
+   */
+  public WorkStatus(int result, String planID, String status,
+                    String currentState) {
+    this.result = result;
+    this.planID = planID;
+    this.status = status;
+    this.currentState = currentState;
+  }
+
+  /**
+   * Returns result.
+   *
+   * @return long
+   */
+  public int getResult() {
+    return result;
+  }
+
+  /**
+   * Returns planID.
+   *
+   * @return String
+   */
+  public String getPlanID() {
+    return planID;
+  }
+
+  /**
+   * Returns Status.
+   *
+   * @return String
+   */
+  public String getStatus() {
+    return status;
+  }
+
+  /**
+   * Gets current Status.
+   *
+   * @return - Json String
+   */
+  public String getCurrentState() {
+    return currentState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index b65766b..28c8681 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -180,6 +180,26 @@ message CancelPlanRequestProto {
 message CancelPlanResponseProto {
 }
 
+
+/**
+ * This message allows a client to query data node to see
+ * if a disk balancer plan is executing and if so what is
+ * the status.
+ */
+message QueryPlanStatusRequestProto {
+}
+
+/**
+ * This message describes a plan if it is in progress
+ */
+message QueryPlanStatusResponseProto {
+  optional uint32 result = 1;
+  optional string status = 2;
+  optional string planID = 3;
+  optional string currentStatus = 4;
+
+}
+
 /**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
@@ -249,4 +269,10 @@ service ClientDatanodeProtocolService {
    */
   rpc cancelDiskBalancerPlan(CancelPlanRequestProto)
       returns (CancelPlanResponseProto);
+
+  /**
+   * Gets the status of an executing Plan
+   */
+  rpc queryDiskBalancerPlan(QueryPlanStatusRequestProto)
+      returns (QueryPlanStatusResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 8ceb45b..c6a5554 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -19,3 +19,5 @@ HDFS-1312 Change Log
     HDFS-9595. DiskBalancer: Add cancelPlan RPC. (Anu Engineer via
     Arpit Agarwal)
 
+    HDFS-9645. DiskBalancer: Add Query RPC. (Anu Engineer via Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 27fe803..03717c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -51,9 +51,12 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Submit
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
 
 /**
  * Implementation for protobuf service that forwards requests
@@ -281,4 +284,24 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     }
   }
 
+  /**
+   * Gets the status of an executing Plan.
+   */
+  @Override
+  public QueryPlanStatusResponseProto queryDiskBalancerPlan(
+      RpcController controller,  QueryPlanStatusRequestProto request)
+      throws ServiceException {
+    try {
+      WorkStatus result = impl.queryDiskBalancerPlan();
+      return QueryPlanStatusResponseProto
+          .newBuilder()
+          .setResult(result.getResult())
+          .setPlanID(result.getPlanID())
+          .setStatus(result.getStatus())
+          .setCurrentStatus(result.getCurrentState())
+          .build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b2d9994..b5bd7b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3317,4 +3317,9 @@ public class DataNode extends ReconfigurableBase
     throw new DiskbalancerException("Not Implemented", 0);
   }
 
+  @Override
+  public WorkStatus queryDiskBalancerPlan() throws IOException {
+    checkSuperuserPrivilege();
+    throw new DiskbalancerException("Not Implemented", 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96fe685b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 35d3f91..a127816 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -35,6 +35,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.io.IOException;
 import java.net.URI;
 
 public class TestDiskBalancerRPC {
@@ -43,6 +44,7 @@ public class TestDiskBalancerRPC {
 
   private MiniDFSCluster cluster;
   private Configuration conf;
+
   @Before
   public void setUp() throws Exception {
     conf = new HdfsConfiguration();
@@ -113,11 +115,51 @@ public class TestDiskBalancerRPC {
 
     // Since submitDiskBalancerPlan is not implemented yet, it throws an
     // Exception, this will be modified with the actual implementation.
-    thrown.expect(DiskbalancerException.class);
-    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-
+    try {
+      dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    } catch (DiskbalancerException ex) {
+      // Let us ignore this for time being.
+    }
     thrown.expect(DiskbalancerException.class);
     dataNode.cancelDiskBalancePlan(planHash);
 
   }
+
+  @Test
+  public void TestQueryTestRpc() throws Exception {
+    final int dnIndex = 0;
+    cluster.restartDataNode(dnIndex);
+    cluster.waitActive();
+    ClusterConnector nameNodeConnector =
+        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
+        (nameNodeConnector);
+    diskBalancerCluster.readClusterInfo();
+    Assert.assertEquals(cluster.getDataNodes().size(),
+        diskBalancerCluster.getNodes().size());
+    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
+
+    final int planVersion = 0; // So far we support only one version.
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    String planHash = DigestUtils.sha512Hex(plan.toJson());
+
+    // Since submitDiskBalancerPlan is not implemented yet, it throws an
+    // Exception, this will be modified with the actual implementation.
+    try {
+      dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    } catch (DiskbalancerException ex) {
+      // Let us ignore this for time being.
+    }
+
+    // TODO : This will be fixed when we have implementation for this
+    // function in server side.
+    thrown.expect(DiskbalancerException.class);
+    dataNode.queryDiskBalancerPlan();
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/49] hadoop git commit: HDFS-10540. Diskbalancer: The CLI error message for disk balancer is not enabled is not clear. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10540. Diskbalancer: The CLI error message for disk balancer is not enabled is not clear. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cb68e5b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cb68e5b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cb68e5b3

Branch: refs/heads/trunk
Commit: cb68e5b3bdb0079af867a9e49559827ecee03010
Parents: 3225c24
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Jun 17 23:25:26 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:00 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/DiskBalancer.java      |  2 +
 .../server/diskbalancer/command/Command.java    |  2 +-
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 62 ++++++++------------
 3 files changed, 28 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb68e5b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 5fde7c5..b31b997 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -256,6 +256,8 @@ public class DiskBalancer {
       }
       ObjectMapper mapper = new ObjectMapper();
       return mapper.writeValueAsString(pathMap);
+    } catch (DiskBalancerException ex) {
+      throw ex;
     } catch (IOException e) {
       throw new DiskBalancerException("Internal error, Unable to " +
           "create JSON string.", e,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb68e5b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index d2813e7..19f9945 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -171,7 +171,7 @@ public abstract class Command extends Configured {
       diskBalancerLogs = new Path(path);
     }
     if (fs.exists(diskBalancerLogs)) {
-      LOG.error("Another Diskbalancer instance is running ? - Target " +
+      LOG.debug("Another Diskbalancer instance is running ? - Target " +
           "Directory already exists. {}", diskBalancerLogs);
       throw new IOException("Another DiskBalancer files already exist at the " +
           "target location. " + diskBalancerLogs.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb68e5b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index d83a49c..67703c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -36,9 +36,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.io.PrintStream;
-import java.net.URISyntaxException;
 
 /**
  * DiskBalancer is a tool that can be used to ensure that data is spread evenly
@@ -169,7 +167,7 @@ public class DiskBalancer extends Configured implements Tool {
       res = ToolRunner.run(shell, argv);
     } catch (Exception ex) {
       LOG.error(ex.toString());
-      System.exit(1);
+      res = 1;
     }
     System.exit(res);
   }
@@ -449,51 +447,41 @@ public class DiskBalancer extends Configured implements Tool {
    * @param cmd  - CommandLine
    * @param opts options of command line
    * @param out  the output stream used for printing
-   * @throws IOException
-   * @throws URISyntaxException
    */
   private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
-      throws IOException, URISyntaxException {
+      throws Exception {
     Command currentCommand = null;
+    if (cmd.hasOption(DiskBalancer.PLAN)) {
+      currentCommand = new PlanCommand(getConf());
+    }
 
-    try {
-
-      if (cmd.hasOption(DiskBalancer.PLAN)) {
-        currentCommand = new PlanCommand(getConf());
-      }
-
-      if (cmd.hasOption(DiskBalancer.EXECUTE)) {
-        currentCommand = new ExecuteCommand(getConf());
-      }
-
-      if (cmd.hasOption(DiskBalancer.QUERY)) {
-        currentCommand = new QueryCommand(getConf());
-      }
+    if (cmd.hasOption(DiskBalancer.EXECUTE)) {
+      currentCommand = new ExecuteCommand(getConf());
+    }
 
-      if (cmd.hasOption(DiskBalancer.CANCEL)) {
-        currentCommand = new CancelCommand(getConf());
-      }
+    if (cmd.hasOption(DiskBalancer.QUERY)) {
+      currentCommand = new QueryCommand(getConf());
+    }
 
-      if (cmd.hasOption(DiskBalancer.REPORT)) {
-        currentCommand = new ReportCommand(getConf(), out);
-      }
+    if (cmd.hasOption(DiskBalancer.CANCEL)) {
+      currentCommand = new CancelCommand(getConf());
+    }
 
-      if (cmd.hasOption(DiskBalancer.HELP)) {
-        currentCommand = new HelpCommand(getConf());
-      }
+    if (cmd.hasOption(DiskBalancer.REPORT)) {
+      currentCommand = new ReportCommand(getConf(), out);
+    }
 
-      // Invoke Main help here.
-      if (currentCommand == null) {
-        new HelpCommand(getConf()).execute(null);
-        return 1;
-      }
+    if (cmd.hasOption(DiskBalancer.HELP)) {
+      currentCommand = new HelpCommand(getConf());
+    }
 
-      currentCommand.execute(cmd);
-    } catch (Exception ex) {
-      System.err.printf(ex.getMessage());
+    // Invoke main help here.
+    if (currentCommand == null) {
+      new HelpCommand(getConf()).execute(null);
       return 1;
     }
+
+    currentCommand.execute(cmd);
     return 0;
   }
-
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/49] hadoop git commit: HDFS-9703. DiskBalancer: getBandwidth implementation. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9703. DiskBalancer: getBandwidth implementation. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/75a711a2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/75a711a2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/75a711a2

Branch: refs/heads/trunk
Commit: 75a711a2d53966361f5d5fa727b43c9fddb01504
Parents: 918722b
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Mar 14 12:57:29 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DataNode.java   |  6 +++--
 .../hdfs/server/datanode/DiskBalancer.java      | 17 +++++++++++++
 .../diskbalancer/TestDiskBalancerRPC.java       | 26 ++++++++++++--------
 3 files changed, 37 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75a711a2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 00e124d..8a61291 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1031,7 +1031,7 @@ public class DataNode extends ReconfigurableBase
    * @param  data - FSDataSet
    * @param conf - Config
    */
-  private synchronized void initDiskBalancer(FsDatasetSpi data,
+  private void initDiskBalancer(FsDatasetSpi data,
                                              Configuration conf) {
     if (this.diskBalancer != null) {
       return;
@@ -1045,7 +1045,7 @@ public class DataNode extends ReconfigurableBase
   /**
    * Shutdown disk balancer.
    */
-  private synchronized void shutdownDiskBalancer() {
+  private  void shutdownDiskBalancer() {
     if (this.diskBalancer != null) {
       this.diskBalancer.shutdown();
       this.diskBalancer = null;
@@ -3375,6 +3375,8 @@ public class DataNode extends ReconfigurableBase
     switch (key) {
     case DiskBalancerConstants.DISKBALANCER_VOLUME_NAME:
       return this.diskBalancer.getVolumeNames();
+    case DiskBalancerConstants.DISKBALANCER_BANDWIDTH :
+      return Long.toString(this.diskBalancer.getBandwidth());
     default:
       LOG.error("Disk Balancer - Unknown key in get balancer setting. Key: " +
           key);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75a711a2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 9e41d2e..d1bc1f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -73,6 +73,7 @@ public class DiskBalancer {
   private Future future;
   private String planID;
   private DiskBalancerWorkStatus.Result currentResult;
+  private long bandwidth;
 
   /**
    * Constructs a Disk Balancer object. This object takes care of reading a
@@ -159,6 +160,7 @@ public class DiskBalancer {
       createWorkPlan(nodePlan);
       this.planID = planID;
       this.currentResult = Result.PLAN_UNDER_PROGRESS;
+      this.bandwidth = bandwidth;
       executePlan();
     } finally {
       lock.unlock();
@@ -248,6 +250,21 @@ public class DiskBalancer {
     }
   }
 
+  /**
+   * Returns the current bandwidth.
+   *
+   * @return string representation of bandwidth.
+   * @throws DiskBalancerException
+   */
+  public long getBandwidth() throws DiskBalancerException {
+    lock.lock();
+    try {
+      checkDiskBalancerEnabled();
+      return this.bandwidth;
+    } finally {
+      lock.unlock();
+    }
+  }
 
   /**
    * Throws if Disk balancer is disabled.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75a711a2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 37a6216..9cd41c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -187,6 +187,22 @@ public class TestDiskBalancerRPC {
     dataNode.getDiskBalancerSetting(invalidSetting);
   }
 
+  @Test
+  public void testgetDiskBalancerBandwidth() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    String bandwidthString = dataNode.getDiskBalancerSetting(
+        DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
+    long value = Long.decode(bandwidthString);
+    Assert.assertEquals(10L, value);
+  }
+
+
 
   @Test
   public void testQueryPlan() throws Exception {
@@ -211,16 +227,6 @@ public class TestDiskBalancerRPC {
     Assert.assertTrue(status.getResult() == NO_PLAN);
   }
 
-  @Test
-  public void testGetDiskBalancerSetting() throws Exception {
-    final int dnIndex = 0;
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
-    thrown.expect(DiskBalancerException.class);
-    thrown.expect(new
-        ResultVerifier(Result.UNKNOWN_KEY));
-    dataNode.getDiskBalancerSetting(
-        DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
-  }
 
   private class RpcTestHelper {
     private NodePlan plan;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/49] hadoop git commit: HDFS-10403. DiskBalancer: Add cancel command. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10403. DiskBalancer: Add cancel command. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/43eee509
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43eee509
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43eee509

Branch: refs/heads/trunk
Commit: 43eee50966191e9cfdb7ab19383edb3a44f93481
Parents: 9e5fcb5
Author: Anu Engineer <ae...@apache.org>
Authored: Mon May 23 14:51:35 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/CancelCommand.java     | 137 +++++++++++++++++++
 .../server/diskbalancer/command/Command.java    |  14 ++
 .../diskbalancer/command/ExecuteCommand.java    |  16 +--
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  |  24 +++-
 4 files changed, 177 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43eee509/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
new file mode 100644
index 0000000..f395802
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
@@ -0,0 +1,137 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
+import java.io.IOException;
+
+/**
+ * Cancels a running plan.
+ */
+public class CancelCommand extends Command {
+  /**
+   * Contructs a cancel Command.
+   *
+   * @param conf - Conf
+   */
+  public CancelCommand(Configuration conf) {
+    super(conf);
+    addValidCommandParameters(DiskBalancer.CANCEL, "Cancels a running plan.");
+    addValidCommandParameters(DiskBalancer.NODE, "Node to run the command " +
+        "against in node:port format.");
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.info("Executing \"Cancel plan\" command.");
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.CANCEL));
+    verifyCommandOptions(DiskBalancer.CANCEL, cmd);
+
+    // We can cancel a plan using datanode address and plan ID
+    // that you can read from a datanode using queryStatus
+    if(cmd.hasOption(DiskBalancer.NODE)) {
+      String nodeAddress = cmd.getOptionValue(DiskBalancer.NODE);
+      String planHash = cmd.getOptionValue(DiskBalancer.CANCEL);
+      cancelPlanUsingHash(nodeAddress, planHash);
+    } else {
+      // Or you can cancel a plan using the plan file. If the user
+      // points us to the plan file, we can compute the hash as well as read
+      // the address of the datanode from the plan file.
+      String planFile = cmd.getOptionValue(DiskBalancer.CANCEL);
+      Preconditions.checkArgument(planFile == null || planFile.isEmpty(),
+          "Invalid plan file specified.");
+      String planData = null;
+      try (FSDataInputStream plan = open(planFile)) {
+        planData = IOUtils.toString(plan);
+      }
+      cancelPlan(planData);
+    }
+  }
+
+  /**
+   * Cancels a running plan.
+   *
+   * @param planData - Plan data.
+   * @throws IOException
+   */
+  private void cancelPlan(String planData) throws IOException {
+    Preconditions.checkNotNull(planData);
+    NodePlan plan = readPlan(planData);
+    String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
+    Preconditions.checkNotNull(dataNodeAddress);
+    ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);
+    String planHash = DigestUtils.sha512Hex(planData);
+    try {
+      dataNode.cancelDiskBalancePlan(planHash);
+    } catch (DiskBalancerException ex) {
+      LOG.error("Cancelling plan on  {} failed. Result: {}, Message: {}",
+          plan.getNodeName(), ex.getResult().toString(), ex.getMessage());
+      throw ex;
+    }
+  }
+
+  /**
+   * Cancels a running plan.
+   * @param nodeAddress - Address of the data node.
+   * @param hash - Sha512 hash of the plan, which can be read from datanode
+   *             using query status command.
+   * @throws IOException
+   */
+  private void cancelPlanUsingHash(String nodeAddress, String hash) throws
+      IOException {
+    Preconditions.checkNotNull(nodeAddress);
+    Preconditions.checkNotNull(hash);
+    ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeAddress);
+    try {
+      dataNode.cancelDiskBalancePlan(hash);
+    } catch (DiskBalancerException ex) {
+      LOG.error("Cancelling plan on  {} failed. Result: {}, Message: {}",
+          nodeAddress, ex.getResult().toString(), ex.getMessage());
+      throw ex;
+    }
+  }
+
+
+  /**
+   * Gets extended help for this command.
+   *
+   * @return Help Message
+   */
+  @Override
+  protected String getHelp() {
+    return "Cancels a running command. e.g -cancel <PlanFile> or -cancel " +
+        "<planID> -node <datanode>";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43eee509/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 919d549..feee977 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode
 
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -392,4 +394,16 @@ public abstract class Command extends Configured {
   protected DiskBalancerCluster getCluster() {
     return cluster;
   }
+
+  /**
+   * Returns a plan from the Json Data.
+   *
+   * @param planData - Json String
+   * @return NodePlan
+   * @throws IOException
+   */
+  protected NodePlan readPlan(String planData) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(planData, NodePlan.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43eee509/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index 1f7e81f..c17ef00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
-import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
-import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+
 import java.io.IOException;
 
 
@@ -94,17 +94,7 @@ public class ExecuteCommand extends Command {
     }
   }
 
-  /**
-   * Returns a plan from the Json Data.
-   *
-   * @param planData - Json String
-   * @return NodePlan
-   * @throws IOException
-   */
-  private NodePlan readPlan(String planData) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.readValue(planData, NodePlan.class);
-  }
+
 
   /**
    * Gets extended help for this command.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43eee509/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index f5dbe4e..4005652 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -24,6 +24,7 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.CancelCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
@@ -105,7 +106,10 @@ public class DiskBalancer extends Configured implements Tool {
    * Reports the status of disk balancer operation.
    */
   public static final String QUERY = "query";
-
+  /**
+   * Cancels a running plan.
+   */
+  public static final String CANCEL = "cancel";
   /**
    * Template for the Before File. It is node.before.json.
    */
@@ -168,6 +172,7 @@ public class DiskBalancer extends Configured implements Tool {
     addPlanCommands(opts);
     addExecuteCommands(opts);
     addQueryCommands(opts);
+    addCancelCommands(opts);
     return opts;
   }
 
@@ -234,6 +239,19 @@ public class DiskBalancer extends Configured implements Tool {
   }
 
   /**
+   * Adds cancel command options.
+   * @param opt Options
+   */
+  private void addCancelCommands(Options opt) {
+    Option cancel = new Option(CANCEL, true, "Cancels a running plan. -cancel" +
+        " <planFile> or -cancel <planID> -node <datanode:port>");
+    opt.addOption(cancel);
+    Option node = new Option(NODE, true, "Name of the datanode in name:port " +
+        "format");
+    opt.addOption(node);
+  }
+
+  /**
    * This function parses all command line arguments and returns the appropriate
    * values.
    *
@@ -271,6 +289,10 @@ public class DiskBalancer extends Configured implements Tool {
         currentCommand = new QueryCommand(getConf());
       }
 
+      if(cmd.hasOption(DiskBalancer.CANCEL)) {
+        currentCommand = new CancelCommand(getConf());
+      }
+
       if(currentCommand == null) {
         HelpFormatter helpFormatter = new HelpFormatter();
         helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/49] hadoop git commit: HDFS-10496. DiskBalancer: ExecuteCommand checks planFile in a wrong way. Contributed by Lei (Eddy) Xu.

Posted by ar...@apache.org.
HDFS-10496. DiskBalancer: ExecuteCommand checks planFile in a wrong way. Contributed by Lei (Eddy) Xu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d2ff793a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d2ff793a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d2ff793a

Branch: refs/heads/trunk
Commit: d2ff793ae4788418116b64e7141e80897d4f9c24
Parents: 64ccb23
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 7 15:10:22 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ff793a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
index c17ef00..6d30e86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -61,7 +61,7 @@ public class ExecuteCommand extends Command {
     verifyCommandOptions(DiskBalancer.EXECUTE, cmd);
 
     String planFile = cmd.getOptionValue(DiskBalancer.EXECUTE);
-    Preconditions.checkArgument(planFile == null || planFile.isEmpty(),
+    Preconditions.checkArgument(planFile != null && !planFile.isEmpty(),
         "Invalid plan file specified.");
 
     String planData = null;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/49] hadoop git commit: HDFS-9546: DiskBalancer: Add Execute command. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-9546: DiskBalancer: Add Execute command. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1b39b283
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1b39b283
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1b39b283

Branch: refs/heads/trunk
Commit: 1b39b283c70854bf3b77f5ba9fbcce064bfea5c3
Parents: 75882ec
Author: Anu Engineer <ae...@apache.org>
Authored: Fri May 13 10:52:58 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/Command.java    |  18 ++-
 .../diskbalancer/command/ExecuteCommand.java    | 119 +++++++++++++++++++
 .../diskbalancer/command/PlanCommand.java       |  22 ++--
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  |  71 ++++++-----
 4 files changed, 187 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b39b283/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 6522434..919d549 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -24,6 +24,7 @@ import org.apache.commons.cli.Option;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -81,8 +82,6 @@ public abstract class Command extends Configured {
   public Command(Configuration conf) {
     super(conf);
     // These arguments are valid for all commands.
-    addValidCommandParameters(DiskBalancer.NAMENODEURI, "Name Node URI or " +
-        "file URI for cluster");
     addValidCommandParameters(DiskBalancer.HELP, "Help for this command");
     addValidCommandParameters("arg", "");
   }
@@ -348,10 +347,25 @@ public abstract class Command extends Configured {
    * @return OutputStream.
    */
   protected FSDataOutputStream create(String fileName) throws IOException {
+    Preconditions.checkNotNull(fileName);
+    if(fs == null) {
+      fs = FileSystem.get(getConf());
+    }
     return fs.create(new Path(this.diskBalancerLogs, fileName));
   }
 
   /**
+   * Returns a InputStream to read data.
+   */
+  protected FSDataInputStream open(String fileName) throws IOException {
+    Preconditions.checkNotNull(fileName);
+    if(fs == null) {
+      fs = FileSystem.get(getConf());
+    }
+    return  fs.open(new Path(fileName));
+  }
+
+  /**
    * Returns the output path where the plan and snapshot gets written.
    *
    * @return Path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b39b283/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
new file mode 100644
index 0000000..1f7e81f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
@@ -0,0 +1,119 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer.command;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
+import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+
+
+/**
+ * executes a given plan.
+ */
+public class ExecuteCommand extends Command {
+
+  /**
+   * Constructs ExecuteCommand.
+   *
+   * @param conf - Configuration.
+   */
+  public ExecuteCommand(Configuration conf) {
+    super(conf);
+    addValidCommandParameters(DiskBalancer.EXECUTE, "Executes a given plan.");
+    addValidCommandParameters(DiskBalancer.NODE, "Name of the target node.");
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   */
+  @Override
+  public void execute(CommandLine cmd) throws Exception {
+    LOG.info("Executing \"execute plan\" command");
+    Preconditions.checkState(cmd.hasOption(DiskBalancer.EXECUTE));
+    verifyCommandOptions(DiskBalancer.EXECUTE, cmd);
+
+    String planFile = cmd.getOptionValue(DiskBalancer.EXECUTE);
+    Preconditions.checkArgument(planFile == null || planFile.isEmpty(),
+        "Invalid plan file specified.");
+
+    String planData = null;
+    try (FSDataInputStream plan = open(planFile)) {
+      planData = IOUtils.toString(plan);
+    }
+    submitPlan(planData);
+  }
+
+  /**
+   * Submits plan to a given data node.
+   *
+   * @param planData - PlanData Json String.
+   * @throws IOException
+   */
+  private void submitPlan(String planData) throws IOException {
+    Preconditions.checkNotNull(planData);
+    NodePlan plan = readPlan(planData);
+    String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
+    Preconditions.checkNotNull(dataNodeAddress);
+    ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);
+    String planHash = DigestUtils.sha512Hex(planData);
+    try {
+      dataNode.submitDiskBalancerPlan(planHash, DiskBalancer.PLAN_VERSION,
+          planData, false); // TODO : Support skipping date check.
+    } catch (DiskBalancerException ex) {
+      LOG.error("Submitting plan on  {} failed. Result: {}, Message: {}",
+          plan.getNodeName(), ex.getResult().toString(), ex.getMessage());
+      throw ex;
+    }
+  }
+
+  /**
+   * Returns a plan from the Json Data.
+   *
+   * @param planData - Json String
+   * @return NodePlan
+   * @throws IOException
+   */
+  private NodePlan readPlan(String planData) throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(planData, NodePlan.class);
+  }
+
+  /**
+   * Gets extended help for this command.
+   *
+   * @return Help Message
+   */
+  @Override
+  protected String getHelp() {
+    return "Execute command takes a plan and runs it against the node. e.g. " +
+        "hdfs diskbalancer -execute <nodename.plan.json> ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b39b283/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index 2422215..d346c84 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -24,7 +24,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -54,6 +55,9 @@ public class PlanCommand extends Command {
     this.thresholdPercentage = 1;
     this.bandwidth = 0;
     this.maxError = 0;
+    addValidCommandParameters(DiskBalancer.NAMENODEURI, "Name Node URI or " +
+        "file URI for cluster");
+
     addValidCommandParameters(DiskBalancer.OUTFILE, "Output file");
     addValidCommandParameters(DiskBalancer.BANDWIDTH, "Maximum Bandwidth to " +
         "be used while copying.");
@@ -61,7 +65,6 @@ public class PlanCommand extends Command {
         "we tolerate before diskbalancer starts working.");
     addValidCommandParameters(DiskBalancer.MAXERROR, "Max errors to tolerate " +
         "between 2 disks");
-    addValidCommandParameters(DiskBalancer.NODE, "Name / Address of the node.");
     addValidCommandParameters(DiskBalancer.VERBOSE, "Run plan command in " +
         "verbose mode.");
   }
@@ -79,7 +82,7 @@ public class PlanCommand extends Command {
     Preconditions.checkState(cmd.hasOption(DiskBalancer.PLAN));
     verifyCommandOptions(DiskBalancer.PLAN, cmd);
 
-    if (!cmd.hasOption(DiskBalancer.NODE)) {
+    if (cmd.getOptionValue(DiskBalancer.PLAN) == null) {
       throw new IllegalArgumentException("A node name is required to create a" +
           " plan.");
     }
@@ -101,10 +104,11 @@ public class PlanCommand extends Command {
     }
     setOutputPath(output);
 
-    DiskBalancerDataNode node = getNode(cmd.getOptionValue(DiskBalancer.NODE));
+    // -plan nodename is the command line argument.
+    DiskBalancerDataNode node = getNode(cmd.getOptionValue(DiskBalancer.PLAN));
     if (node == null) {
       throw new IllegalArgumentException("Unable to find the specified node. " +
-          cmd.getOptionValue(DiskBalancer.NODE));
+          cmd.getOptionValue(DiskBalancer.PLAN));
     }
     this.thresholdPercentage = getThresholdPercentage(cmd);
     setNodesToProcess(node);
@@ -115,16 +119,16 @@ public class PlanCommand extends Command {
     LOG.info("Writing plan to : {}", getOutputPath());
     System.out.printf("Writing plan to : %s%n", getOutputPath());
 
-    try(FSDataOutputStream beforeStream = create(String.format(
+    try (FSDataOutputStream beforeStream = create(String.format(
         DiskBalancer.BEFORE_TEMPLATE,
-        cmd.getOptionValue(DiskBalancer.NODE)))) {
+        cmd.getOptionValue(DiskBalancer.PLAN)))) {
       beforeStream.write(getCluster().toJson()
           .getBytes(StandardCharsets.UTF_8));
     }
 
-    try(FSDataOutputStream planStream = create(String.format(
+    try (FSDataOutputStream planStream = create(String.format(
         DiskBalancer.PLAN_TEMPLATE,
-        cmd.getOptionValue(DiskBalancer.NODE)))) {
+        cmd.getOptionValue(DiskBalancer.PLAN)))) {
       planStream.write(getPlan(plans).getBytes(StandardCharsets.UTF_8));
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b39b283/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index e44d3dc..87fbf4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.Command;
+import org.apache.hadoop.hdfs.server.diskbalancer.command.ExecuteCommand;
 import org.apache.hadoop.hdfs.server.diskbalancer.command.PlanCommand;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -48,15 +49,6 @@ import java.net.URISyntaxException;
  */
 public class DiskBalancer extends Configured implements Tool {
   /**
-   * Construct a DiskBalancer.
-   *
-   * @param conf
-   */
-  public DiskBalancer(Configuration conf) {
-    super(conf);
-  }
-
-  /**
    * NameNodeURI can point to either a real namenode, or a json file that
    * contains the diskBalancer data in json form, that jsonNodeConnector knows
    * how to deserialize.
@@ -66,12 +58,10 @@ public class DiskBalancer extends Configured implements Tool {
    * hdfs://namenode.uri or file:///data/myCluster.json
    */
   public static final String NAMENODEURI = "uri";
-
   /**
    * Computes a plan for a given set of nodes.
    */
   public static final String PLAN = "plan";
-
   /**
    * Output file name, for commands like report, plan etc. This is an optional
    * argument, by default diskbalancer will write all its output to
@@ -79,53 +69,58 @@ public class DiskBalancer extends Configured implements Tool {
    * against.
    */
   public static final String OUTFILE = "out";
-
   /**
    * Help for the program.
    */
   public static final String HELP = "help";
-
   /**
    * Percentage of data unevenness that we are willing to live with. For example
    * - a value like 10 indicates that we are okay with 10 % +/- from
    * idealStorage Target.
    */
   public static final String THRESHOLD = "thresholdPercentage";
-
   /**
    * Specifies the maximum disk bandwidth to use per second.
    */
   public static final String BANDWIDTH = "bandwidth";
-
   /**
    * Specifies the maximum errors to tolerate.
    */
   public static final String MAXERROR = "maxerror";
-
   /**
-   * Node name or IP against which Disk Balancer is being run.
+   * Executes a given plan file on the target datanode.
+   */
+  public static final String EXECUTE = "execute";
+  /**
+   * Name or address of the node to execute against.
    */
   public static final String NODE = "node";
-
   /**
    * Runs the command in verbose mode.
    */
   public static final String VERBOSE = "v";
-
+  public static final int PLAN_VERSION = 1;
   /**
    * Template for the Before File. It is node.before.json.
    */
   public static final String BEFORE_TEMPLATE = "%s.before.json";
-
   /**
    * Template for the plan file. it is node.plan.json.
    */
   public static final String PLAN_TEMPLATE = "%s.plan.json";
-
   private static final Logger LOG =
       LoggerFactory.getLogger(DiskBalancer.class);
 
   /**
+   * Construct a DiskBalancer.
+   *
+   * @param conf
+   */
+  public DiskBalancer(Configuration conf) {
+    super(conf);
+  }
+
+  /**
    * Main for the  DiskBalancer Command handling.
    *
    * @param argv - System Args Strings[]
@@ -164,16 +159,16 @@ public class DiskBalancer extends Configured implements Tool {
    */
   private Options getOpts() {
     Options opts = new Options();
-    addCommands(opts);
+    addPlanCommands(opts);
     return opts;
   }
 
   /**
-   * Adds commands that we handle to opts.
+   * Adds commands for plan command.
    *
-   * @param opt - Optins
+   * @param opt - Options
    */
-  private void addCommands(Options opt) {
+  private void addPlanCommands(Options opt) {
 
     Option nameNodeUri =
         new Option(NAMENODEURI, true, "NameNode URI. e.g http://namenode" +
@@ -187,7 +182,8 @@ public class DiskBalancer extends Configured implements Tool {
             "e.g -out outfile.txt");
     opt.addOption(outFile);
 
-    Option plan = new Option(PLAN, false, "write plan to the default file");
+    Option plan = new Option(PLAN, true , "create a plan for the given node. " +
+        "e.g -plan <nodename> | <nodeIP> | <nodeUUID>");
     opt.addOption(plan);
 
     Option bandwidth = new Option(BANDWIDTH, true, "Maximum disk bandwidth to" +
@@ -204,13 +200,19 @@ public class DiskBalancer extends Configured implements Tool {
         "can be tolerated while copying between a pair of disks.");
     opt.addOption(maxErrors);
 
-    Option node = new Option(NODE, true, "Node Name or IP");
-    opt.addOption(node);
-
     Option help =
         new Option(HELP, true, "Help about a command or this message");
     opt.addOption(help);
+  }
 
+  /**
+   * Adds execute command options.
+   * @param opt Options
+   */
+  private void addExecuteCommands(Options opt) {
+    Option execute = new Option(EXECUTE, true , "Takes a plan file and " +
+        "submits it for execution to the datanode. e.g -execute <planfile>");
+    opt.addOption(execute);
   }
 
   /**
@@ -238,18 +240,23 @@ public class DiskBalancer extends Configured implements Tool {
     Command currentCommand = null;
 
     try {
+
       if (cmd.hasOption(DiskBalancer.PLAN)) {
         currentCommand = new PlanCommand(getConf());
-      } else {
+      }
+
+      if(cmd.hasOption(DiskBalancer.EXECUTE)) {
+        currentCommand = new ExecuteCommand(getConf());
+      }
+
+      if(currentCommand == null) {
         HelpFormatter helpFormatter = new HelpFormatter();
         helpFormatter.printHelp(80, "hdfs diskbalancer -uri [args]",
             "disk balancer commands", opts,
             "Please correct your command and try again.");
         return 1;
       }
-
       currentCommand.execute(cmd);
-
     } catch (Exception ex) {
       System.err.printf(ex.getMessage());
       return 1;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/49] hadoop git commit: HDFS-9588. DiskBalancer: Add submitDiskbalancer RPC. (Anu Engineer via Arpit Agarwal)

Posted by ar...@apache.org.
HDFS-9588. DiskBalancer: Add submitDiskbalancer RPC. (Anu Engineer via Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7100c0da
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7100c0da
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7100c0da

Branch: refs/heads/trunk
Commit: 7100c0da353d0960d3db71b029a36247838b24c6
Parents: 5724a10
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Jan 11 20:31:18 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |  7 ++
 .../ClientDatanodeProtocolTranslatorPB.java     | 32 +++++++
 .../src/main/proto/ClientDatanodeProtocol.proto | 23 ++++++
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |  3 +
 ...tDatanodeProtocolServerSideTranslatorPB.java | 27 ++++++
 .../hadoop/hdfs/server/datanode/DataNode.java   | 24 ++++++
 .../diskbalancer/DiskbalancerException.java     | 86 +++++++++++++++++++
 .../diskbalancer/TestDiskBalancerRPC.java       | 87 ++++++++++++++++++++
 8 files changed, 289 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index e541388..6e9cef0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -163,4 +163,11 @@ public interface ClientDatanodeProtocol {
    * @return balancer bandwidth
    */
   long getBalancerBandwidth() throws IOException;
+
+  /**
+   * Submit a disk balancer plan for execution.
+   */
+  void submitDiskBalancerPlan(String planID, long planVersion, long bandwidth,
+                              String plan) throws IOException;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 6aaa025..da8d962 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.GetRe
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ShutdownDatanodeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.StartReconfigurationRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -326,4 +327,35 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  /**
+   * Submits a disk balancer plan to the datanode.
+   * @param planID - Plan ID is the hash512 string of the plan that is
+   *               submitted. This is used by clients when they want to find
+   *               local copies of these plans.
+   * @param planVersion - The data format of the plans - for future , not
+   *                    used now.
+   * @param bandwidth - Maximum disk bandwidth to consume, setting this value
+   *                  to zero allows datanode to use the value defined in
+   *                  configration.
+   * @param plan - Actual plan.
+   * @return Success or throws Exception.
+   * @throws Exception
+   */
+  @Override
+  public void submitDiskBalancerPlan(String planID, long planVersion,
+      long bandwidth, String plan) throws IOException {
+    try {
+      SubmitDiskBalancerPlanRequestProto request =
+          SubmitDiskBalancerPlanRequestProto.newBuilder()
+              .setPlanID(planID)
+              .setPlanVersion(planVersion)
+              .setMaxDiskBandwidth(bandwidth)
+              .setPlan(plan)
+              .build();
+      rpcProxy.submitDiskBalancerPlan(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index e135df8..d11979b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -150,6 +150,23 @@ message GetBalancerBandwidthResponseProto {
 }
 
 /**
+ * This message allows a client to submit a disk
+ * balancer plan to a data node.
+ */
+message SubmitDiskBalancerPlanRequestProto {
+  required string planID = 1; // A hash of the plan like SHA512
+  required string plan = 2; // Json String that describes the plan
+  optional uint64 planVersion = 3; // Plan version number
+  optional uint64 maxDiskBandwidth = 4; // optional bandwidth control.
+}
+
+/**
+ * Response from the DataNode on Plan Submit request
+ */
+message SubmitDiskBalancerPlanResponseProto {
+}
+
+/**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
  */
@@ -207,4 +224,10 @@ service ClientDatanodeProtocolService {
    */
   rpc getBalancerBandwidth(GetBalancerBandwidthRequestProto)
       returns(GetBalancerBandwidthResponseProto);
+
+  /**
+   * Submit a disk balancer plan for execution
+   */
+  rpc submitDiskBalancerPlan(SubmitDiskBalancerPlanRequestProto)
+      returns (SubmitDiskBalancerPlanResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 940e1b5..6d8cde0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -13,3 +13,6 @@ HDFS-1312 Change Log
 
     HDFS-9469. DiskBalancer: Add Planner. (Anu Engineer via Arpit Agarwal)
 
+    HDFS-9588. DiskBalancer: Add submitDiskbalancer RPC. (Anu Engineer via
+    Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index e0401f7..824f050 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -47,6 +47,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.Start
 import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.StartReconfigurationResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.TriggerBlockReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.SubmitDiskBalancerPlanResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -232,4 +234,29 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     return GetBalancerBandwidthResponseProto.newBuilder()
         .setBandwidth(bandwidth).build();
   }
+
+  /**
+   * Submit a disk balancer plan for execution.
+   * @param controller  - RpcController
+   * @param request   - Request
+   * @return   Response
+   * @throws ServiceException
+   */
+  @Override
+  public SubmitDiskBalancerPlanResponseProto submitDiskBalancerPlan(
+      RpcController controller, SubmitDiskBalancerPlanRequestProto request)
+      throws ServiceException {
+    try {
+      impl.submitDiskBalancerPlan(request.getPlanID(),
+          request.hasPlanVersion() ? request.getPlanVersion() : 0,
+          request.hasMaxDiskBandwidth() ? request.getMaxDiskBandwidth() : 0,
+          request.getPlan());
+      SubmitDiskBalancerPlanResponseProto response =
+          SubmitDiskBalancerPlanResponseProto.newBuilder()
+              .build();
+      return response;
+    } catch(Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index a59a59f..e06555f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -169,6 +169,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.web.DatanodeHttpServer;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskbalancerException;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -3286,4 +3287,27 @@ public class DataNode extends ReconfigurableBase
   public Tracer getTracer() {
     return tracer;
   }
+
+  /**
+   * Allows submission of a disk balancer Job.
+   * @param planID  - Hash value of the plan.
+   * @param planVersion - Plan version, reserved for future use. We have only
+   *                    version 1 now.
+   * @param bandwidth - Max disk bandwidth to use, 0 means use value defined
+   *                  in the configration.
+   * @param plan - Actual plan
+   * @return  success or throws an exception.
+   * @throws Exception
+   */
+  @Override
+  public void submitDiskBalancerPlan(String planID,
+      long planVersion, long bandwidth, String plan) throws IOException {
+
+    // TODO : This will be replaced with actual code later.
+    // Right now throwing DiskbalancerException instead
+    // NotImplementedException to indicate the eventually disk balancer code
+    // will throw DiskbalancerException.
+    throw new DiskbalancerException("Not Implemented", 0);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
new file mode 100644
index 0000000..9d47dc3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
@@ -0,0 +1,86 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import java.io.IOException;
+
+/**
+ * Disk Balancer Exceptions.
+ */
+public class DiskbalancerException extends IOException {
+  private int result;
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   *                the
+   *                {@link #getMessage()} method)
+   */
+  public DiskbalancerException(String message, int result) {
+    super(message);
+    this.result = result;
+  }
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message and
+   * cause.
+   * <p/>
+   * <p> Note that the detail message associated with {@code cause} is
+   * <i>not</i>
+   * automatically incorporated into this exception's detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   *                the
+   *                {@link #getMessage()} method)
+   * @param cause   The cause (which is saved for later retrieval by the {@link
+   *                #getCause()} method).  (A null value is permitted, and
+   *                indicates that the cause is nonexistent or unknown.)
+   * @since 1.6
+   */
+  public DiskbalancerException(String message, Throwable cause, int result) {
+    super(message, cause);
+    this.result = result;
+  }
+
+  /**
+   * Constructs an {@code IOException} with the specified cause and a detail
+   * message of {@code (cause==null ? null : cause.toString())} (which typically
+   * contains the class and detail message of {@code cause}). This
+   * constructor is
+   * useful for IO exceptions that are little more than wrappers for other
+   * throwables.
+   *
+   * @param cause The cause (which is saved for later retrieval by the {@link
+   *              #getCause()} method).  (A null value is permitted, and
+   *              indicates
+   *              that the cause is nonexistent or unknown.)
+   * @since 1.6
+   */
+  public DiskbalancerException(Throwable cause, int result) {
+    super(cause);
+    this.result = result;
+  }
+
+  /**
+   * Returns the result.
+   * @return int
+   */
+  public int getResult() {
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7100c0da/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
new file mode 100644
index 0000000..e047d5a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -0,0 +1,87 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
+import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.net.URI;
+
+public class TestDiskBalancerRPC {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void TestSubmitTestRpc() throws Exception {
+    URI clusterJson = getClass()
+        .getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
+    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
+        null);
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(jsonConnector);
+    diskBalancerCluster.readClusterInfo();
+    Assert.assertEquals(3, diskBalancerCluster.getNodes().size());
+    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
+    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
+        ());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    final int dnIndex = 0;
+    final int planVersion = 0; // So far we support only one version.
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    String planHash = DigestUtils.sha512Hex(plan.toJson());
+
+    // Since submitDiskBalancerPlan is not implemented yet, it throws an
+    // Exception, this will be modified with the actual implementation.
+    thrown.expect(DiskbalancerException.class);
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/49] hadoop git commit: HDFS-10557. Fix handling of the -fs Generic option. (Arpit Agarwal)

Posted by ar...@apache.org.
HDFS-10557. Fix handling of the -fs Generic option. (Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/66fa34c8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/66fa34c8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/66fa34c8

Branch: refs/heads/trunk
Commit: 66fa34c839c89733839cb67878fdfdc4b1f65ab8
Parents: 7b23ad1
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Jun 22 08:23:45 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:01 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/Command.java    | 27 +++-----------------
 .../diskbalancer/command/PlanCommand.java       | 11 ++++----
 .../diskbalancer/command/ReportCommand.java     |  6 ++---
 .../diskbalancer/planner/GreedyPlanner.java     |  8 +++---
 .../apache/hadoop/hdfs/tools/DiskBalancer.java  | 18 -------------
 .../src/site/markdown/HDFSDiskbalancer.md       |  6 ++---
 .../command/TestDiskBalancerCommand.java        | 23 +++++++----------
 7 files changed, 27 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
index 19f9945..3ea1b03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
@@ -45,9 +45,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.nio.charset.Charset;
 import java.nio.file.Files;
@@ -93,8 +91,7 @@ public abstract class Command extends Configured {
    * Executes the Client Calls.
    *
    * @param cmd - CommandLine
-   * @throws IOException
-   * @throws URISyntaxException
+   * @throws Exception
    */
   public abstract void execute(CommandLine cmd) throws Exception;
 
@@ -104,22 +101,6 @@ public abstract class Command extends Configured {
   public abstract void printHelp();
 
   /**
-   * verifies user provided URL.
-   *
-   * @param uri - UrlString
-   * @return URL
-   * @throws URISyntaxException, MalformedURLException
-   */
-  protected URI verifyURI(String uri)
-      throws URISyntaxException, MalformedURLException {
-    if ((uri == null) || uri.isEmpty()) {
-      throw new MalformedURLException(
-          "A valid URI is needed to execute this command.");
-    }
-    return new URI(uri);
-  }
-
-  /**
    * Process the URI and return the cluster with nodes setup. This is used in
    * all commands.
    *
@@ -130,11 +111,8 @@ public abstract class Command extends Configured {
   protected DiskBalancerCluster readClusterInfo(CommandLine cmd) throws
       Exception {
     Preconditions.checkNotNull(cmd);
-    Preconditions
-        .checkState(cmd.getOptionValue(DiskBalancer.NAMENODEURI) != null,
-            "Required argument missing : uri");
 
-    setClusterURI(verifyURI(cmd.getOptionValue(DiskBalancer.NAMENODEURI)));
+    setClusterURI(FileSystem.getDefaultUri(getConf()));
     LOG.debug("using name node URI : {}", this.getClusterURI());
     ClusterConnector connector = ConnectorFactory.getCluster(this.clusterURI,
         getConf());
@@ -346,6 +324,7 @@ public abstract class Command extends Configured {
    *
    * @param fileName - fileName to open.
    * @return OutputStream.
+   * @throws IOException
    */
   protected FSDataOutputStream create(String fileName) throws IOException {
     Preconditions.checkNotNull(fileName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index c13399b..20b4c6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -63,10 +63,9 @@ public class PlanCommand extends Command {
     this.thresholdPercentage = 1;
     this.bandwidth = 0;
     this.maxError = 0;
-    addValidCommandParameters(DiskBalancer.NAMENODEURI, "Name Node URI or " +
-        "file URI for cluster");
-
-    addValidCommandParameters(DiskBalancer.OUTFILE, "Output file");
+    addValidCommandParameters(DiskBalancer.OUTFILE, "Output directory in " +
+        "HDFS. The generated plan will be written to a file in this " +
+        "directory.");
     addValidCommandParameters(DiskBalancer.BANDWIDTH, "Maximum Bandwidth to " +
         "be used while copying.");
     addValidCommandParameters(DiskBalancer.THRESHOLD, "Percentage skew that " +
@@ -188,7 +187,7 @@ public class PlanCommand extends Command {
    */
   @Override
   public void printHelp() {
-    String header = "creates a plan that describes how much data should be " +
+    String header = "Creates a plan that describes how much data should be " +
         "moved between disks.\n\n";
 
     String footer = "\nPlan command creates a set of steps that represent a " +
@@ -196,7 +195,7 @@ public class PlanCommand extends Command {
         " will balance the data.";
 
     HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp("hdfs diskbalancer -uri <namenode> -plan " +
+    helpFormatter.printHelp("hdfs diskbalancer -plan " +
         "<hostname> [options]", header, DiskBalancer.getPlanOptions(), footer);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
index 40729f8..9f8e399 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -201,9 +201,9 @@ public class ReportCommand extends Command {
         " datanode, or prints out the list of nodes that will benefit from " +
         "running disk balancer. Top defaults to " + getDefaultTop();
     String footer = ". E.g.:\n"
-        + "hdfs diskbalancer -fs http://namenode.uri -report\n"
-        + "hdfs diskbalancer -fs http://namenode.uri -report -top 5\n"
-        + "hdfs diskbalancer -fs http://namenode.uri -report "
+        + "hdfs diskbalancer -report\n"
+        + "hdfs diskbalancer -report -top 5\n"
+        + "hdfs diskbalancer -report "
         + "-node {DataNodeID | IP | Hostname}";
 
     HelpFormatter helpFormatter = new HelpFormatter();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
index b3d51c4..0df9843 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
@@ -67,7 +67,8 @@ public class GreedyPlanner implements Planner {
     long startTime = Time.monotonicNow();
     NodePlan plan = new NodePlan(node.getDataNodeName(),
         node.getDataNodePort());
-    LOG.info("Starting plan for Node : " + node.getDataNodeUUID());
+    LOG.info("Starting plan for Node : {}:{}",
+        node.getDataNodeName(), node.getDataNodePort());
     while (node.isBalancingNeeded(this.threshold)) {
       for (DiskBalancerVolumeSet vSet : node.getVolumeSets().values()) {
         balanceVolumeSet(node, vSet, plan);
@@ -76,8 +77,9 @@ public class GreedyPlanner implements Planner {
 
     long endTime = Time.monotonicNow();
     String message = String
-        .format("Compute Plan for Node : %s took %d ms ",
-            node.getDataNodeUUID(), endTime - startTime);
+        .format("Compute Plan for Node : %s:%d took %d ms ",
+            node.getDataNodeName(), node.getDataNodePort(),
+            endTime - startTime);
     LOG.info(message);
     return plan;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
index 8a900ba..612aa2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
@@ -52,16 +52,6 @@ import java.io.PrintStream;
  */
 public class DiskBalancer extends Configured implements Tool {
   /**
-   * NameNodeURI can point to either a real namenode, or a json file that
-   * contains the diskBalancer data in json form, that jsonNodeConnector knows
-   * how to deserialize.
-   * <p>
-   * Expected formats are :
-   * <p>
-   * hdfs://namenode.uri or file:///data/myCluster.json
-   */
-  public static final String NAMENODEURI = "fs";
-  /**
    * Computes a plan for a given set of nodes.
    */
   public static final String PLAN = "plan";
@@ -275,13 +265,6 @@ public class DiskBalancer extends Configured implements Tool {
    */
   private void addPlanCommands(Options opt) {
 
-    Option uri = OptionBuilder.withLongOpt(NAMENODEURI)
-        .withDescription("Address of the Namenode. e,g. hdfs://namenode")
-        .hasArg()
-        .create();
-    getPlanOptions().addOption(uri);
-    opt.addOption(uri);
-
     Option plan = OptionBuilder.withLongOpt(PLAN)
         .withDescription("creates a plan for datanode.")
         .hasArg()
@@ -336,7 +319,6 @@ public class DiskBalancer extends Configured implements Tool {
   private void addHelpCommands(Options opt) {
     Option help = OptionBuilder.withLongOpt(HELP)
         .hasOptionalArg()
-        .withArgName(HELP)
         .withDescription("valid commands are plan | execute | query | cancel" +
             " | report")
         .create();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
index 522dc5a..c5f73cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
@@ -53,11 +53,9 @@ The following sections discusses what commands are supported by disk balancer
 
  The plan command can be run against a given datanode by running
 
- `hdfs diskbalancer -uri hdfs://mycluster.com -plan node1.mycluster.com`
+ `hdfs diskbalancer -plan node1.mycluster.com`
 
- uri is the address of the namenode and -plan points to the datanode that we
- need to plan for. By deafult, plan command writes its output to
- **/system/diskbalancer**.
+ The command accepts [Generic Options](../hadoop-common/CommandsManual.html#Generic_Options).
 
  The plan command also has a set of parameters that allows user to control
  the output and execution of the plan.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66fa34c8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index 3accbc2..c1c137d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -31,6 +31,7 @@ import java.util.Scanner;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -74,8 +75,7 @@ public class TestDiskBalancerCommand {
   /* test basic report */
   @Test(timeout=60000)
   public void testReportSimple() throws Exception {
-    final String cmdLine = String.format("hdfs diskbalancer -fs %s -report",
-        clusterJson.toString());
+    final String cmdLine = "hdfs diskbalancer -report";
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -103,8 +103,7 @@ public class TestDiskBalancerCommand {
   /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
   @Test(timeout=60000)
   public void testReportLessThanTotal() throws Exception {
-    final String cmdLine = String.format(
-        "hdfs diskbalancer -fs %s -report -top 32", clusterJson.toString());
+    final String cmdLine = "hdfs diskbalancer -report -top 32";
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -127,8 +126,7 @@ public class TestDiskBalancerCommand {
   /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
   @Test(timeout=60000)
   public void testReportMoreThanTotal() throws Exception {
-    final String cmdLine = String.format(
-        "hdfs diskbalancer -fs %s -report -top 128", clusterJson.toString());
+    final String cmdLine = "hdfs diskbalancer -report -top 128";
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -152,8 +150,7 @@ public class TestDiskBalancerCommand {
   /* test invalid top limit, e.g., -report -top xx */
   @Test(timeout=60000)
   public void testReportInvalidTopLimit() throws Exception {
-    final String cmdLine = String.format(
-        "hdfs diskbalancer -fs %s -report -top xx", clusterJson.toString());
+    final String cmdLine = "hdfs diskbalancer -report -top xx";
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -177,14 +174,11 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
   }
 
-  /* test -report -node DataNodeID */
   @Test(timeout=60000)
   public void testReportNode() throws Exception {
-    final String cmdLine = String
-        .format(
-            "hdfs diskbalancer -fs %s -report -node "
-                + "a87654a9-54c7-4693-8dd9-c9c7021dc340",
-            clusterJson.toString());
+    final String cmdLine =
+            "hdfs diskbalancer -report -node " +
+            "a87654a9-54c7-4693-8dd9-c9c7021dc340";
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -275,6 +269,7 @@ public class TestDiskBalancerCommand {
     org.apache.hadoop.hdfs.tools.DiskBalancer db =
         new org.apache.hadoop.hdfs.tools.DiskBalancer(conf);
 
+    FileSystem.setDefaultUri(conf, clusterJson);
     ByteArrayOutputStream bufOut = new ByteArrayOutputStream();
     PrintStream out = new PrintStream(bufOut);
     db.run(cmds, out);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/49] hadoop git commit: HDFS-9671. DiskBalancer: SubmitPlan implementation. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9671. DiskBalancer: SubmitPlan implementation. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2b1b2faf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2b1b2faf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2b1b2faf

Branch: refs/heads/trunk
Commit: 2b1b2faf76a7ff148650a7836935a85439f60c49
Parents: 66f0bb6
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Feb 22 11:45:51 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   |   4 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../server/datanode/DiskBalancerWorkItem.java   | 160 ++++++
 .../server/datanode/DiskBalancerWorkStatus.java |  87 +++
 .../hadoop/hdfs/server/datanode/WorkStatus.java |  85 ---
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   5 +
 ...tDatanodeProtocolServerSideTranslatorPB.java |   4 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  56 +-
 .../hdfs/server/datanode/DiskBalancer.java      | 542 +++++++++++++++++++
 .../diskbalancer/DiskBalancerConstants.java     |   9 +
 .../diskbalancer/DiskBalancerException.java     |  98 ++++
 .../diskbalancer/DiskbalancerException.java     |  86 ---
 .../datamodel/DiskBalancerCluster.java          |  14 +
 .../diskbalancer/TestDiskBalancerRPC.java       |  28 +-
 15 files changed, 984 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index dede89e..d8df7fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
-import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 
 /** An client-datanode protocol for block recovery
  */
@@ -182,7 +182,7 @@ public interface ClientDatanodeProtocol {
   /**
    * Gets the status of an executing diskbalancer Plan.
    */
-  WorkStatus queryDiskBalancerPlan() throws IOException;
+  DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException;
 
   /**
    * Gets a run-time configuration value from running diskbalancer instance.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index e7e0d94..786d834 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryP
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
@@ -345,8 +345,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
    *                  to zero allows datanode to use the value defined in
    *                  configration.
    * @param plan - Actual plan.
-   * @return Success or throws Exception.
-   * @throws Exception
+   * @throws IOException
    */
   @Override
   public void submitDiskBalancerPlan(String planID, long planVersion,
@@ -387,13 +386,14 @@ public class ClientDatanodeProtocolTranslatorPB implements
    * Gets the status of an executing diskbalancer Plan.
    */
   @Override
-  public WorkStatus queryDiskBalancerPlan() throws IOException {
+  public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException {
     try {
       QueryPlanStatusRequestProto request =
           QueryPlanStatusRequestProto.newBuilder().build();
       QueryPlanStatusResponseProto response =
           rpcProxy.queryDiskBalancerPlan(NULL_CONTROLLER, request);
-      return new WorkStatus(response.hasResult() ? response.getResult() : 0,
+      return new DiskBalancerWorkStatus(response.hasResult() ?
+          response.getResult() : 0,
           response.hasPlanID() ? response.getPlanID() : null,
           response.hasStatus() ? response.getStatus() : null,
           response.hasCurrentStatus() ? response.getCurrentStatus() : null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
new file mode 100644
index 0000000..11730e2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
@@ -0,0 +1,160 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.datanode;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+
+/**
+ * Keeps track of how much work has finished.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DiskBalancerWorkItem {
+  private final long bytesToCopy;
+  private long bytesCopied;
+  private long errorCount;
+  private String errMsg;
+  private long blocksCopied;
+
+  /**
+   * Constructs a DiskBalancerWorkItem.
+   *
+   * @param bytesToCopy - Total bytes to copy from a disk
+   * @param bytesCopied - Copied So far.
+   */
+  public DiskBalancerWorkItem(long bytesToCopy, long bytesCopied) {
+    this.bytesToCopy = bytesToCopy;
+    this.bytesCopied = bytesCopied;
+  }
+
+  /**
+   * Reads a DiskBalancerWorkItem Object from a Json String.
+   *
+   * @param json - Json String.
+   * @return DiskBalancerWorkItem Object
+   * @throws IOException
+   */
+  public static DiskBalancerWorkItem parseJson(String json) throws IOException {
+    Preconditions.checkNotNull(json);
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(json, DiskBalancerWorkItem.class);
+  }
+
+  /**
+   * Gets the error message.
+   */
+  public String getErrMsg() {
+    return errMsg;
+  }
+
+  /**
+   * Sets the error message.
+   *
+   * @param errMsg - Msg.
+   */
+  public void setErrMsg(String errMsg) {
+    this.errMsg = errMsg;
+  }
+
+  /**
+   * Returns the number of errors encountered.
+   *
+   * @return long
+   */
+  public long getErrorCount() {
+    return errorCount;
+  }
+
+  /**
+   * Incs Error Count.
+   */
+  public void incErrorCount() {
+    this.errorCount++;
+  }
+
+  /**
+   * Returns bytes copied so far.
+   *
+   * @return long
+   */
+  public long getBytesCopied() {
+    return bytesCopied;
+  }
+
+  /**
+   * Sets bytes copied so far.
+   *
+   * @param bytesCopied - long
+   */
+  public void setBytesCopied(long bytesCopied) {
+    this.bytesCopied = bytesCopied;
+  }
+
+  /**
+   * Increments bytesCopied by delta.
+   *
+   * @param delta - long
+   */
+  public void incCopiedSoFar(long delta) {
+    this.bytesCopied += delta;
+  }
+
+  /**
+   * Returns bytes to copy.
+   *
+   * @return - long
+   */
+  public long getBytesToCopy() {
+    return bytesToCopy;
+  }
+
+  /**
+   * Returns number of blocks copied for this DiskBalancerWorkItem.
+   *
+   * @return long count of blocks.
+   */
+  public long getBlocksCopied() {
+    return blocksCopied;
+  }
+
+  /**
+   * increments the number of blocks copied.
+   */
+  public void incBlocksCopied() {
+    blocksCopied++;
+  }
+
+  /**
+   * returns a serialized json string.
+   *
+   * @return String - json
+   * @throws IOException
+   */
+  public String toJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(this);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
new file mode 100644
index 0000000..6b29ce8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
@@ -0,0 +1,87 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.datanode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Helper class that reports how much work has has been done by the node.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DiskBalancerWorkStatus {
+  private final int result;
+  private final String planID;
+  private final String status;
+  private final String currentState;
+
+  /**
+   * Constructs a workStatus Object.
+   *
+   * @param result       - int
+   * @param planID       - Plan ID
+   * @param status       - Current Status
+   * @param currentState - Current State
+   */
+  public DiskBalancerWorkStatus(int result, String planID, String status,
+                                String currentState) {
+    this.result = result;
+    this.planID = planID;
+    this.status = status;
+    this.currentState = currentState;
+  }
+
+  /**
+   * Returns result.
+   *
+   * @return long
+   */
+  public int getResult() {
+    return result;
+  }
+
+  /**
+   * Returns planID.
+   *
+   * @return String
+   */
+  public String getPlanID() {
+    return planID;
+  }
+
+  /**
+   * Returns Status.
+   *
+   * @return String
+   */
+  public String getStatus() {
+    return status;
+  }
+
+  /**
+   * Gets current Status.
+   *
+   * @return - Json String
+   */
+  public String getCurrentState() {
+    return currentState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
deleted file mode 100644
index 259a311..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/WorkStatus.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * 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.hdfs.server.datanode;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Helper class that reports how much work has has been done by the node.
- */
-@InterfaceAudience.Private
-public class WorkStatus {
-  private int result;
-  private String planID;
-  private String status;
-  private String currentState;
-
-  /**
-   * Constructs a workStatus Object.
-   *
-   * @param result       - int
-   * @param planID       - Plan ID
-   * @param status       - Current Status
-   * @param currentState - Current State
-   */
-  public WorkStatus(int result, String planID, String status,
-                    String currentState) {
-    this.result = result;
-    this.planID = planID;
-    this.status = status;
-    this.currentState = currentState;
-  }
-
-  /**
-   * Returns result.
-   *
-   * @return long
-   */
-  public int getResult() {
-    return result;
-  }
-
-  /**
-   * Returns planID.
-   *
-   * @return String
-   */
-  public String getPlanID() {
-    return planID;
-  }
-
-  /**
-   * Returns Status.
-   *
-   * @return String
-   */
-  public String getStatus() {
-    return status;
-  }
-
-  /**
-   * Gets current Status.
-   *
-   * @return - Json String
-   */
-  public String getCurrentState() {
-    return currentState;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index d3bdedf..27de7d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -21,6 +21,9 @@ HDFS-1312 Change Log
 
     HDFS-9645. DiskBalancer: Add Query RPC. (Anu Engineer via Arpit Agarwal)
 
-    HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer
-    via Arpit Agarwal)
+    HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer via
+    Arpit Agarwal)
+
+    HDFS-9671skBalancer: SubmitPlan implementation. (Anu Engineer via
+    Arpit Agarwal)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f18a6c6..224ab3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -930,6 +930,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_DEFAULT =
       HdfsConstants.DEFAULT_DATA_SOCKET_SIZE;
 
+  // Disk Balancer Keys
+  public static final String DFS_DISK_BALANCER_ENABLED =
+      "dfs.disk.balancer.enabled";
+  public static final boolean DFS_DISK_BALANCER_ENABLED_DEFAULT = false;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 3246633..692fca3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBa
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
-import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
+import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 
 /**
  * Implementation for protobuf service that forwards requests
@@ -293,7 +293,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       RpcController controller, QueryPlanStatusRequestProto request)
       throws ServiceException {
     try {
-      WorkStatus result = impl.queryDiskBalancerPlan();
+      DiskBalancerWorkStatus result = impl.queryDiskBalancerPlan();
       return QueryPlanStatusResponseProto
           .newBuilder()
           .setResult(result.getResult())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 836dc81..8d805a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -169,7 +169,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.web.DatanodeHttpServer;
-import org.apache.hadoop.hdfs.server.diskbalancer.DiskbalancerException;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -386,6 +386,8 @@ public class DataNode extends ReconfigurableBase
   private static final int NUM_CORES = Runtime.getRuntime()
       .availableProcessors();
   private static final double CONGESTION_RATIO = 1.5;
+  private DiskBalancer diskBalancer;
+
 
   private static Tracer createTracer(Configuration conf) {
     return new Tracer.Builder("DataNode").
@@ -1022,7 +1024,33 @@ public class DataNode extends ReconfigurableBase
       directoryScanner.shutdown();
     }
   }
-  
+
+  /**
+   * Initilizes {@link DiskBalancer}.
+   * @param  data - FSDataSet
+   * @param conf - Config
+   */
+  private synchronized void initDiskBalancer(FsDatasetSpi data,
+                                             Configuration conf) {
+    if (this.diskBalancer != null) {
+      return;
+    }
+
+    DiskBalancer.BlockMover mover = new DiskBalancer.DiskBalancerMover(data,
+        conf);
+    this.diskBalancer = new DiskBalancer(getDatanodeUuid(), conf, mover);
+  }
+
+  /**
+   * Shutdown disk balancer.
+   */
+  private synchronized void shutdownDiskBalancer() {
+    if (this.diskBalancer != null) {
+      this.diskBalancer.shutdown();
+      this.diskBalancer = null;
+    }
+  }
+
   private void initDataXceiver(Configuration conf) throws IOException {
     // find free port or use privileged port provided
     TcpPeerServer tcpPeerServer;
@@ -1530,6 +1558,7 @@ public class DataNode extends ReconfigurableBase
     data.addBlockPool(nsInfo.getBlockPoolID(), conf);
     blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
     initDirectoryScanner(conf);
+    initDiskBalancer(data, conf);
   }
 
   List<BPOfferService> getAllBpOs() {
@@ -1867,6 +1896,7 @@ public class DataNode extends ReconfigurableBase
 
     // Terminate directory scanner and block scanner
     shutdownPeriodicScanners();
+    shutdownDiskBalancer();
 
     // Stop the web server
     if (httpServer != null) {
@@ -3296,31 +3326,30 @@ public class DataNode extends ReconfigurableBase
    * @param bandwidth - Max disk bandwidth to use, 0 means use value defined
    *                  in the configration.
    * @param plan - Actual plan
-   * @return  success or throws an exception.
-   * @throws Exception
+   * @throws IOException
    */
   @Override
   public void submitDiskBalancerPlan(String planID,
       long planVersion, long bandwidth, String plan) throws IOException {
 
-    // TODO : This will be replaced with actual code later.
-    // Right now throwing DiskbalancerException instead
-    // NotImplementedException to indicate the eventually disk balancer code
-    // will throw DiskbalancerException.
-    throw new DiskbalancerException("Not Implemented", 0);
+    checkSuperuserPrivilege();
+    // TODO : Support force option
+    this.diskBalancer.submitPlan(planID, planVersion, plan, bandwidth, false);
   }
 
   @Override
   public void cancelDiskBalancePlan(String planID) throws
       IOException {
     checkSuperuserPrivilege();
-    throw new DiskbalancerException("Not Implemented", 0);
+    throw new DiskBalancerException("Not Implemented",
+        DiskBalancerException.Result.INTERNAL_ERROR);
   }
 
   @Override
-  public WorkStatus queryDiskBalancerPlan() throws IOException {
+  public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException {
     checkSuperuserPrivilege();
-    throw new DiskbalancerException("Not Implemented", 0);
+    throw new DiskBalancerException("Not Implemented",
+        DiskBalancerException.Result.INTERNAL_ERROR);
   }
 
   /**
@@ -3334,6 +3363,7 @@ public class DataNode extends ReconfigurableBase
   @Override
   public String getDiskBalancerSetting(String key) throws IOException {
     checkSuperuserPrivilege();
-    throw new DiskbalancerException("Not Implemented", 0);
+    throw new DiskBalancerException("Not Implemented",
+        DiskBalancerException.Result.INTERNAL_ERROR);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
new file mode 100644
index 0000000..1c8ba4cf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -0,0 +1,542 @@
+/*
+ * 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
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.datanode;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
+import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.apache.hadoop.util.Time;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.*;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Worker class for Disk Balancer.
+ * <p/>
+ * Here is the high level logic executed by this class. Users can submit disk
+ * balancing plans using submitPlan calls. After a set of sanity checks the plan
+ * is admitted and put into workMap.
+ * <p/>
+ * The executePlan launches a thread that picks up work from workMap and hands
+ * it over to the BlockMover#copyBlocks function.
+ * <p/>
+ * Constraints :
+ * <p/>
+ * Only one plan can be executing in a datanode at any given time. This is
+ * ensured by checking the future handle of the worker thread in submitPlan.
+ */
+@InterfaceAudience.Private
+public class DiskBalancer {
+
+  private static final Log LOG = LogFactory.getLog(DiskBalancer.class);
+  private final FsDatasetSpi<?> dataset;
+  private final String dataNodeUUID;
+  private final BlockMover blockMover;
+  private final ReentrantLock lock;
+  private final ConcurrentHashMap<VolumePair, DiskBalancerWorkItem> workMap;
+  private boolean isDiskBalancerEnabled = false;
+  private ExecutorService scheduler;
+  private Future future;
+  private String planID;
+
+  /**
+   * Constructs a Disk Balancer object. This object takes care of reading a
+   * NodePlan and executing it against a set of volumes.
+   *
+   * @param dataNodeUUID - Data node UUID
+   * @param conf         - Hdfs Config
+   * @param blockMover   - Object that supports moving blocks.
+   */
+  public DiskBalancer(String dataNodeUUID,
+                      Configuration conf, BlockMover blockMover) {
+    this.blockMover = blockMover;
+    this.dataset = this.blockMover.getDataset();
+    this.dataNodeUUID = dataNodeUUID;
+    scheduler = Executors.newSingleThreadExecutor();
+    lock = new ReentrantLock();
+    workMap = new ConcurrentHashMap<>();
+    this.isDiskBalancerEnabled = conf.getBoolean(
+        DFSConfigKeys.DFS_DISK_BALANCER_ENABLED,
+        DFSConfigKeys.DFS_DISK_BALANCER_ENABLED_DEFAULT);
+  }
+
+  /**
+   * Shutdown  disk balancer services.
+   */
+  public void shutdown() {
+    lock.lock();
+    try {
+      this.isDiskBalancerEnabled = false;
+      if ((this.future != null) && (!this.future.isDone())) {
+        this.blockMover.setExitFlag();
+        shutdownExecutor();
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Shutdown the executor.
+   */
+  private void shutdownExecutor() {
+    scheduler.shutdown();
+    try {
+      if(!scheduler.awaitTermination(30, TimeUnit.SECONDS)) {
+        scheduler.shutdownNow();
+        if (!scheduler.awaitTermination(30, TimeUnit.SECONDS)) {
+          LOG.error("Disk Balancer : Scheduler did not terminate.");
+        }
+      }
+    } catch (InterruptedException ex) {
+      scheduler.shutdownNow();
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
+   * Takes a client submitted plan and converts into a set of work items that
+   * can be executed by the blockMover.
+   *
+   * @param planID      - A SHA512 of the plan string
+   * @param planVersion - version of the plan string - for future use.
+   * @param plan        - Actual Plan
+   * @param bandwidth   - BytesPerSec to copy
+   * @param force       - Skip some validations and execute the plan file.
+   * @throws DiskBalancerException
+   */
+  public void submitPlan(String planID, long planVersion, String plan,
+                         long bandwidth, boolean force)
+      throws DiskBalancerException {
+
+    lock.lock();
+    try {
+      checkDiskBalancerEnabled();
+      if ((this.future != null) && (!this.future.isDone())) {
+        LOG.error("Disk Balancer - Executing another plan, submitPlan failed.");
+        throw new DiskBalancerException("Executing another plan",
+            DiskBalancerException.Result.PLAN_ALREADY_IN_PROGRESS);
+      }
+      NodePlan nodePlan =
+          verifyPlan(planID, planVersion, plan, bandwidth, force);
+      createWorkPlan(nodePlan);
+      this.planID = planID;
+      executePlan();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Throws if Disk balancer is disabled.
+   *
+   * @throws DiskBalancerException
+   */
+  private void checkDiskBalancerEnabled()
+      throws DiskBalancerException {
+    if (!isDiskBalancerEnabled) {
+      LOG.error("Disk Balancer is not enabled.");
+      throw new DiskBalancerException("Disk Balancer is not enabled.",
+          DiskBalancerException.Result.DISK_BALANCER_NOT_ENABLED);
+    }
+  }
+
+  /**
+   * Verifies that user provided plan is valid.
+   *
+   * @param planID      - SHA 512 of the plan.
+   * @param planVersion - Version of the plan, for future use.
+   * @param plan        - Plan String in Json.
+   * @param bandwidth   - Max disk bandwidth to use per second.
+   * @param force       - Skip verifying when the plan was generated.
+   * @return a NodePlan Object.
+   * @throws DiskBalancerException
+   */
+  private NodePlan verifyPlan(String planID, long planVersion, String plan,
+                              long bandwidth, boolean force)
+      throws DiskBalancerException {
+
+    Preconditions.checkState(lock.isHeldByCurrentThread());
+    verifyPlanVersion(planVersion);
+    NodePlan nodePlan = verifyPlanHash(planID, plan);
+    if (!force) {
+      verifyTimeStamp(nodePlan);
+    }
+    verifyNodeUUID(nodePlan);
+    return nodePlan;
+  }
+
+  /**
+   * Verifies the plan version is something that we support.
+   *
+   * @param planVersion - Long version.
+   * @throws DiskBalancerException
+   */
+  private void verifyPlanVersion(long planVersion)
+      throws DiskBalancerException {
+    if ((planVersion < DiskBalancerConstants.DISKBALANCER_MIN_VERSION) ||
+        (planVersion > DiskBalancerConstants.DISKBALANCER_MAX_VERSION)) {
+      LOG.error("Disk Balancer - Invalid plan version.");
+      throw new DiskBalancerException("Invalid plan version.",
+          DiskBalancerException.Result.INVALID_PLAN_VERSION);
+    }
+  }
+
+  /**
+   * Verifies that plan matches the SHA512 provided by the client.
+   *
+   * @param planID - Sha512 Hex Bytes
+   * @param plan   - Plan String
+   * @throws DiskBalancerException
+   */
+  private NodePlan verifyPlanHash(String planID, String plan)
+      throws DiskBalancerException {
+    final long sha512Length = 128;
+    if (plan == null || plan.length() == 0) {
+      LOG.error("Disk Balancer -  Invalid plan.");
+      throw new DiskBalancerException("Invalid plan.",
+          DiskBalancerException.Result.INVALID_PLAN);
+    }
+
+    if ((planID == null) ||
+        (planID.length() != sha512Length) ||
+        !DigestUtils.sha512Hex(plan.getBytes(Charset.forName("UTF-8")))
+            .equalsIgnoreCase(planID)) {
+      LOG.error("Disk Balancer - Invalid plan hash.");
+      throw new DiskBalancerException("Invalid or mis-matched hash.",
+          DiskBalancerException.Result.INVALID_PLAN_HASH);
+    }
+
+    try {
+      return NodePlan.parseJson(plan);
+    } catch (IOException ex) {
+      throw new DiskBalancerException("Parsing plan failed.", ex,
+          DiskBalancerException.Result.MALFORMED_PLAN);
+    }
+  }
+
+  /**
+   * Verifies that this plan is not older than 24 hours.
+   *
+   * @param plan - Node Plan
+   */
+  private void verifyTimeStamp(NodePlan plan) throws DiskBalancerException {
+    long now = Time.now();
+    long planTime = plan.getTimeStamp();
+
+    // TODO : Support Valid Plan hours as a user configurable option.
+    if ((planTime +
+        (TimeUnit.HOURS.toMillis(
+            DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS))) < now) {
+      String hourString = "Plan was generated more than " +
+           Integer.toString(DiskBalancerConstants.DISKBALANCER_VALID_PLAN_HOURS)
+          +  " hours ago.";
+      LOG.error("Disk Balancer - " + hourString);
+      throw new DiskBalancerException(hourString,
+          DiskBalancerException.Result.OLD_PLAN_SUBMITTED);
+    }
+  }
+
+  /**
+   * Verify Node UUID.
+   *
+   * @param plan - Node Plan
+   */
+  private void verifyNodeUUID(NodePlan plan) throws DiskBalancerException {
+    if ((plan.getNodeUUID() == null) ||
+        !plan.getNodeUUID().equals(this.dataNodeUUID)) {
+      LOG.error("Disk Balancer - Plan was generated for another node.");
+      throw new DiskBalancerException(
+          "Plan was generated for another node.",
+          DiskBalancerException.Result.DATANODE_ID_MISMATCH);
+    }
+  }
+
+  /**
+   * Convert a node plan to DiskBalancerWorkItem that Datanode can execute.
+   *
+   * @param plan - Node Plan
+   */
+  private void createWorkPlan(NodePlan plan) throws DiskBalancerException {
+    Preconditions.checkState(lock.isHeldByCurrentThread());
+
+    // Cleanup any residual work in the map.
+    workMap.clear();
+    Map<String, FsVolumeSpi> pathMap = getStorageIDToVolumeMap();
+
+    for (Step step : plan.getVolumeSetPlans()) {
+      String sourceuuid = step.getSourceVolume().getUuid();
+      String destinationuuid = step.getDestinationVolume().getUuid();
+
+      FsVolumeSpi sourceVol = pathMap.get(sourceuuid);
+      if (sourceVol == null) {
+        LOG.error("Disk Balancer - Unable to find source volume. submitPlan " +
+            "failed.");
+        throw new DiskBalancerException("Unable to find source volume.",
+            DiskBalancerException.Result.INVALID_VOLUME);
+      }
+
+      FsVolumeSpi destVol = pathMap.get(destinationuuid);
+      if (destVol == null) {
+        LOG.error("Disk Balancer - Unable to find destination volume. " +
+            "submitPlan failed.");
+        throw new DiskBalancerException("Unable to find destination volume.",
+            DiskBalancerException.Result.INVALID_VOLUME);
+      }
+      createWorkPlan(sourceVol, destVol, step.getBytesToMove());
+    }
+  }
+
+  /**
+   * Returns a path to Volume Map.
+   *
+   * @return Map
+   * @throws DiskBalancerException
+   */
+  private Map<String, FsVolumeSpi> getStorageIDToVolumeMap()
+      throws DiskBalancerException {
+    Map<String, FsVolumeSpi> pathMap = new HashMap<>();
+    FsDatasetSpi.FsVolumeReferences references;
+    try {
+      synchronized (this.dataset) {
+        references = this.dataset.getFsVolumeReferences();
+        for (int ndx = 0; ndx < references.size(); ndx++) {
+          FsVolumeSpi vol = references.get(ndx);
+          pathMap.put(vol.getStorageID(), vol);
+        }
+        references.close();
+      }
+    } catch (IOException ex) {
+      LOG.error("Disk Balancer - Internal Error.", ex);
+      throw new DiskBalancerException("Internal error", ex,
+          DiskBalancerException.Result.INTERNAL_ERROR);
+    }
+    return pathMap;
+  }
+
+  /**
+   * Starts Executing the plan, exits when the plan is done executing.
+   */
+  private void executePlan() {
+    Preconditions.checkState(lock.isHeldByCurrentThread());
+    this.blockMover.setRunnable();
+    if (this.scheduler.isShutdown()) {
+      this.scheduler = Executors.newSingleThreadExecutor();
+    }
+
+    this.future = scheduler.submit(new Runnable() {
+      @Override
+      public void run() {
+        Thread.currentThread().setName("DiskBalancerThread");
+        LOG.info("Executing Disk balancer plan. Plan ID -  " + planID);
+
+        for (Map.Entry<VolumePair, DiskBalancerWorkItem> entry :
+            workMap.entrySet()) {
+          blockMover.copyBlocks(entry.getKey(), entry.getValue());
+        }
+      }
+    });
+  }
+
+  /**
+   * Insert work items to work map.
+   *
+   * @param source      - Source vol
+   * @param dest        - destination volume
+   * @param bytesToMove - number of bytes to move
+   */
+  private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest,
+                              long bytesToMove) throws DiskBalancerException {
+
+    if(source.getStorageID().equals(dest.getStorageID())) {
+      throw new DiskBalancerException("Same source and destination",
+          DiskBalancerException.Result.INVALID_MOVE);
+    }
+    VolumePair pair = new VolumePair(source, dest);
+
+    // In case we have a plan with more than
+    // one line of same <source, dest>
+    // we compress that into one work order.
+    if (workMap.containsKey(pair)) {
+      bytesToMove += workMap.get(pair).getBytesToCopy();
+    }
+
+    DiskBalancerWorkItem work = new DiskBalancerWorkItem(bytesToMove, 0);
+    workMap.put(pair, work);
+  }
+
+  /**
+   * BlockMover supports moving blocks across Volumes.
+   */
+  public interface BlockMover {
+    /**
+     * Copies blocks from a set of volumes.
+     *
+     * @param pair - Source and Destination Volumes.
+     * @param item - Number of bytes to move from volumes.
+     */
+    void copyBlocks(VolumePair pair, DiskBalancerWorkItem item);
+
+    /**
+     * Begin the actual copy operations. This is useful in testing.
+     */
+    void setRunnable();
+
+    /**
+     * Tells copyBlocks to exit from the copy routine.
+     */
+    void setExitFlag();
+
+    /**
+     * Returns a pointer to the current dataset we are operating against.
+     *
+     * @return FsDatasetSpi
+     */
+    FsDatasetSpi getDataset();
+  }
+
+  /**
+   * Holds references to actual volumes that we will be operating against.
+   */
+  static class VolumePair {
+    private final FsVolumeSpi source;
+    private final FsVolumeSpi dest;
+
+    /**
+     * Constructs a volume pair.
+     *
+     * @param source - Source Volume
+     * @param dest   - Destination Volume
+     */
+    public VolumePair(FsVolumeSpi source, FsVolumeSpi dest) {
+      this.source = source;
+      this.dest = dest;
+    }
+
+    /**
+     * gets source volume.
+     *
+     * @return volume
+     */
+    public FsVolumeSpi getSource() {
+      return source;
+    }
+
+    /**
+     * Gets Destination volume.
+     *
+     * @return volume.
+     */
+    public FsVolumeSpi getDest() {
+      return dest;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      VolumePair that = (VolumePair) o;
+      return source.equals(that.source) && dest.equals(that.dest);
+    }
+
+    @Override
+    public int hashCode() {
+      int result = source.getBasePath().hashCode();
+      result = 31 * result + dest.getBasePath().hashCode();
+      return result;
+    }
+  }
+
+  /**
+   * Actual DataMover class for DiskBalancer.
+   * <p/>
+   * TODO : Add implementation for this class. This is here as a place holder so
+   * that Datanode can make calls into this class.
+   */
+  public static class DiskBalancerMover implements BlockMover {
+    private final FsDatasetSpi dataset;
+
+    /**
+     * Constructs diskBalancerMover.
+     *
+     * @param dataset Dataset
+     * @param conf    Configuration
+     */
+    public DiskBalancerMover(FsDatasetSpi dataset, Configuration conf) {
+      this.dataset = dataset;
+      // TODO : Read Config values.
+    }
+
+    /**
+     * Copies blocks from a set of volumes.
+     *
+     * @param pair - Source and Destination Volumes.
+     * @param item - Number of bytes to move from volumes.
+     */
+    @Override
+    public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
+
+    }
+
+    /**
+     * Begin the actual copy operations. This is useful in testing.
+     */
+    @Override
+    public void setRunnable() {
+
+    }
+
+    /**
+     * Tells copyBlocks to exit from the copy routine.
+     */
+    @Override
+    public void setExitFlag() {
+
+    }
+
+    /**
+     * Returns a pointer to the current dataset we are operating against.
+     *
+     * @return FsDatasetSpi
+     */
+    @Override
+    public FsDatasetSpi getDataset() {
+      return dataset;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
index 553827e..7144a50 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
@@ -29,6 +29,15 @@ public final class DiskBalancerConstants {
   public static final String DISKBALANCER_VOLUME_NAME =
       "DiskBalancerVolumeName";
 
+  /** Min and Max Plan file versions that we know of. **/
+  public static final int DISKBALANCER_MIN_VERSION = 1;
+  public static final int DISKBALANCER_MAX_VERSION = 1;
+
+  /**
+   * We treat a plan as stale if it was generated before the hours
+   * defined by the constant below. Defaults to 24 hours.
+   */
+  public static final int DISKBALANCER_VALID_PLAN_HOURS = 24;
   // never constructed.
   private DiskBalancerConstants() {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
new file mode 100644
index 0000000..a5e1581
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -0,0 +1,98 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.hdfs.server.diskbalancer;
+
+import java.io.IOException;
+
+/**
+ * Disk Balancer Exceptions.
+ */
+public class DiskBalancerException extends IOException {
+  /** Possible results from DiskBalancer. **/
+  public enum Result {
+    DISK_BALANCER_NOT_ENABLED,
+    INVALID_PLAN_VERSION,
+    INVALID_PLAN,
+    INVALID_PLAN_HASH,
+    OLD_PLAN_SUBMITTED,
+    DATANODE_ID_MISMATCH,
+    MALFORMED_PLAN,
+    PLAN_ALREADY_IN_PROGRESS,
+    INVALID_VOLUME,
+    INVALID_MOVE,
+    INTERNAL_ERROR
+  }
+
+  private final Result result;
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   *                the {@link #getMessage()} method)
+   */
+  public DiskBalancerException(String message, Result result) {
+    super(message);
+    this.result = result;
+  }
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message and
+   * cause.
+   * <p/>
+   * <p> Note that the detail message associated with {@code cause} is
+   * <i>not</i>
+   * automatically incorporated into this exception's detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   *                the
+   *                {@link #getMessage()} method)
+   * @param cause   The cause (which is saved for later retrieval by the {@link
+   *                #getCause()} method).  (A null value is permitted, and
+   *                indicates that the cause is nonexistent or unknown.)
+   */
+  public DiskBalancerException(String message, Throwable cause, Result result) {
+    super(message, cause);
+    this.result = result;
+  }
+
+  /**
+   * Constructs an {@code IOException} with the specified cause and a detail
+   * message of {@code (cause==null ? null : cause.toString())} (which typically
+   * contains the class and detail message of {@code cause}). This
+   * constructor is useful for IO exceptions that are little more than
+   * wrappers for other throwables.
+   *
+   * @param cause The cause (which is saved for later retrieval by the {@link
+   *              #getCause()} method).  (A null value is permitted, and
+   *              indicates
+   *              that the cause is nonexistent or unknown.)
+   */
+  public DiskBalancerException(Throwable cause, Result result) {
+    super(cause);
+    this.result = result;
+  }
+
+  /**
+   * Returns the result.
+   * @return int
+   */
+  public Result getResult() {
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
deleted file mode 100644
index 9d47dc3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskbalancerException.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * 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.hdfs.server.diskbalancer;
-
-import java.io.IOException;
-
-/**
- * Disk Balancer Exceptions.
- */
-public class DiskbalancerException extends IOException {
-  private int result;
-
-  /**
-   * Constructs an {@code IOException} with the specified detail message.
-   *
-   * @param message The detail message (which is saved for later retrieval by
-   *                the
-   *                {@link #getMessage()} method)
-   */
-  public DiskbalancerException(String message, int result) {
-    super(message);
-    this.result = result;
-  }
-
-  /**
-   * Constructs an {@code IOException} with the specified detail message and
-   * cause.
-   * <p/>
-   * <p> Note that the detail message associated with {@code cause} is
-   * <i>not</i>
-   * automatically incorporated into this exception's detail message.
-   *
-   * @param message The detail message (which is saved for later retrieval by
-   *                the
-   *                {@link #getMessage()} method)
-   * @param cause   The cause (which is saved for later retrieval by the {@link
-   *                #getCause()} method).  (A null value is permitted, and
-   *                indicates that the cause is nonexistent or unknown.)
-   * @since 1.6
-   */
-  public DiskbalancerException(String message, Throwable cause, int result) {
-    super(message, cause);
-    this.result = result;
-  }
-
-  /**
-   * Constructs an {@code IOException} with the specified cause and a detail
-   * message of {@code (cause==null ? null : cause.toString())} (which typically
-   * contains the class and detail message of {@code cause}). This
-   * constructor is
-   * useful for IO exceptions that are little more than wrappers for other
-   * throwables.
-   *
-   * @param cause The cause (which is saved for later retrieval by the {@link
-   *              #getCause()} method).  (A null value is permitted, and
-   *              indicates
-   *              that the cause is nonexistent or unknown.)
-   * @since 1.6
-   */
-  public DiskbalancerException(Throwable cause, int result) {
-    super(cause);
-    this.result = result;
-  }
-
-  /**
-   * Returns the result.
-   * @return int
-   */
-  public int getResult() {
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
index af9e9af..c86fc9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
@@ -358,4 +358,18 @@ public class DiskBalancerCluster {
       return (10 - modValue) + threadRatio;
     }
   }
+
+  /**
+   * Returns a node by UUID.
+   * @param uuid - Node's UUID
+   * @return DiskBalancerDataNode.
+   */
+  public DiskBalancerDataNode getNodeByUUID(String uuid) {
+    for(DiskBalancerDataNode node : this.getNodes()) {
+      if(node.getDataNodeUUID().equals(uuid)) {
+        return node;
+      }
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b1b2faf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 143b776..dc24787 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer;
 
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -35,9 +36,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import java.io.IOException;
-import java.net.URI;
-
 public class TestDiskBalancerRPC {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
@@ -48,6 +46,7 @@ public class TestDiskBalancerRPC {
   @Before
   public void setUp() throws Exception {
     conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
   }
@@ -72,22 +71,19 @@ public class TestDiskBalancerRPC {
     Assert.assertEquals(cluster.getDataNodes().size(),
                                     diskBalancerCluster.getNodes().size());
     diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
-    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(dnIndex);
+
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
+        dataNode.getDatanodeUuid());
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
     NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
         ());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
-    final int planVersion = 0; // So far we support only one version.
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    final int planVersion = 1; // So far we support only one version.
 
     String planHash = DigestUtils.sha512Hex(plan.toJson());
 
-    // Since submitDiskBalancerPlan is not implemented yet, it throws an
-    // Exception, this will be modified with the actual implementation.
-    thrown.expect(DiskbalancerException.class);
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-
-
   }
 
   @Test
@@ -117,10 +113,10 @@ public class TestDiskBalancerRPC {
     // Exception, this will be modified with the actual implementation.
     try {
       dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-    } catch (DiskbalancerException ex) {
+    } catch (DiskBalancerException ex) {
       // Let us ignore this for time being.
     }
-    thrown.expect(DiskbalancerException.class);
+    thrown.expect(DiskBalancerException.class);
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
@@ -152,13 +148,13 @@ public class TestDiskBalancerRPC {
     // Exception, this will be modified with the actual implementation.
     try {
       dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-    } catch (DiskbalancerException ex) {
+    } catch (DiskBalancerException ex) {
       // Let us ignore this for time being.
     }
 
     // TODO : This will be fixed when we have implementation for this
     // function in server side.
-    thrown.expect(DiskbalancerException.class);
+    thrown.expect(DiskBalancerException.class);
     dataNode.queryDiskBalancerPlan();
   }
 
@@ -166,7 +162,7 @@ public class TestDiskBalancerRPC {
   public void testgetDiskBalancerSetting() throws Exception {
     final int dnIndex = 0;
     DataNode dataNode = cluster.getDataNodes().get(dnIndex);
-    thrown.expect(DiskbalancerException.class);
+    thrown.expect(DiskBalancerException.class);
     dataNode.getDiskBalancerSetting(
         DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/49] hadoop git commit: HDFS-9683. DiskBalancer: Add cancelPlan implementation. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9683. DiskBalancer: Add cancelPlan implementation. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/98476406
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/98476406
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/98476406

Branch: refs/heads/trunk
Commit: 9847640603ace60d169206a40a256f988b314983
Parents: e646c2e
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Mar 3 17:00:52 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |   3 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |   7 +-
 .../hdfs/server/datanode/DiskBalancer.java      |  26 +++
 .../diskbalancer/DiskBalancerException.java     |   3 +-
 .../diskbalancer/TestDiskBalancerRPC.java       | 213 ++++++++++++-------
 5 files changed, 173 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98476406/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index 07403cf..919d73e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -30,3 +30,6 @@ HDFS-1312 Change Log
     HDFS-9681. DiskBalancer: Add QueryPlan implementation. (Anu Engineer via
     Arpit Agarwal)
 
+    HDFS-9683. DiskBalancer: Add cancelPlan implementation. (Anu Engineer via
+    Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98476406/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 56585a8..126deb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3337,12 +3337,15 @@ public class DataNode extends ReconfigurableBase
     this.diskBalancer.submitPlan(planID, planVersion, plan, bandwidth, false);
   }
 
+  /**
+   * Cancels a running plan.
+   * @param planID - Hash string that identifies a plan
+   */
   @Override
   public void cancelDiskBalancePlan(String planID) throws
       IOException {
     checkSuperuserPrivilege();
-    throw new DiskBalancerException("Not Implemented",
-        DiskBalancerException.Result.INTERNAL_ERROR);
+    this.diskBalancer.cancelPlan(planID);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98476406/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index c01fb4e..81dbb2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -197,6 +197,32 @@ public class DiskBalancer {
   }
 
   /**
+   * Cancels a running plan.
+   * @param planID - Hash of the plan to cancel.
+   * @throws DiskBalancerException
+   */
+  public void cancelPlan(String planID) throws DiskBalancerException {
+    lock.lock();
+    try {
+      checkDiskBalancerEnabled();
+      if ((this.planID == null) || (!this.planID.equals(planID))) {
+        LOG.error("Disk Balancer - No such plan. Cancel plan failed. PlanID: " +
+            planID);
+        throw new DiskBalancerException("No such plan.",
+            DiskBalancerException.Result.NO_SUCH_PLAN);
+      }
+      if (!this.future.isDone()) {
+        this.blockMover.setExitFlag();
+        shutdownExecutor();
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+
+
+  /**
    * Throws if Disk balancer is disabled.
    *
    * @throws DiskBalancerException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98476406/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
index a5e1581..00fe53d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -35,7 +35,8 @@ public class DiskBalancerException extends IOException {
     PLAN_ALREADY_IN_PROGRESS,
     INVALID_VOLUME,
     INVALID_MOVE,
-    INTERNAL_ERROR
+    INTERNAL_ERROR,
+    NO_SUCH_PLAN
   }
 
   private final Result result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/98476406/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 974e973..e29b3b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -5,9 +5,9 @@
  * 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
@@ -37,6 +37,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
 
@@ -63,103 +64,163 @@ public class TestDiskBalancerRPC {
   }
 
   @Test
-  public void testSubmitTestRpc() throws Exception {
-    final int dnIndex = 0;
-    cluster.restartDataNode(dnIndex);
-    cluster.waitActive();
-    ClusterConnector nameNodeConnector =
-        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+  public void testSubmitPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+  }
 
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
-    diskBalancerCluster.readClusterInfo();
-    Assert.assertEquals(cluster.getDataNodes().size(),
-                                    diskBalancerCluster.getNodes().size());
-    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+  @Test
+  public void testSubmitPlanWithInvalidHash() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    char hashArray[] = planHash.toCharArray();
+    hashArray[0]++;
+    planHash = String.valueOf(hashArray);
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+    thrown.expect(DiskBalancerException.class);
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+  }
 
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
-    DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
-        dataNode.getDatanodeUuid());
-    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
-    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
-    final int planVersion = 1; // So far we support only one version.
+  @Test
+  public void testSubmitPlanWithInvalidVersion() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    planVersion++;
+    NodePlan plan = rpcTestHelper.getPlan();
+    thrown.expect(DiskBalancerException.class);
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+  }
 
-    String planHash = DigestUtils.sha512Hex(plan.toJson());
+  @Test
+  public void testSubmitPlanWithInvalidPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+    thrown.expect(DiskBalancerException.class);
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, "");
+  }
 
+  @Test
+  public void testCancelPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
     dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+    dataNode.cancelDiskBalancePlan(planHash);
   }
 
   @Test
-  public void testCancelTestRpc() throws Exception {
-    final int dnIndex = 0;
-    cluster.restartDataNode(dnIndex);
-    cluster.waitActive();
-    ClusterConnector nameNodeConnector =
-        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
-
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(nameNodeConnector);
-    diskBalancerCluster.readClusterInfo();
-    Assert.assertEquals(cluster.getDataNodes().size(),
-        diskBalancerCluster.getNodes().size());
-    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
-    DiskBalancerDataNode node = diskBalancerCluster.getNodes().get(0);
-    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
-    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
-
-    final int planVersion = 0; // So far we support only one version.
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
-    String planHash = DigestUtils.sha512Hex(plan.toJson());
-
-    // Since submitDiskBalancerPlan is not implemented yet, it throws an
-    // Exception, this will be modified with the actual implementation.
-    try {
-      dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
-    } catch (DiskBalancerException ex) {
-      // Let us ignore this for time being.
-    }
+  public void testCancelNonExistentPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    char hashArray[] = planHash.toCharArray();
+    hashArray[0]++;
+    planHash = String.valueOf(hashArray);
+    NodePlan plan = rpcTestHelper.getPlan();
     thrown.expect(DiskBalancerException.class);
     dataNode.cancelDiskBalancePlan(planHash);
   }
 
   @Test
-  public void testQueryTestRpc() throws Exception {
-    final int dnIndex = 0;
-    cluster.restartDataNode(dnIndex);
-    cluster.waitActive();
-    ClusterConnector nameNodeConnector =
-        ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
-
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
-        (nameNodeConnector);
-    diskBalancerCluster.readClusterInfo();
-    Assert.assertEquals(cluster.getDataNodes().size(),
-        diskBalancerCluster.getNodes().size());
-    diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
-    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
-    DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
-        dataNode.getDatanodeUuid());
-    GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
-    planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
-
-    final int planVersion = 1; // So far we support only one version.
-    String planHash = DigestUtils.sha512Hex(plan.toJson());
-      dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
+  public void testCancelEmptyPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = "";
+    NodePlan plan = rpcTestHelper.getPlan();
+    thrown.expect(DiskBalancerException.class);
+    dataNode.cancelDiskBalancePlan(planHash);
+  }
+
+
+  @Test
+  public void testQueryPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, 10, plan.toJson());
     DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
     Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
         status.getResult() == PLAN_DONE);
   }
 
   @Test
-  public void testgetDiskBalancerSetting() throws Exception {
+  public void testQueryPlanWithoutSubmit() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+
+    DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
+    Assert.assertTrue(status.getResult() == NO_PLAN);
+  }
+
+  @Test
+  public void testGetDiskBalancerSetting() throws Exception {
     final int dnIndex = 0;
     DataNode dataNode = cluster.getDataNodes().get(dnIndex);
     thrown.expect(DiskBalancerException.class);
     dataNode.getDiskBalancerSetting(
         DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
   }
+
+  private class RpcTestHelper {
+    private NodePlan plan;
+    private int planVersion;
+    private DataNode dataNode;
+    private String planHash;
+
+    public NodePlan getPlan() {
+      return plan;
+    }
+
+    public int getPlanVersion() {
+      return planVersion;
+    }
+
+    public DataNode getDataNode() {
+      return dataNode;
+    }
+
+    public String getPlanHash() {
+      return planHash;
+    }
+
+    public RpcTestHelper invoke() throws Exception {
+      final int dnIndex = 0;
+      cluster.restartDataNode(dnIndex);
+      cluster.waitActive();
+      ClusterConnector nameNodeConnector =
+          ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
+
+      DiskBalancerCluster diskBalancerCluster =
+          new DiskBalancerCluster(nameNodeConnector);
+      diskBalancerCluster.readClusterInfo();
+      Assert.assertEquals(cluster.getDataNodes().size(),
+          diskBalancerCluster.getNodes().size());
+      diskBalancerCluster.setNodesToProcess(diskBalancerCluster.getNodes());
+      dataNode = cluster.getDataNodes().get(dnIndex);
+      DiskBalancerDataNode node = diskBalancerCluster.getNodeByUUID(
+          dataNode.getDatanodeUuid());
+      GreedyPlanner planner = new GreedyPlanner(10.0f, node);
+      plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort());
+      planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
+      planVersion = 1;
+      planHash = DigestUtils.sha512Hex(plan.toJson());
+      return this;
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/49] hadoop git commit: HDFS-8821. Stop tracking CHANGES.txt in the HDFS-1312 feature branch. Contributed by Xiaobing Zhou

Posted by ar...@apache.org.
HDFS-8821. Stop tracking CHANGES.txt in the HDFS-1312 feature branch. Contributed by Xiaobing Zhou


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4b93ddae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4b93ddae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4b93ddae

Branch: refs/heads/trunk
Commit: 4b93ddae07ba4332f40f896542ee2c6d7bf899ed
Parents: 747227e
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Mar 8 20:35:59 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           | 35 --------------------
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |  1 -
 2 files changed, 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b93ddae/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
deleted file mode 100644
index 919d73e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ /dev/null
@@ -1,35 +0,0 @@
-HDFS-1312 Change Log
-
-  NEW FEATURES
-
-    HDFS-9420. Add DataModels for DiskBalancer. (Anu Engineer via szetszwo)
-
-    HDFS-9449. DiskBalancer: Add connectors. (Anu Engineer via szetszwo)
-
-    HDFS-9526. Fix jackson annotation imports. (Xiaobing Zhou via szetszwo)
-
-    HDFS-9611. DiskBalancer: Replace htrace json imports with jackson.
-    (Anu Engineer via Arpit Agarwal)
-
-    HDFS-9469. DiskBalancer: Add Planner. (Anu Engineer via Arpit Agarwal)
-
-    HDFS-9588. DiskBalancer: Add submitDiskbalancer RPC. (Anu Engineer via
-    Arpit Agarwal)
-
-    HDFS-9595. DiskBalancer: Add cancelPlan RPC. (Anu Engineer via
-    Arpit Agarwal)
-
-    HDFS-9645. DiskBalancer: Add Query RPC. (Anu Engineer via Arpit Agarwal)
-
-    HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer via
-    Arpit Agarwal)
-
-    HDFS-9671. DiskBalancer: SubmitPlan implementation. (Anu Engineer via
-    Arpit Agarwal)
-
-    HDFS-9681. DiskBalancer: Add QueryPlan implementation. (Anu Engineer via
-    Arpit Agarwal)
-
-    HDFS-9683. DiskBalancer: Add cancelPlan implementation. (Anu Engineer via
-    Arpit Agarwal)
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b93ddae/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 93e7392..e8397e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -397,7 +397,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/static/dataTables.bootstrap.css</exclude>
             <exclude>src/main/webapps/static/dataTables.bootstrap.js</exclude>
             <exclude>src/test/resources/diskBalancer/data-cluster-3node-3disk.json</exclude>
-            <exclude>HDFS-1312_CHANGES.txt</exclude>
           </excludes>
         </configuration>
       </plugin>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/49] hadoop git commit: HDFS-10520. DiskBalancer: Fix Checkstyle issues in test code. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10520. DiskBalancer: Fix Checkstyle issues in test code. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3225c24e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3225c24e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3225c24e

Branch: refs/heads/trunk
Commit: 3225c24e0efb8627ea84ba23ad09859942cd81f0
Parents: 7e2be5c
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Jun 15 15:28:22 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:00 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/DiskBalancerException.java     |  3 +
 .../server/datanode/SimulatedFSDataset.java     |  2 +-
 .../DiskBalancerResultVerifier.java             |  3 +
 .../diskbalancer/DiskBalancerTestUtil.java      |  4 +-
 .../server/diskbalancer/TestConnectors.java     | 29 +++++----
 .../server/diskbalancer/TestDataModels.java     | 23 ++++----
 .../server/diskbalancer/TestDiskBalancer.java   | 62 +++++++++++---------
 .../diskbalancer/TestDiskBalancerRPC.java       | 16 +++--
 .../TestDiskBalancerWithMockMover.java          | 29 +++++----
 .../hdfs/server/diskbalancer/TestPlanner.java   | 39 +++++++-----
 10 files changed, 120 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
index a55bcf3..a420b04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -22,6 +22,9 @@ import java.io.IOException;
  * Disk Balancer Exceptions.
  */
 public class DiskBalancerException extends IOException {
+  /**
+   * Results returned by the RPC layer of DiskBalancer.
+   */
   public enum Result {
     DISK_BALANCER_NOT_ENABLED,
     INVALID_PLAN_VERSION,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 24f4a52..0565260 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -1362,7 +1362,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block,
-    FsVolumeSpi destination) throws IOException {
+      FsVolumeSpi destination) throws IOException {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
index 5abb33c..22367ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
@@ -20,6 +20,9 @@ package org.apache.hadoop.hdfs.server.diskbalancer;
 import org.hamcrest.Description;
 import org.hamcrest.TypeSafeMatcher;
 
+/**
+ * Helps in verifying test results.
+ */
 public class DiskBalancerResultVerifier
     extends TypeSafeMatcher<DiskBalancerException> {
   private final DiskBalancerException.Result expectedResult;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
index 43bb184..5e98eb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
@@ -42,7 +42,7 @@ public class DiskBalancerTestUtil {
   public static final long TB = GB * 1024L;
   private static int[] diskSizes =
       {1, 2, 3, 4, 5, 6, 7, 8, 9, 100, 200, 300, 400, 500, 600, 700, 800, 900};
-  Random rand;
+  private Random rand;
   private String stringTable =
       "ABCDEDFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0987654321";
 
@@ -251,7 +251,7 @@ public class DiskBalancerTestUtil {
    * @throws IOException
    */
   public static void moveAllDataToDestVolume(FsDatasetSpi fsDataset,
-    FsVolumeSpi source, FsVolumeSpi dest) throws IOException {
+      FsVolumeSpi source, FsVolumeSpi dest) throws IOException {
 
     for (String blockPoolID : source.getBlockPoolList()) {
       FsVolumeSpi.BlockIterator sourceIter =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
index bec488e..d7bdbc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
@@ -30,11 +30,14 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+/**
+ * Test Class that tests connectors.
+ */
 public class TestConnectors {
   private MiniDFSCluster cluster;
-  final int numDatanodes = 3;
-  final int volumeCount = 2; // default volumes in MiniDFSCluster.
-  Configuration conf;
+  private final int numDatanodes = 3;
+  private final int volumeCount = 2; // default volumes in MiniDFSCluster.
+  private Configuration conf;
 
   @Before
   public void setup() throws IOException {
@@ -51,12 +54,12 @@ public class TestConnectors {
   }
 
   @Test
-  public void TestNameNodeConnector() throws Exception {
+  public void testNameNodeConnector() throws Exception {
     cluster.waitActive();
     ClusterConnector nameNodeConnector =
         ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
-        (nameNodeConnector);
+    DiskBalancerCluster diskBalancerCluster =
+        new DiskBalancerCluster(nameNodeConnector);
     diskBalancerCluster.readClusterInfo();
     Assert.assertEquals("Expected number of Datanodes not found.",
         numDatanodes, diskBalancerCluster.getNodes().size());
@@ -65,18 +68,18 @@ public class TestConnectors {
   }
 
   @Test
-  public void TestJsonConnector() throws Exception {
+  public void testJsonConnector() throws Exception {
     cluster.waitActive();
     ClusterConnector nameNodeConnector =
         ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
-    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster
-        (nameNodeConnector);
+    DiskBalancerCluster diskBalancerCluster =
+        new DiskBalancerCluster(nameNodeConnector);
     diskBalancerCluster.readClusterInfo();
     String diskBalancerJson = diskBalancerCluster.toJson();
-    DiskBalancerCluster serializedCluster = DiskBalancerCluster.parseJson
-        (diskBalancerJson);
+    DiskBalancerCluster serializedCluster =
+        DiskBalancerCluster.parseJson(diskBalancerJson);
     Assert.assertEquals("Parsed cluster is not equal to persisted info.",
-        diskBalancerCluster.getNodes().size(), serializedCluster.getNodes()
-            .size());
+        diskBalancerCluster.getNodes().size(),
+        serializedCluster.getNodes().size());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
index 3507c96..ace8212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
@@ -30,9 +30,12 @@ import java.util.List;
 import java.util.TreeSet;
 import java.util.UUID;
 
+/**
+ * Tests DiskBalancer Data models.
+ */
 public class TestDataModels {
   @Test
-  public void TestCreateRandomVolume() throws Exception {
+  public void testCreateRandomVolume() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerVolume vol = util.createRandomVolume(StorageType.DISK);
     Assert.assertNotNull(vol.getUuid());
@@ -46,7 +49,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestCreateRandomVolumeSet() throws Exception {
+  public void testCreateRandomVolumeSet() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerVolumeSet vSet =
         util.createRandomVolumeSet(StorageType.SSD, 10);
@@ -57,7 +60,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestCreateRandomDataNode() throws Exception {
+  public void testCreateRandomDataNode() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node = util.createRandomDataNode(
         new StorageType[]{StorageType.DISK, StorageType.RAM_DISK}, 10);
@@ -65,7 +68,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestDiskQueues() throws Exception {
+  public void testDiskQueues() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node = util.createRandomDataNode(
         new StorageType[]{StorageType.DISK, StorageType.RAM_DISK}, 3);
@@ -93,7 +96,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestNoBalancingNeededEvenDataSpread() throws Exception {
+  public void testNoBalancingNeededEvenDataSpread() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node =
         new DiskBalancerDataNode(UUID.randomUUID().toString());
@@ -119,7 +122,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestNoBalancingNeededTransientDisks() throws Exception {
+  public void testNoBalancingNeededTransientDisks() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node =
         new DiskBalancerDataNode(UUID.randomUUID().toString());
@@ -145,7 +148,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestNoBalancingNeededFailedDisks() throws Exception {
+  public void testNoBalancingNeededFailedDisks() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node =
         new DiskBalancerDataNode(UUID.randomUUID().toString());
@@ -172,7 +175,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestNeedBalancingUnevenDataSpread() throws Exception {
+  public void testNeedBalancingUnevenDataSpread() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerDataNode node =
         new DiskBalancerDataNode(UUID.randomUUID().toString());
@@ -196,7 +199,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestVolumeSerialize() throws Exception {
+  public void testVolumeSerialize() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
     DiskBalancerVolume volume = util.createRandomVolume(StorageType.DISK);
     String originalString = volume.toJson();
@@ -207,7 +210,7 @@ public class TestDataModels {
   }
 
   @Test
-  public void TestClusterSerialize() throws Exception {
+  public void testClusterSerialize() throws Exception {
     DiskBalancerTestUtil util = new DiskBalancerTestUtil();
 
     // Create a Cluster with 3 datanodes, 3 disk types and 3 disks in each type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
index bd969c7..1e10539 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
@@ -35,7 +35,8 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
 import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
-import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerDataNode;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -51,10 +52,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Test Disk Balancer.
+ */
 public class TestDiskBalancer {
 
   @Test
-  public void TestDiskBalancerNameNodeConnectivity() throws Exception {
+  public void testDiskBalancerNameNodeConnectivity() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
     final int numDatanodes = 2;
@@ -65,13 +69,13 @@ public class TestDiskBalancer {
       ClusterConnector nameNodeConnector =
           ConnectorFactory.getCluster(cluster.getFileSystem(0).getUri(), conf);
 
-      DiskBalancerCluster DiskBalancerCluster = new DiskBalancerCluster
-          (nameNodeConnector);
-      DiskBalancerCluster.readClusterInfo();
-      assertEquals(DiskBalancerCluster.getNodes().size(), numDatanodes);
+      DiskBalancerCluster diskBalancerCluster =
+          new DiskBalancerCluster(nameNodeConnector);
+      diskBalancerCluster.readClusterInfo();
+      assertEquals(diskBalancerCluster.getNodes().size(), numDatanodes);
       DataNode dnNode = cluster.getDataNodes().get(0);
       DiskBalancerDataNode dbDnNode =
-          DiskBalancerCluster.getNodeByUUID(dnNode.getDatanodeUuid());
+          diskBalancerCluster.getNodeByUUID(dnNode.getDatanodeUuid());
       assertEquals(dnNode.getDatanodeUuid(), dbDnNode.getDataNodeUUID());
       assertEquals(dnNode.getDatanodeId().getIpAddr(),
           dbDnNode.getDataNodeIP());
@@ -88,24 +92,23 @@ public class TestDiskBalancer {
 
   /**
    * This test simulates a real Data node working with DiskBalancer.
-   *
+   * <p>
    * Here is the overview of this test.
-   *
+   * <p>
    * 1. Write a bunch of blocks and move them to one disk to create imbalance.
-   * 2. Rewrite  the capacity of the disks in DiskBalancer Model so that
-   * planner will produce a move plan.
-   * 3. Execute the move plan and wait unitl the plan is done.
-   * 4. Verify the source disk has blocks now.
+   * 2. Rewrite  the capacity of the disks in DiskBalancer Model so that planner
+   * will produce a move plan. 3. Execute the move plan and wait unitl the plan
+   * is done. 4. Verify the source disk has blocks now.
    *
    * @throws Exception
    */
   @Test
-  public void TestDiskBalancerEndToEnd() throws Exception {
+  public void testDiskBalancerEndToEnd() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    final int DEFAULT_BLOCK_SIZE = 100;
+    final int defaultBlockSize = 100;
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, defaultBlockSize);
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
     final int numDatanodes = 1;
     final String fileName = "/tmp.txt";
@@ -116,12 +119,12 @@ public class TestDiskBalancer {
 
 
     // Write a file and restart the cluster
-    long [] capacities = new long[]{ DEFAULT_BLOCK_SIZE * 2 * fileLen,
-        DEFAULT_BLOCK_SIZE * 2 * fileLen };
+    long[] capacities = new long[]{defaultBlockSize * 2 * fileLen,
+        defaultBlockSize * 2 * fileLen};
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numDatanodes)
         .storageCapacities(capacities)
-        .storageTypes(new StorageType[] {StorageType.DISK, StorageType.DISK})
+        .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})
         .storagesPerDatanode(2)
         .build();
     FsVolumeImpl source = null;
@@ -144,9 +147,9 @@ public class TestDiskBalancer {
         source = (FsVolumeImpl) refs.get(0);
         dest = (FsVolumeImpl) refs.get(1);
         assertTrue(DiskBalancerTestUtil.getBlockCount(source) > 0);
-        DiskBalancerTestUtil.moveAllDataToDestVolume(
-            dnNode.getFSDataset(), source, dest);
-       assertTrue(DiskBalancerTestUtil.getBlockCount(source) == 0);
+        DiskBalancerTestUtil.moveAllDataToDestVolume(dnNode.getFSDataset(),
+            source, dest);
+        assertTrue(DiskBalancerTestUtil.getBlockCount(source) == 0);
       }
 
       cluster.restartDataNodes();
@@ -164,7 +167,8 @@ public class TestDiskBalancer {
 
       // Rewrite the capacity in the model to show that disks need
       // re-balancing.
-      setVolumeCapacity(diskBalancerCluster, DEFAULT_BLOCK_SIZE * 2 * fileLen, "DISK");
+      setVolumeCapacity(diskBalancerCluster, defaultBlockSize * 2 * fileLen,
+          "DISK");
       // Pick a node to process.
       nodesToProcess.add(diskBalancerCluster.getNodeByUUID(dnNode
           .getDatanodeUuid()));
@@ -220,13 +224,12 @@ public class TestDiskBalancer {
       }
 
 
-
       // Tolerance
       long delta = (plan.getVolumeSetPlans().get(0).getBytesToMove()
           * 10) / 100;
       assertTrue(
           (DiskBalancerTestUtil.getBlockCount(source) *
-              DEFAULT_BLOCK_SIZE + delta) >=
+              defaultBlockSize + delta) >=
               plan.getVolumeSetPlans().get(0).getBytesToMove());
 
     } finally {
@@ -236,13 +239,14 @@ public class TestDiskBalancer {
 
   /**
    * Sets alll Disks capacity to size specified.
-   * @param cluster   - DiskBalancerCluster
-   * @param size   - new size of the disk
+   *
+   * @param cluster - DiskBalancerCluster
+   * @param size    - new size of the disk
    */
   private void setVolumeCapacity(DiskBalancerCluster cluster, long size,
                                  String diskType) {
     Preconditions.checkNotNull(cluster);
-    for(DiskBalancerDataNode node : cluster.getNodes()) {
+    for (DiskBalancerDataNode node : cluster.getNodes()) {
       for (DiskBalancerVolume vol :
           node.getVolumeSets().get(diskType).getVolumes()) {
         vol.setCapacity(size);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index 81a0609..ce40204 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -53,6 +53,9 @@ import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Resu
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_UNDER_PROGRESS;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Test DiskBalancer RPC.
+ */
 public class TestDiskBalancerRPC {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
@@ -91,7 +94,7 @@ public class TestDiskBalancerRPC {
     RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
     DataNode dataNode = rpcTestHelper.getDataNode();
     String planHash = rpcTestHelper.getPlanHash();
-    char hashArray[] = planHash.toCharArray();
+    char[] hashArray = planHash.toCharArray();
     hashArray[0]++;
     planHash = String.valueOf(hashArray);
     int planVersion = rpcTestHelper.getPlanVersion();
@@ -126,7 +129,8 @@ public class TestDiskBalancerRPC {
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN));
     dataNode.submitDiskBalancerPlan(planHash, planVersion, "",
-        false);  }
+        false);
+  }
 
   @Test
   public void testCancelPlan() throws Exception {
@@ -145,7 +149,7 @@ public class TestDiskBalancerRPC {
     RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
     DataNode dataNode = rpcTestHelper.getDataNode();
     String planHash = rpcTestHelper.getPlanHash();
-    char hashArray[] = planHash.toCharArray();
+    char[] hashArray= planHash.toCharArray();
     hashArray[0]++;
     planHash = String.valueOf(hashArray);
     NodePlan plan = rpcTestHelper.getPlan();
@@ -234,10 +238,10 @@ public class TestDiskBalancerRPC {
   @Test
   public void testMoveBlockAcrossVolume() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    final int DEFAULT_BLOCK_SIZE = 100;
+    final int defaultBlockSize = 100;
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultBlockSize);
+    conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, defaultBlockSize);
     String fileName = "/tmp.txt";
     Path filePath = new Path(fileName);
     final int numDatanodes = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
index b73b290..564b985 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -59,19 +59,22 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Tests diskbalancer with a mock mover.
+ */
 public class TestDiskBalancerWithMockMover {
   static final Log LOG = LogFactory.getLog(TestDiskBalancerWithMockMover.class);
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
-  MiniDFSCluster cluster;
-  String sourceName;
-  String destName;
-  String sourceUUID;
-  String destUUID;
-  String nodeID;
-  DataNode dataNode;
+  private MiniDFSCluster cluster;
+  private String sourceName;
+  private String destName;
+  private String sourceUUID;
+  private String destUUID;
+  private String nodeID;
+  private DataNode dataNode;
 
   /**
    * Checks that we return the right error if diskbalancer is not enabled.
@@ -178,12 +181,12 @@ public class TestDiskBalancerWithMockMover {
 
   @Test
   public void testSubmitWithOlderPlan() throws Exception {
-    final long MILLISECOND_IN_AN_HOUR = 1000 * 60 * 60L;
+    final long millisecondInAnHour = 1000 * 60 * 60L;
     MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
     NodePlan plan = mockMoverHelper.getPlan();
     DiskBalancer balancer = mockMoverHelper.getBalancer();
 
-    plan.setTimeStamp(Time.now() - (32 * MILLISECOND_IN_AN_HOUR));
+    plan.setTimeStamp(Time.now() - (32 * millisecondInAnHour));
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
         .Result.OLD_PLAN_SUBMITTED));
@@ -316,10 +319,10 @@ public class TestDiskBalancerWithMockMover {
   @Before
   public void setUp() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    final int NUM_STORAGES_PER_DN = 2;
+    final int numStoragesPerDn = 2;
     cluster = new MiniDFSCluster
         .Builder(conf).numDataNodes(3)
-        .storagesPerDatanode(NUM_STORAGES_PER_DN)
+        .storagesPerDatanode(numStoragesPerDn)
         .build();
     cluster.waitActive();
     dataNode = cluster.getDataNodes().get(0);
@@ -602,8 +605,8 @@ public class TestDiskBalancerWithMockMover {
       DiskBalancerDataNode node = balancerCluster.getNodes().get(dnIndex);
       node.setDataNodeUUID(nodeID);
       GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-      NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-          ());
+      NodePlan plan = new NodePlan(node.getDataNodeName(),
+          node.getDataNodePort());
       planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
       setVolumeNames(plan);
       return plan;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3225c24e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
index d01e8ad..c722df7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
@@ -40,6 +40,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Test Planner.
+ */
 public class TestPlanner {
   static final Logger LOG =
       LoggerFactory.getLogger(TestPlanner.class);
@@ -56,8 +59,8 @@ public class TestPlanner {
     cluster.setNodesToProcess(cluster.getNodes());
     DiskBalancerDataNode node = cluster.getNodes().get(0);
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
   }
 
@@ -115,8 +118,8 @@ public class TestPlanner {
     Assert.assertEquals(1, cluster.getNodes().size());
 
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     // With a single volume we should not have any plans for moves.
@@ -183,8 +186,8 @@ public class TestPlanner {
     Assert.assertEquals(1, cluster.getNodes().size());
 
     GreedyPlanner planner = new GreedyPlanner(5.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeUUID(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeUUID(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     // We should have only one planned move from
@@ -219,8 +222,8 @@ public class TestPlanner {
     Assert.assertEquals(1, cluster.getNodes().size());
 
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     // since we have same size of data in all disks , we should have
@@ -250,8 +253,8 @@ public class TestPlanner {
     Assert.assertEquals(1, cluster.getNodes().size());
 
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     // We should see 2 move plans. One from volume100 to volume0-1
@@ -260,10 +263,12 @@ public class TestPlanner {
     assertEquals(2, plan.getVolumeSetPlans().size());
     Step step = plan.getVolumeSetPlans().get(0);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertTrue(step.getSizeString(step.getBytesToMove()).matches("33.[2|3|4] G"));
+    assertTrue(step.getSizeString(
+        step.getBytesToMove()).matches("33.[2|3|4] G"));
     step = plan.getVolumeSetPlans().get(1);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertTrue(step.getSizeString(step.getBytesToMove()).matches("33.[2|3|4] G"));
+    assertTrue(step.getSizeString(
+        step.getBytesToMove()).matches("33.[2|3|4] G"));
   }
 
   @Test
@@ -287,8 +292,8 @@ public class TestPlanner {
     Assert.assertEquals(1, cluster.getNodes().size());
 
     GreedyPlanner planner = new GreedyPlanner(10.0f, node);
-    NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
-        ());
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
     planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
 
     //We should see NO moves since the total data on the volume100
@@ -315,11 +320,13 @@ public class TestPlanner {
 
     Step step = newPlan.getVolumeSetPlans().get(0);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertTrue(step.getSizeString(step.getBytesToMove()).matches("18.[6|7|8] G"));
+    assertTrue(step.getSizeString(
+        step.getBytesToMove()).matches("18.[6|7|8] G"));
 
     step = newPlan.getVolumeSetPlans().get(1);
     assertEquals("volume100", step.getSourceVolume().getPath());
-    assertTrue(step.getSizeString(step.getBytesToMove()).matches("18.[6|7|8] G"));
+    assertTrue(
+        step.getSizeString(step.getBytesToMove()).matches("18.[6|7|8] G"));
 
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/49] hadoop git commit: HDFS-10399. DiskBalancer: Add JMX for DiskBalancer. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10399. DiskBalancer: Add JMX for DiskBalancer. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5df2d2b8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5df2d2b8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5df2d2b8

Branch: refs/heads/trunk
Commit: 5df2d2b8fd2932fb6e931d948fb6620ab7bcf0bb
Parents: 1b39b28
Author: Anu Engineer <ae...@apache.org>
Authored: Fri May 20 08:53:28 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../server/datanode/DiskBalancerWorkItem.java   | 10 ++++++-
 .../server/datanode/DiskBalancerWorkStatus.java | 30 +++++++++++++++++++-
 ...tDatanodeProtocolServerSideTranslatorPB.java |  2 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   | 10 +++++++
 .../hdfs/server/datanode/DataNodeMXBean.java    |  8 ++++++
 .../server/diskbalancer/TestDiskBalancer.java   |  6 ++++
 .../TestDiskBalancerWithMockMover.java          | 22 ++++++++------
 7 files changed, 77 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
index 7381499..fe908d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
@@ -34,7 +34,7 @@ import java.io.IOException;
 @InterfaceStability.Unstable
 @JsonInclude(JsonInclude.Include.NON_DEFAULT)
 public class DiskBalancerWorkItem {
-  private final long bytesToCopy;
+  private long bytesToCopy;
   private long bytesCopied;
   private long errorCount;
   private String errMsg;
@@ -45,6 +45,14 @@ public class DiskBalancerWorkItem {
   private long bandwidth;
 
   /**
+   * Empty constructor for Json serialization.
+   */
+  public DiskBalancerWorkItem() {
+
+  }
+
+
+  /**
    * Constructs a DiskBalancerWorkItem.
    *
    * @param bytesToCopy - Total bytes to copy from a disk

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
index d6943cf..ca5e5f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
@@ -126,11 +126,29 @@ public class DiskBalancerWorkStatus {
    *
    * @throws IOException
    **/
-  public String getCurrentStateString() throws IOException {
+  public String currentStateString() throws IOException {
     ObjectMapper mapper = new ObjectMapper();
     return mapper.writeValueAsString(currentState);
   }
 
+  public String toJsonString() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(this);
+
+  }
+
+  /**
+   * Returns a DiskBalancerWorkStatus object from the Json .
+   * @param json - json String
+   * @return DiskBalancerWorkStatus
+   * @throws IOException
+   */
+  public static DiskBalancerWorkStatus parseJson(String json) throws
+      IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(json, DiskBalancerWorkStatus.class);
+  }
+
 
   /**
    * Adds a new work entry to the list.
@@ -177,6 +195,16 @@ public class DiskBalancerWorkStatus {
     private DiskBalancerWorkItem workItem;
 
     /**
+     * Constructor needed for json serialization.
+     */
+    public DiskBalancerWorkEntry() {
+    }
+
+    public DiskBalancerWorkEntry(String workItem) throws IOException {
+      this.workItem = DiskBalancerWorkItem.parseJson(workItem);
+    }
+
+    /**
      * Constructs a Work Entry class.
      *
      * @param sourcePath - Source Path where we are moving data from.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 482e86f..b716347 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -298,7 +298,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
           .newBuilder()
           .setResult(result.getResult().getIntResult())
           .setPlanID(result.getPlanID())
-          .setCurrentStatus(result.getCurrentStateString())
+          .setCurrentStatus(result.currentStateString())
           .build();
     } catch (Exception e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index d6be2e0..94de27b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -2958,6 +2958,16 @@ public class DataNode extends ReconfigurableBase
   public synchronized String getClusterId() {
     return clusterId;
   }
+
+  @Override // DataNodeMXBean
+  public String getDiskBalancerStatus() {
+    try {
+      return this.diskBalancer.queryWorkStatus().toJsonString();
+    } catch (IOException ex) {
+      LOG.debug("Reading diskbalancer Status failed. ex:{}", ex);
+      return "";
+    }
+  }
   
   public void refreshNamenodes(Configuration conf) throws IOException {
     blockPoolManager.refreshNamenodes(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
index 0119dc8..a59dc1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
@@ -90,4 +90,12 @@ public interface DataNodeMXBean {
    * Gets the network error counts on a per-Datanode basis.
    */
   public Map<String, Map<String, Long>> getDatanodeNetworkCounts();
+
+  /**
+   * Gets the diskBalancer Status.
+   * Please see implementation for the format of the returned information.
+   *
+   * @return  DiskBalancer Status
+   */
+  String getDiskBalancerStatus();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
index f50637c..bd969c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
@@ -189,6 +189,12 @@ public class TestDiskBalancer {
 
       // Submit the plan and wait till the execution is done.
       newDN.submitDiskBalancerPlan(planID, 1, planJson, false);
+      String jmxString = newDN.getDiskBalancerStatus();
+      assertNotNull(jmxString);
+      DiskBalancerWorkStatus status =
+          DiskBalancerWorkStatus.parseJson(jmxString);
+      DiskBalancerWorkStatus realStatus = newDN.queryDiskBalancerPlan();
+      assertEquals(realStatus.getPlanID(), status.getPlanID());
 
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5df2d2b8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
index 1cc90e5..491fccb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hdfs.server.diskbalancer;
 
 import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -40,6 +41,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
 import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
@@ -53,7 +55,6 @@ import java.util.Iterator;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.NO_PLAN;
-import static org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus.Result.PLAN_DONE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -156,15 +157,20 @@ public class TestDiskBalancerWithMockMover {
   public void testSubmitDiskBalancerPlan() throws Exception {
     MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
     NodePlan plan = mockMoverHelper.getPlan();
-    DiskBalancer balancer = mockMoverHelper.getBalancer();
+    final DiskBalancer balancer = mockMoverHelper.getBalancer();
 
     executeSubmitPlan(plan, balancer);
-    int counter = 0;
-    while ((balancer.queryWorkStatus().getResult() != PLAN_DONE) &&
-        (counter < 3)) {
-      Thread.sleep(1000);
-      counter++;
-    }
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          return balancer.queryWorkStatus().getResult() ==
+              DiskBalancerWorkStatus.Result.PLAN_DONE;
+        } catch (IOException ex) {
+          return false;
+        }
+      }
+    }, 1000, 100000);
 
     // Asserts that submit plan caused an execution in the background.
     assertTrue(mockMoverHelper.getBlockMover().getRunCount() == 1);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/49] hadoop git commit: HDFS-10170. DiskBalancer: Force rebase diskbalancer branch. Contributed by Anu Engineer

Posted by ar...@apache.org.
HDFS-10170. DiskBalancer: Force rebase diskbalancer branch. Contributed by Anu Engineer


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ec601673
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ec601673
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ec601673

Branch: refs/heads/trunk
Commit: ec601673a84832a47e6de0b0e34d0804cfed279d
Parents: 75a711a
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Mar 15 11:45:24 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/DiskBalancerException.java     | 32 +++++++++-----------
 1 file changed, 14 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec601673/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
index 38455a7..c3571c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
@@ -1,19 +1,18 @@
 /**
- * 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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 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.
+ * 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.hdfs.server.diskbalancer;
 
@@ -23,7 +22,6 @@ import java.io.IOException;
  * Disk Balancer Exceptions.
  */
 public class DiskBalancerException extends IOException {
-  /** Possible results from DiskBalancer. **/
   public enum Result {
     DISK_BALANCER_NOT_ENABLED,
     INVALID_PLAN_VERSION,
@@ -35,9 +33,7 @@ public class DiskBalancerException extends IOException {
     PLAN_ALREADY_IN_PROGRESS,
     INVALID_VOLUME,
     INVALID_MOVE,
-    INTERNAL_ERROR,
-    NO_SUCH_PLAN,
-    UNKNOWN_KEY
+    INTERNAL_ERROR
   }
 
   private final Result result;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/49] hadoop git commit: HDFS-10550. DiskBalancer: fix issue of order dependency in iteration in ReportCommand test. Contributed by Xiaobing Zhou.

Posted by ar...@apache.org.
HDFS-10550. DiskBalancer: fix issue of order dependency in iteration in ReportCommand test. Contributed by Xiaobing Zhou.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7b23ad1e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7b23ad1e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7b23ad1e

Branch: refs/heads/trunk
Commit: 7b23ad1ef76739bc04374aefe6e8e88c23449653
Parents: c6ed548
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 21 17:34:58 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:27:01 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/ReportCommand.java     | 17 +++--
 .../command/TestDiskBalancerCommand.java        | 74 +++++++++-----------
 2 files changed, 44 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b23ad1e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
index eb6afcc..40729f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 
 import java.io.PrintStream;
 import java.util.Collections;
+import java.util.List;
 import java.util.ListIterator;
 
 import org.apache.commons.cli.CommandLine;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSe
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 /**
  * Executes the report command.
@@ -164,9 +166,10 @@ public class ReportCommand extends Command {
             dbdn.getVolumeCount(),
             dbdn.getNodeDataDensity()));
 
+        List<String> volumeList = Lists.newArrayList();
         for (DiskBalancerVolumeSet vset : dbdn.getVolumeSets().values()) {
           for (DiskBalancerVolume vol : vset.getVolumes()) {
-            result.appendln(String.format(volumeFormat,
+            volumeList.add(String.format(volumeFormat,
                 vol.getStorageType(),
                 vol.getPath(),
                 vol.getUsedRatio(),
@@ -181,6 +184,10 @@ public class ReportCommand extends Command {
                 vol.isTransient() ? trueStr : falseStr));
           }
         }
+
+        Collections.sort(volumeList);
+        result.appendln(
+            StringUtils.join(volumeList.toArray(), System.lineSeparator()));
       }
     }
   }
@@ -194,13 +201,13 @@ public class ReportCommand extends Command {
         " datanode, or prints out the list of nodes that will benefit from " +
         "running disk balancer. Top defaults to " + getDefaultTop();
     String footer = ". E.g.:\n"
-        + "hdfs diskbalancer -uri http://namenode.uri -report\n"
-        + "hdfs diskbalancer -uri http://namenode.uri -report -top 5\n"
-        + "hdfs diskbalancer -uri http://namenode.uri -report "
+        + "hdfs diskbalancer -fs http://namenode.uri -report\n"
+        + "hdfs diskbalancer -fs http://namenode.uri -report -top 5\n"
+        + "hdfs diskbalancer -fs http://namenode.uri -report "
         + "-node {DataNodeID | IP | Hostname}";
 
     HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp("hdfs diskbalancer -uri http://namenode.uri " +
+    helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +
         "-report [options]",
         header, DiskBalancer.getReportOptions(), footer);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b23ad1e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index 57e59f6..3accbc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -71,8 +71,10 @@ public class TestDiskBalancerCommand {
     }
   }
 
-  private void testReportSimple() throws Exception {
-    final String cmdLine = String.format("hdfs diskbalancer -uri %s -report",
+  /* test basic report */
+  @Test(timeout=60000)
+  public void testReportSimple() throws Exception {
+    final String cmdLine = String.format("hdfs diskbalancer -fs %s -report",
         clusterJson.toString());
     final List<String> outputs = runCommand(cmdLine);
 
@@ -98,9 +100,11 @@ public class TestDiskBalancerCommand {
 
   }
 
-  private void testReportLessThanTotal() throws Exception {
+  /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
+  @Test(timeout=60000)
+  public void testReportLessThanTotal() throws Exception {
     final String cmdLine = String.format(
-        "hdfs diskbalancer -uri %s -report -top 32", clusterJson.toString());
+        "hdfs diskbalancer -fs %s -report -top 32", clusterJson.toString());
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -120,9 +124,11 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
   }
 
-  private void testReportMoreThanTotal() throws Exception {
+  /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
+  @Test(timeout=60000)
+  public void testReportMoreThanTotal() throws Exception {
     final String cmdLine = String.format(
-        "hdfs diskbalancer -uri %s -report -top 128", clusterJson.toString());
+        "hdfs diskbalancer -fs %s -report -top 128", clusterJson.toString());
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -143,9 +149,11 @@ public class TestDiskBalancerCommand {
 
   }
 
-  private void testReportInvalidTopLimit() throws Exception {
+  /* test invalid top limit, e.g., -report -top xx */
+  @Test(timeout=60000)
+  public void testReportInvalidTopLimit() throws Exception {
     final String cmdLine = String.format(
-        "hdfs diskbalancer -uri %s -report -top xx", clusterJson.toString());
+        "hdfs diskbalancer -fs %s -report -top xx", clusterJson.toString());
     final List<String> outputs = runCommand(cmdLine);
 
     assertThat(
@@ -169,10 +177,12 @@ public class TestDiskBalancerCommand {
             containsString("9 volumes with node data density 1.97"))));
   }
 
-  private void testReportNode() throws Exception {
+  /* test -report -node DataNodeID */
+  @Test(timeout=60000)
+  public void testReportNode() throws Exception {
     final String cmdLine = String
         .format(
-            "hdfs diskbalancer -uri %s -report -node "
+            "hdfs diskbalancer -fs %s -report -node "
                 + "a87654a9-54c7-4693-8dd9-c9c7021dc340",
             clusterJson.toString());
     final List<String> outputs = runCommand(cmdLine);
@@ -192,9 +202,9 @@ public class TestDiskBalancerCommand {
     assertThat(
         outputs.get(3),
         is(allOf(containsString("DISK"),
-            containsString("/tmp/disk/xx3j3ph3zd"),
-            containsString("0.72 used: 289544224916/400000000000"),
-            containsString("0.28 free: 110455775084/400000000000"))));
+            containsString("/tmp/disk/KmHefYNURo"),
+            containsString("0.20 used: 39160240782/200000000000"),
+            containsString("0.80 free: 160839759218/200000000000"))));
     assertThat(
         outputs.get(4),
         is(allOf(containsString("DISK"),
@@ -204,16 +214,15 @@ public class TestDiskBalancerCommand {
     assertThat(
         outputs.get(5),
         is(allOf(containsString("DISK"),
-            containsString("DISK"),
-            containsString("/tmp/disk/KmHefYNURo"),
-            containsString("0.20 used: 39160240782/200000000000"),
-            containsString("0.80 free: 160839759218/200000000000"))));
+            containsString("/tmp/disk/xx3j3ph3zd"),
+            containsString("0.72 used: 289544224916/400000000000"),
+            containsString("0.28 free: 110455775084/400000000000"))));
     assertThat(
         outputs.get(6),
         is(allOf(containsString("RAM_DISK"),
-            containsString("/tmp/disk/MXRyYsCz3U"),
-            containsString("0.55 used: 438102096853/800000000000"),
-            containsString("0.45 free: 361897903147/800000000000"))));
+            containsString("/tmp/disk/BoBlQFxhfw"),
+            containsString("0.60 used: 477590453390/800000000000"),
+            containsString("0.40 free: 322409546610/800000000000"))));
     assertThat(
         outputs.get(7),
         is(allOf(containsString("RAM_DISK"),
@@ -223,9 +232,9 @@ public class TestDiskBalancerCommand {
     assertThat(
         outputs.get(8),
         is(allOf(containsString("RAM_DISK"),
-            containsString("/tmp/disk/BoBlQFxhfw"),
-            containsString("0.60 used: 477590453390/800000000000"),
-            containsString("0.40 free: 322409546610/800000000000"))));
+            containsString("/tmp/disk/MXRyYsCz3U"),
+            containsString("0.55 used: 438102096853/800000000000"),
+            containsString("0.45 free: 361897903147/800000000000"))));
     assertThat(
         outputs.get(9),
         is(allOf(containsString("SSD"),
@@ -247,25 +256,6 @@ public class TestDiskBalancerCommand {
   }
 
   @Test(timeout=60000)
-  public void testReportCommmand() throws Exception {
-
-    /* test basic report */
-    testReportSimple();
-
-    /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
-    testReportLessThanTotal();
-
-    /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
-    testReportMoreThanTotal();
-
-    /* test invalid top limit, e.g., -report -top xx */
-    testReportInvalidTopLimit();
-
-    /* test -report -node DataNodeID */
-    testReportNode();
-  }
-
-  @Test
   public void testReadClusterFromJson() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/49] hadoop git commit: HDFS-10501. DiskBalancer: Use the default datanode port if port is not provided. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10501. DiskBalancer: Use the default datanode port if port is not provided. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/121142cf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/121142cf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/121142cf

Branch: refs/heads/trunk
Commit: 121142cf952a4f9af1eb2488fe1714b6b8e685b6
Parents: 78a1032
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Jun 9 19:47:01 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/QueryCommand.java | 18 ++++++++++++++++--
 1 file changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/121142cf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index 36448b8..ea7dbcc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -22,10 +22,12 @@ package org.apache.hadoop.hdfs.server.diskbalancer.command;
 import com.google.common.base.Preconditions;
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
 import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerException;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
+import org.apache.hadoop.net.NetUtils;
 
 /**
  * Gets the current status of disk balancer command.
@@ -55,10 +57,22 @@ public class QueryCommand extends Command {
     verifyCommandOptions(DiskBalancer.QUERY, cmd);
     String nodeName = cmd.getOptionValue(DiskBalancer.QUERY);
     Preconditions.checkNotNull(nodeName);
-    ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeName);
+    nodeName = nodeName.trim();
+    String nodeAddress = nodeName;
+
+    // if the string is not name:port format use the default port.
+    if(!nodeName.matches("^.*:\\d$")) {
+      int defaultIPC = NetUtils.createSocketAddr(
+          getConf().getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
+              DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
+      nodeAddress = nodeName + ":" + defaultIPC;
+      LOG.debug("Using default data node port :  {}", nodeAddress);
+    }
+
+    ClientDatanodeProtocol dataNode = getDataNodeProxy(nodeAddress);
     try {
       DiskBalancerWorkStatus workStatus = dataNode.queryDiskBalancerPlan();
-      System.out.printf("Plan ID: %s Result: %s%n", workStatus.getPlanID(),
+      System.out.printf("Plan ID: %s %nResult: %s%n", workStatus.getPlanID(),
           workStatus.getResult().toString());
 
       if(cmd.hasOption(DiskBalancer.VERBOSE)) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/49] hadoop git commit: HDFS-10518. DiskBalancer: Pretty-print json in Query command. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10518. DiskBalancer: Pretty-print json in Query command. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7e2be5c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7e2be5c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7e2be5c4

Branch: refs/heads/trunk
Commit: 7e2be5c4a0b68b556ec6afcb0e14e0ab5ef1a9b2
Parents: af11ab3
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Jun 13 14:11:23 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../server/datanode/DiskBalancerWorkItem.java   | 42 ++++++++++++++++
 .../server/datanode/DiskBalancerWorkStatus.java |  2 +
 .../hdfs/server/datanode/DiskBalancer.java      | 53 ++++++++++++++++++--
 .../diskbalancer/command/QueryCommand.java      |  5 +-
 .../TestDiskBalancerWithMockMover.java          | 20 ++++++++
 5 files changed, 116 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2be5c4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
index fe908d8..f46a987 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
@@ -34,6 +34,8 @@ import java.io.IOException;
 @InterfaceStability.Unstable
 @JsonInclude(JsonInclude.Include.NON_DEFAULT)
 public class DiskBalancerWorkItem {
+  private  long startTime;
+  private long secondsElapsed;
   private long bytesToCopy;
   private long bytesCopied;
   private long errorCount;
@@ -242,4 +244,44 @@ public class DiskBalancerWorkItem {
   public void setBandwidth(long bandwidth) {
     this.bandwidth = bandwidth;
   }
+
+
+  /**
+   * Records the Start time of execution.
+   * @return startTime
+   */
+  public long getStartTime() {
+    return startTime;
+  }
+
+  /**
+   * Sets the Start time.
+   * @param startTime  - Time stamp for start of execution.
+   */
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  /**
+   * Gets the number of seconds elapsed from the start time.
+   *
+   * The reason why we have this is of time skews. The client's current time
+   * may not match with the server time stamp, hence the elapsed second
+   * cannot be computed from only startTime.
+   *
+   * @return seconds elapsed from start time.
+   */
+  public long getSecondsElapsed() {
+    return secondsElapsed;
+  }
+
+  /**
+   * Sets number of seconds elapsed.
+   *
+   * This is updated whenever we update the other counters.
+   * @param secondsElapsed  - seconds elapsed.
+   */
+  public void setSecondsElapsed(long secondsElapsed) {
+    this.secondsElapsed = secondsElapsed;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2be5c4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
index ca5e5f0..1f62f47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
@@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
 
 import static org.codehaus.jackson.map.type.TypeFactory.defaultInstance;
 
@@ -128,6 +129,7 @@ public class DiskBalancerWorkStatus {
    **/
   public String currentStateString() throws IOException {
     ObjectMapper mapper = new ObjectMapper();
+    mapper.enable(SerializationConfig.Feature.INDENT_OUTPUT);
     return mapper.writeValueAsString(currentState);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2be5c4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
index 7f768ea..5fde7c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
@@ -552,6 +552,21 @@ public class DiskBalancer {
      * @return FsDatasetSpi
      */
     FsDatasetSpi getDataset();
+
+    /**
+     * Returns time when this plan started executing.
+     *
+     * @return Start time in milliseconds.
+     */
+    long getStartTime();
+
+    /**
+     * Number of seconds elapsed.
+     *
+     * @return time in seconds
+     */
+    long getElapsedSeconds();
+
   }
 
   /**
@@ -622,6 +637,8 @@ public class DiskBalancer {
     private long maxDiskErrors;
     private int poolIndex;
     private AtomicBoolean shouldRun;
+    private long startTime;
+    private long secondsElapsed;
 
     /**
      * Constructs diskBalancerMover.
@@ -897,6 +914,9 @@ public class DiskBalancer {
       FsVolumeSpi source = pair.getSource();
       FsVolumeSpi dest = pair.getDest();
       List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>();
+      startTime = Time.now();
+      item.setStartTime(startTime);
+      secondsElapsed = 0;
 
       if (source.isTransientStorage() || dest.isTransientStorage()) {
         return;
@@ -937,7 +957,7 @@ public class DiskBalancer {
             if (block == null) {
               this.setExitFlag();
               LOG.error("No source blocks, exiting the copy. Source: {}, " +
-                      "dest:{}", source.getBasePath(), dest.getBasePath());
+                  "dest:{}", source.getBasePath(), dest.getBasePath());
               continue;
             }
 
@@ -973,9 +993,6 @@ public class DiskBalancer {
                 block.getNumBytes(), source.getBasePath(),
                 dest.getBasePath());
 
-            item.incCopiedSoFar(block.getNumBytes());
-            item.incBlocksCopied();
-
             // Check for the max throughput constraint.
             // We sleep here to keep the promise that we will not
             // copy more than Max MB/sec. we sleep enough time
@@ -984,6 +1001,14 @@ public class DiskBalancer {
             // we exit via Thread Interrupted exception.
             Thread.sleep(computeDelay(block.getNumBytes(), timeUsed, item));
 
+            // We delay updating the info to avoid confusing the user.
+            // This way we report the copy only if it is under the
+            // throughput threshold.
+            item.incCopiedSoFar(block.getNumBytes());
+            item.incBlocksCopied();
+            secondsElapsed = TimeUnit.MILLISECONDS.toSeconds(Time.now() -
+                startTime);
+            item.setSecondsElapsed(secondsElapsed);
           } catch (IOException ex) {
             LOG.error("Exception while trying to copy blocks. error: {}", ex);
             item.incErrorCount();
@@ -1009,5 +1034,25 @@ public class DiskBalancer {
     public FsDatasetSpi getDataset() {
       return dataset;
     }
+
+    /**
+     * Returns time when this plan started executing.
+     *
+     * @return Start time in milliseconds.
+     */
+    @Override
+    public long getStartTime() {
+      return startTime;
+    }
+
+    /**
+     * Number of seconds elapsed.
+     *
+     * @return time in seconds
+     */
+    @Override
+    public long getElapsedSeconds() {
+      return secondsElapsed;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2be5c4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
index 3a3b97f..6c759e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
@@ -46,6 +46,7 @@ public class QueryCommand extends Command {
         " plan running on a given datanode.");
     addValidCommandParameters(DiskBalancer.VERBOSE, "Prints verbose results.");
   }
+
   /**
    * Executes the Client Calls.
    *
@@ -62,7 +63,7 @@ public class QueryCommand extends Command {
     String nodeAddress = nodeName;
 
     // if the string is not name:port format use the default port.
-    if(!nodeName.matches("^.*:\\d$")) {
+    if (!nodeName.matches("^.*:\\d$")) {
       int defaultIPC = NetUtils.createSocketAddr(
           getConf().getTrimmed(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY,
               DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_DEFAULT)).getPort();
@@ -76,7 +77,7 @@ public class QueryCommand extends Command {
       System.out.printf("Plan ID: %s %nResult: %s%n", workStatus.getPlanID(),
           workStatus.getResult().toString());
 
-      if(cmd.hasOption(DiskBalancer.VERBOSE)) {
+      if (cmd.hasOption(DiskBalancer.VERBOSE)) {
         System.out.printf("%s", workStatus.currentStateString());
       }
     } catch (DiskBalancerException ex) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e2be5c4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
index 491fccb..b73b290 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
@@ -436,6 +436,26 @@ public class TestDiskBalancerWithMockMover {
       return this.dataset;
     }
 
+    /**
+     * Returns time when this plan started executing.
+     *
+     * @return Start time in milliseconds.
+     */
+    @Override
+    public long getStartTime() {
+      return 0;
+    }
+
+    /**
+     * Number of seconds elapsed.
+     *
+     * @return time in seconds
+     */
+    @Override
+    public long getElapsedSeconds() {
+      return 0;
+    }
+
     public int getRunCount() {
       synchronized (runCount) {
         LOG.info("Run count : " + runCount.intValue());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/49] hadoop git commit: HDFS-9526. Fix jackson annotation imports. Contributed by Xiaobing Zhou

Posted by ar...@apache.org.
HDFS-9526. Fix jackson annotation imports. Contributed by Xiaobing Zhou


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/599eca07
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/599eca07
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/599eca07

Branch: refs/heads/trunk
Commit: 599eca07b422d8dbf015ed01a5d5a13e4488468c
Parents: 30c6ebd
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Dec 9 12:58:55 2015 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt           | 2 ++
 .../hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java  | 5 +++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/599eca07/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index cad8e49..952813b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -5,3 +5,5 @@ HDFS-1312 Change Log
     HDFS-9420. Add DataModels for DiskBalancer. (Anu Engineer via szetszwo)
 
     HDFS-9449. DiskBalancer: Add connectors. (Anu Engineer via szetszwo)
+
+    HDFS-9526. Fix jackson annotation imports. (Xiaobing Zhou via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/599eca07/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
index a608248..24e891f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
@@ -18,8 +18,9 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.datamodel;
 
 import com.google.common.base.Preconditions;
-import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/49] hadoop git commit: HDFS-10478. DiskBalancer: resolve volume path names. Contributed by Anu Engineer.

Posted by ar...@apache.org.
HDFS-10478. DiskBalancer: resolve volume path names. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/64ccb232
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64ccb232
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64ccb232

Branch: refs/heads/trunk
Commit: 64ccb232ccf204991a28fa0211917fa935ad30c5
Parents: 47dcb0f
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Jun 7 10:29:35 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:21:08 2016 -0700

----------------------------------------------------------------------
 .../diskbalancer/command/PlanCommand.java       | 37 +++++++++++++++
 .../connectors/DBNameNodeConnector.java         |  1 -
 .../hdfs/server/diskbalancer/TestPlanner.java   | 47 ++++++++++++++++++++
 3 files changed, 84 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64ccb232/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
index d346c84..7cf0df1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
@@ -23,6 +23,10 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
 import org.apache.hadoop.hdfs.tools.DiskBalancer;
 import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
     .DiskBalancerDataNode;
@@ -32,7 +36,9 @@ import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Class that implements Plan Command.
@@ -111,7 +117,10 @@ public class PlanCommand extends Command {
           cmd.getOptionValue(DiskBalancer.PLAN));
     }
     this.thresholdPercentage = getThresholdPercentage(cmd);
+
+    LOG.debug("threshold Percentage is {}", this.thresholdPercentage);
     setNodesToProcess(node);
+    populatePathNames(node);
 
     List<NodePlan> plans = getCluster().computePlan(this.thresholdPercentage);
     setPlanParams(plans);
@@ -137,6 +146,32 @@ public class PlanCommand extends Command {
     }
   }
 
+
+  /**
+   * Reads the Physical path of the disks we are balancing. This is needed to
+   * make the disk balancer human friendly and not used in balancing.
+   *
+   * @param node - Disk Balancer Node.
+   */
+  private void populatePathNames(DiskBalancerDataNode node) throws IOException {
+    String dnAddress = node.getDataNodeIP() + ":" + node.getDataNodePort();
+    ClientDatanodeProtocol dnClient = getDataNodeProxy(dnAddress);
+    String volumeNameJson = dnClient.getDiskBalancerSetting(
+        DiskBalancerConstants.DISKBALANCER_VOLUME_NAME);
+    ObjectMapper mapper = new ObjectMapper();
+
+    @SuppressWarnings("unchecked")
+    Map<String, String> volumeMap =
+        mapper.readValue(volumeNameJson, HashMap.class);
+    for (DiskBalancerVolumeSet set : node.getVolumeSets().values()) {
+      for (DiskBalancerVolume vol : set.getVolumes()) {
+        if (volumeMap.containsKey(vol.getUuid())) {
+          vol.setPath(volumeMap.get(vol.getUuid()));
+        }
+      }
+    }
+  }
+
   /**
    * Gets extended help for this command.
    *
@@ -198,9 +233,11 @@ public class PlanCommand extends Command {
     for (NodePlan plan : plans) {
       for (Step step : plan.getVolumeSetPlans()) {
         if (this.bandwidth > 0) {
+          LOG.debug("Setting bandwidth to {}", this.bandwidth);
           step.setBandwidth(this.bandwidth);
         }
         if (this.maxError > 0) {
+          LOG.debug("Setting max error to {}", this.maxError);
           step.setMaxDiskErrors(this.maxError);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64ccb232/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
index acf1fa1..b044baf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
@@ -155,7 +155,6 @@ class DBNameNodeConnector implements ClusterConnector {
           .READ_ONLY_SHARED) || report.isFailed());
       volume.setStorageType(storage.getStorageType().name());
       volume.setIsTransient(storage.getStorageType().isTransient());
-      //volume.setPath(storage.getVolumePath());
       node.addVolume(volume);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64ccb232/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
index 77c2aa3..d01e8ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
@@ -456,4 +456,51 @@ public class TestPlanner {
     assertEquals(newPlan.getVolumeSetPlans().size(),
         copy.getVolumeSetPlans().size());
   }
+
+  @Test
+  public void testGreedyPlannerLargeDisksWithData() throws Exception {
+    NullConnector nullConnector = new NullConnector();
+    DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
+
+    DiskBalancerDataNode node =
+        new DiskBalancerDataNode(UUID.randomUUID().toString());
+
+    // All disks have same capacity of data
+    DiskBalancerVolume volume1 = createVolume("volume1", 1968, 88);
+    DiskBalancerVolume volume2 = createVolume("volume2", 1968, 88);
+    DiskBalancerVolume volume3 = createVolume("volume3", 1968, 111);
+    DiskBalancerVolume volume4 = createVolume("volume4", 1968, 111);
+    DiskBalancerVolume volume5 = createVolume("volume5", 1968, 30);
+    DiskBalancerVolume volume6 = createVolume("volume6", 1563, 30);
+    DiskBalancerVolume volume7 = createVolume("volume7", 1563, 30);
+    DiskBalancerVolume volume8 = createVolume("volume8", 1563, 30);
+    DiskBalancerVolume volume9 = createVolume("volume9", 1563, 210);
+
+
+
+
+    node.addVolume(volume1);
+    node.addVolume(volume2);
+    node.addVolume(volume3);
+
+    node.addVolume(volume4);
+    node.addVolume(volume5);
+    node.addVolume(volume6);
+
+    node.addVolume(volume7);
+    node.addVolume(volume8);
+    node.addVolume(volume9);
+
+
+    nullConnector.addNode(node);
+    cluster.readClusterInfo();
+    Assert.assertEquals(1, cluster.getNodes().size());
+
+    GreedyPlanner planner = new GreedyPlanner(1.0f, node);
+    NodePlan plan = new NodePlan(node.getDataNodeName(),
+        node.getDataNodePort());
+    planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
+
+    assertTrue(plan.getVolumeSetPlans().size() > 2);
+  }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/49] hadoop git commit: HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Contributed by Anu Engineer)

Posted by ar...@apache.org.
HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Contributed by Anu Engineer)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/66f0bb64
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/66f0bb64
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/66f0bb64

Branch: refs/heads/trunk
Commit: 66f0bb646d040a80bde75b5b3f7eacafd0034fe4
Parents: 96fe685
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Jan 29 11:05:53 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 23 18:18:48 2016 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/ClientDatanodeProtocol.java   | 10 ++++++
 .../ClientDatanodeProtocolTranslatorPB.java     | 17 +++++++++-
 .../src/main/proto/ClientDatanodeProtocol.proto | 19 +++++++++++
 .../hadoop-hdfs/HDFS-1312_CHANGES.txt           |  3 ++
 ...tDatanodeProtocolServerSideTranslatorPB.java | 24 ++++++++++++--
 .../hadoop/hdfs/server/datanode/DataNode.java   | 14 ++++++++
 .../diskbalancer/DiskBalancerConstants.java     | 35 ++++++++++++++++++++
 .../diskbalancer/TestDiskBalancerRPC.java       | 16 ++++++---
 8 files changed, 130 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
index 705c98f..dede89e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
@@ -183,4 +183,14 @@ public interface ClientDatanodeProtocol {
    * Gets the status of an executing diskbalancer Plan.
    */
   WorkStatus queryDiskBalancerPlan() throws IOException;
+
+  /**
+   * Gets a run-time configuration value from running diskbalancer instance.
+   * For example : Disk Balancer bandwidth of a running instance.
+   *
+   * @param key runtime configuration key
+   * @return value of the key as a string.
+   * @throws IOException - Throws if there is no such key
+   */
+  String getDiskBalancerSetting(String key) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index 59f2fd2..e7e0d94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Submit
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
 import org.apache.hadoop.ipc.ProtobufHelper;
@@ -365,8 +367,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
   /**
    * Cancels an executing disk balancer plan.
-   * @param planID - A SHA512 hash of the plan string.
    *
+   * @param planID - A SHA512 hash of the plan string.
    * @throws IOException on error
    */
   @Override
@@ -399,4 +401,17 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public String getDiskBalancerSetting(String key) throws IOException {
+    try {
+      DiskBalancerSettingRequestProto request =
+          DiskBalancerSettingRequestProto.newBuilder().setKey(key).build();
+      DiskBalancerSettingResponseProto response =
+          rpcProxy.getDiskBalancerSetting(NULL_CONTROLLER, request);
+      return response.hasValue() ? response.getValue() : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
index 28c8681..c61c700 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
@@ -197,7 +197,21 @@ message QueryPlanStatusResponseProto {
   optional string status = 2;
   optional string planID = 3;
   optional string currentStatus = 4;
+}
+
+/**
+  * This message sends a request to data node get a specific setting
+  * that is used by disk balancer.
+  */
+message DiskBalancerSettingRequestProto {
+  required string key = 1;
+}
 
+/**
+ * Response that describes the value of requested disk balancer setting.
+ */
+message DiskBalancerSettingResponseProto {
+  required string value = 1;
 }
 
 /**
@@ -275,4 +289,9 @@ service ClientDatanodeProtocolService {
    */
   rpc queryDiskBalancerPlan(QueryPlanStatusRequestProto)
       returns (QueryPlanStatusResponseProto);
+  /**
+   *  Gets run-time settings of Disk Balancer.
+   */
+  rpc getDiskBalancerSetting(DiskBalancerSettingRequestProto)
+      returns(DiskBalancerSettingResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
index c6a5554..d3bdedf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/HDFS-1312_CHANGES.txt
@@ -21,3 +21,6 @@ HDFS-1312 Change Log
 
     HDFS-9645. DiskBalancer: Add Query RPC. (Anu Engineer via Arpit Agarwal)
 
+    HDFS-9647. DiskBalancer: Add getRuntimeSettings. (Anu Engineer
+    via Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
index 03717c5..3246633 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
@@ -53,7 +53,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.Cancel
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.CancelPlanResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.QueryPlanStatusResponseProto;
-
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.DiskBalancerSettingResponseProto;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.hdfs.server.datanode.WorkStatus;
@@ -279,7 +280,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
     try {
       impl.cancelDiskBalancePlan(request.getPlanID());
       return CancelPlanResponseProto.newBuilder().build();
-    }catch (Exception e) {
+    } catch (Exception e) {
       throw new ServiceException(e);
     }
   }
@@ -289,7 +290,7 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
    */
   @Override
   public QueryPlanStatusResponseProto queryDiskBalancerPlan(
-      RpcController controller,  QueryPlanStatusRequestProto request)
+      RpcController controller, QueryPlanStatusRequestProto request)
       throws ServiceException {
     try {
       WorkStatus result = impl.queryDiskBalancerPlan();
@@ -304,4 +305,21 @@ public class ClientDatanodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  /**
+   * Returns a run-time setting from diskbalancer like Bandwidth.
+   */
+  @Override
+  public DiskBalancerSettingResponseProto getDiskBalancerSetting(
+      RpcController controller, DiskBalancerSettingRequestProto request)
+      throws ServiceException {
+    try {
+      String val = impl.getDiskBalancerSetting(request.getKey());
+      return DiskBalancerSettingResponseProto.newBuilder()
+          .setValue(val)
+          .build();
+    } catch (Exception e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index b5bd7b9..836dc81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3322,4 +3322,18 @@ public class DataNode extends ReconfigurableBase
     checkSuperuserPrivilege();
     throw new DiskbalancerException("Not Implemented", 0);
   }
+
+  /**
+   * Gets a run-time configuration value from running diskbalancer instance. For
+   * example : Disk Balancer bandwidth of a running instance.
+   *
+   * @param key - String that represents the run time key value.
+   * @return value of the key as a string.
+   * @throws IOException - Throws if there is no such key
+   */
+  @Override
+  public String getDiskBalancerSetting(String key) throws IOException {
+    checkSuperuserPrivilege();
+    throw new DiskbalancerException("Not Implemented", 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
new file mode 100644
index 0000000..553827e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
@@ -0,0 +1,35 @@
+/**
+ * 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.hdfs.server.diskbalancer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Constants used by Disk Balancer.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class DiskBalancerConstants {
+  public static final String DISKBALANCER_BANDWIDTH = "DiskBalancerBandwidth";
+  public static final String DISKBALANCER_VOLUME_NAME =
+      "DiskBalancerVolumeName";
+
+  // never constructed.
+  private DiskBalancerConstants() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/66f0bb64/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
index a127816..143b776 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
@@ -60,7 +60,7 @@ public class TestDiskBalancerRPC {
   }
 
   @Test
-  public void TestSubmitTestRpc() throws Exception {
+  public void testSubmitTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -91,7 +91,7 @@ public class TestDiskBalancerRPC {
   }
 
   @Test
-  public void TestCancelTestRpc() throws Exception {
+  public void testCancelTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -122,11 +122,10 @@ public class TestDiskBalancerRPC {
     }
     thrown.expect(DiskbalancerException.class);
     dataNode.cancelDiskBalancePlan(planHash);
-
   }
 
   @Test
-  public void TestQueryTestRpc() throws Exception {
+  public void testQueryTestRpc() throws Exception {
     final int dnIndex = 0;
     cluster.restartDataNode(dnIndex);
     cluster.waitActive();
@@ -162,4 +161,13 @@ public class TestDiskBalancerRPC {
     thrown.expect(DiskbalancerException.class);
     dataNode.queryDiskBalancerPlan();
   }
+
+  @Test
+  public void testgetDiskBalancerSetting() throws Exception {
+    final int dnIndex = 0;
+    DataNode dataNode = cluster.getDataNodes().get(dnIndex);
+    thrown.expect(DiskbalancerException.class);
+    dataNode.getDiskBalancerSetting(
+        DiskBalancerConstants.DISKBALANCER_BANDWIDTH);
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org