Explorar o código

Merge branch 'HDFS-1312' into trunk

Arpit Agarwal %!s(int64=9) %!d(string=hai) anos
pai
achega
6314843881
Modificáronse 62 ficheiros con 19824 adicións e 26 borrados
  1. 30 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  2. 91 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  3. 287 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java
  4. 277 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java
  5. 86 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto
  6. 1 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  7. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  8. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  9. 90 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java
  10. 107 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  11. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeMXBean.java
  12. 1063 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java
  13. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  14. 72 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  15. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  16. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.java
  17. 101 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
  18. 147 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java
  19. 437 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
  20. 115 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java
  21. 109 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java
  22. 271 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java
  23. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java
  24. 214 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
  25. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/package-info.java
  26. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ClusterConnector.java
  27. 55 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java
  28. 162 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/DBNameNodeConnector.java
  29. 78 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java
  30. 37 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java
  31. 391 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
  32. 270 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
  33. 361 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java
  34. 342 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java
  35. 31 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/package-info.java
  36. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/package-info.java
  37. 264 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
  38. 262 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java
  39. 190 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
  40. 28 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Planner.java
  41. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java
  42. 109 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java
  43. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/package-info.java
  44. 475 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java
  45. 40 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  46. 22 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  47. 120 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md
  48. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  49. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  50. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java
  51. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java
  52. 267 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java
  53. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.java
  54. 227 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.java
  55. 257 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java
  56. 323 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java
  57. 628 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java
  58. 513 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java
  59. 390 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
  60. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/NullConnector.java
  61. 380 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-3node-3disk.json
  62. 9484 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/diskBalancer/data-cluster-64node-3disk.json

+ 30 - 0
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.DiskBalancerWorkStatus;
 
 /** An client-datanode protocol for block recovery
  */
@@ -163,4 +164,33 @@ public interface ClientDatanodeProtocol {
    * @return balancer bandwidth
    */
   long getBalancerBandwidth() throws IOException;
+
+  /**
+   * Submit a disk balancer plan for execution.
+   */
+  void submitDiskBalancerPlan(String planID, long planVersion, String plan,
+                              boolean skipDateCheck) throws IOException;
+
+  /**
+   * Cancel an executing plan.
+   *
+   * @param planID - A SHA512 hash of the plan string.
+   */
+  void cancelDiskBalancePlan(String planID) throws IOException;
+
+
+  /**
+   * Gets the status of an executing diskbalancer Plan.
+   */
+  DiskBalancerWorkStatus 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;
 }

+ 91 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -52,7 +52,15 @@ 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.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.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;
@@ -326,4 +334,87 @@ 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 plan - Actual plan.
+   * @param skipDateCheck - Skips the date check.
+   * @throws IOException
+   */
+  @Override
+  public void submitDiskBalancerPlan(String planID, long planVersion,
+      String plan, boolean skipDateCheck) throws IOException {
+    try {
+      SubmitDiskBalancerPlanRequestProto request =
+          SubmitDiskBalancerPlanRequestProto.newBuilder()
+              .setPlanID(planID)
+              .setPlanVersion(planVersion)
+              .setPlan(plan)
+              .setIgnoreDateCheck(skipDateCheck)
+              .build();
+      rpcProxy.submitDiskBalancerPlan(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      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);
+    }
+  }
+
+  /**
+   * Gets the status of an executing diskbalancer Plan.
+   */
+  @Override
+  public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException {
+    try {
+      QueryPlanStatusRequestProto request =
+          QueryPlanStatusRequestProto.newBuilder().build();
+      QueryPlanStatusResponseProto response =
+          rpcProxy.queryDiskBalancerPlan(NULL_CONTROLLER, request);
+      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.hasCurrentStatus() ? response.getCurrentStatus() : null);
+    } catch (ServiceException e) {
+      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);
+    }
+  }
 }

+ 287 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkItem.java

@@ -0,0 +1,287 @@
+/*
+ * 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.apache.htrace.fasterxml.jackson.annotation.JsonInclude;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+
+/**
+ * Keeps track of how much work has finished.
+ */
+@InterfaceAudience.Private
+@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;
+  private String errMsg;
+  private long blocksCopied;
+
+  private long maxDiskErrors;
+  private long tolerancePercent;
+  private long bandwidth;
+
+  /**
+   * Empty constructor for Json serialization.
+   */
+  public DiskBalancerWorkItem() {
+
+  }
+
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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;
+  }
+
+
+  /**
+   * 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;
+  }
+}

+ 277 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancerWorkStatus.java

@@ -0,0 +1,277 @@
+/*
+ * 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 org.codehaus.jackson.map.SerializationConfig;
+
+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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DiskBalancerWorkStatus {
+
+  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 currentState - Current State
+   */
+  public DiskBalancerWorkStatus(Result result, String planID,
+                                List<DiskBalancerWorkEntry> currentState) {
+    this.result = result;
+    this.planID = planID;
+    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 Result getResult() {
+    return result;
+  }
+
+  /**
+   * Returns planID.
+   *
+   * @return String
+   */
+  public String getPlanID() {
+    return planID;
+  }
+
+  /**
+   * Gets current Status.
+   *
+   * @return - Json String
+   */
+  public List<DiskBalancerWorkEntry> getCurrentState() {
+    return currentState;
+  }
+
+  /**
+   * Return current state as a string.
+   *
+   * @throws IOException
+   **/
+  public String currentStateString() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.enable(SerializationConfig.Feature.INDENT_OUTPUT);
+    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.
+   *
+   * @param entry - DiskBalancerWorkEntry
+   */
+
+  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;
+
+    /**
+     * 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.
+     * @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;
+    }
+  }
+}

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientDatanodeProtocol.proto

@@ -149,6 +149,70 @@ message GetBalancerBandwidthResponseProto {
   required uint64 bandwidth = 1;
 }
 
+/**
+ * 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 bool ignoreDateCheck = 4; // Ignore date checks on this plan.
+}
+
+/**
+ * Response from the DataNode on Plan Submit request
+ */
+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 {
+}
+
+
+/**
+ * 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 planID = 2;
+  optional string currentStatus = 3;
+}
+
+/**
+  * 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;
+}
+
 /**
  * Protocol used from client to the Datanode.
  * See the request and response for details of rpc call.
@@ -207,4 +271,26 @@ service ClientDatanodeProtocolService {
    */
   rpc getBalancerBandwidth(GetBalancerBandwidthRequestProto)
       returns(GetBalancerBandwidthResponseProto);
+
+  /**
+   * Submit a disk balancer plan for execution
+   */
+  rpc submitDiskBalancerPlan(SubmitDiskBalancerPlanRequestProto)
+      returns (SubmitDiskBalancerPlanResponseProto);
+  /**
+   * Cancel an executing plan
+   */
+  rpc cancelDiskBalancerPlan(CancelPlanRequestProto)
+      returns (CancelPlanResponseProto);
+
+  /**
+   * Gets the status of an executing Plan
+   */
+  rpc queryDiskBalancerPlan(QueryPlanStatusRequestProto)
+      returns (QueryPlanStatusResponseProto);
+  /**
+   *  Gets run-time settings of Disk Balancer.
+   */
+  rpc getDiskBalancerSetting(DiskBalancerSettingRequestProto)
+      returns(DiskBalancerSettingResponseProto);
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -396,6 +396,7 @@ 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>
           </excludes>
         </configuration>
       </plugin>

+ 6 - 0
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}'"

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -930,6 +930,26 @@ 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;
+
+  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

+ 90 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolServerSideTranslatorPB.java

@@ -47,9 +47,17 @@ 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 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 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.DiskBalancerWorkStatus;
 
 /**
  * Implementation for protobuf service that forwards requests
@@ -232,4 +240,85 @@ 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() : 1,
+          request.getPlan(),
+          request.hasIgnoreDateCheck() ? request.getIgnoreDateCheck() : false);
+      SubmitDiskBalancerPlanResponseProto response =
+          SubmitDiskBalancerPlanResponseProto.newBuilder()
+              .build();
+      return response;
+    } catch(Exception e) {
+      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);
+    }
+  }
+
+  /**
+   * Gets the status of an executing Plan.
+   */
+  @Override
+  public QueryPlanStatusResponseProto queryDiskBalancerPlan(
+      RpcController controller, QueryPlanStatusRequestProto request)
+      throws ServiceException {
+    try {
+      DiskBalancerWorkStatus result = impl.queryDiskBalancerPlan();
+      return QueryPlanStatusResponseProto
+          .newBuilder()
+          .setResult(result.getResult().getIntResult())
+          .setPlanID(result.getPlanID())
+          .setCurrentStatus(result.currentStateString())
+          .build();
+    } catch (Exception e) {
+      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);
+    }
+  }
 }

+ 107 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -169,6 +169,8 @@ 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;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -385,6 +387,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").
@@ -1021,7 +1025,33 @@ public class DataNode extends ReconfigurableBase
       directoryScanner.shutdown();
     }
   }
-  
+
+  /**
+   * Initilizes {@link DiskBalancer}.
+   * @param  data - FSDataSet
+   * @param conf - Config
+   */
+  private 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  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;
@@ -1529,6 +1559,7 @@ public class DataNode extends ReconfigurableBase
     data.addBlockPool(nsInfo.getBlockPoolID(), conf);
     blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
     initDirectoryScanner(conf);
+    initDiskBalancer(data, conf);
   }
 
   List<BPOfferService> getAllBpOs() {
@@ -1866,6 +1897,7 @@ public class DataNode extends ReconfigurableBase
 
     // Terminate directory scanner and block scanner
     shutdownPeriodicScanners();
+    shutdownDiskBalancer();
 
     // Stop the web server
     if (httpServer != null) {
@@ -2926,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);
@@ -3286,4 +3328,68 @@ 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 plan - Actual plan
+   * @throws IOException
+   */
+  @Override
+  public void submitDiskBalancerPlan(String planID,
+      long planVersion, String plan, boolean skipDateCheck) throws IOException {
+
+    checkSuperuserPrivilege();
+    // TODO : Support force option
+    this.diskBalancer.submitPlan(planID, planVersion, plan, skipDateCheck);
+  }
+
+  /**
+   * Cancels a running plan.
+   * @param planID - Hash string that identifies a plan
+   */
+  @Override
+  public void cancelDiskBalancePlan(String planID) throws
+      IOException {
+    checkSuperuserPrivilege();
+    this.diskBalancer.cancelPlan(planID);
+  }
+
+  /**
+   * Returns the status of current or last executed work plan.
+   * @return DiskBalancerWorkStatus.
+   * @throws IOException
+   */
+  @Override
+  public DiskBalancerWorkStatus queryDiskBalancerPlan() throws IOException {
+    checkSuperuserPrivilege();
+    return this.diskBalancer.queryWorkStatus();
+  }
+
+  /**
+   * 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.
+   * @throws IOException - Throws if there is no such key
+   */
+  @Override
+  public String getDiskBalancerSetting(String key) throws IOException {
+    checkSuperuserPrivilege();
+    Preconditions.checkNotNull(key);
+    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);
+      throw new DiskBalancerException("Unknown key",
+          DiskBalancerException.Result.UNKNOWN_KEY);
+    }
+  }
 }

+ 8 - 0
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();
 }

+ 1063 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

@@ -0,0 +1,1063 @@
+/*
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+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;
+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 org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+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;
+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;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * 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 Logger LOG = LoggerFactory.getLogger(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;
+  private DiskBalancerWorkStatus.Result currentResult;
+  private long bandwidth;
+
+  /**
+   * 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.currentResult = Result.NO_PLAN;
+    this.blockMover = blockMover;
+    this.dataset = this.blockMover.getDataset();
+    this.dataNodeUUID = dataNodeUUID;
+    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);
+    this.bandwidth = conf.getInt(
+        DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT,
+        DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_THRUPUT_DEFAULT);
+  }
+
+  /**
+   * Shutdown  disk balancer services.
+   */
+  public void shutdown() {
+    lock.lock();
+    try {
+      this.isDiskBalancerEnabled = false;
+      this.currentResult = Result.NO_PLAN;
+      if ((this.future != null) && (!this.future.isDone())) {
+        this.currentResult = Result.PLAN_CANCELLED;
+        this.blockMover.setExitFlag();
+        shutdownExecutor();
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Shutdown the executor.
+   */
+  private void shutdownExecutor() {
+    final int secondsTowait = 10;
+    scheduler.shutdown();
+    try {
+      if (!scheduler.awaitTermination(secondsTowait, TimeUnit.SECONDS)) {
+        scheduler.shutdownNow();
+        if (!scheduler.awaitTermination(secondsTowait, 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 force       - Skip some validations and execute the plan file.
+   * @throws DiskBalancerException
+   */
+  public void submitPlan(String planID, long planVersion, String plan,
+                         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, force);
+      createWorkPlan(nodePlan);
+      this.planID = planID;
+      this.currentResult = Result.PLAN_UNDER_PROGRESS;
+      executePlan();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * 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();
+    }
+  }
+
+  /**
+   * 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) ||
+          this.planID.isEmpty()) {
+        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();
+        this.currentResult = Result.PLAN_CANCELLED;
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * 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 (DiskBalancerException ex) {
+      throw ex;
+    } catch (IOException e) {
+      throw new DiskBalancerException("Internal error, Unable to " +
+          "create JSON string.", e,
+          DiskBalancerException.Result.INTERNAL_ERROR);
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * 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.
+   *
+   * @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 force       - Skip verifying when the plan was generated.
+   * @return a NodePlan Object.
+   * @throws DiskBalancerException
+   */
+  private NodePlan verifyPlan(String planID, long planVersion, String plan,
+                              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);
+    }
+  }
+
+  /**
+   * 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 step   - Move Step
+   */
+  private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest,
+                              Step step) throws DiskBalancerException {
+
+    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);
+    }
+    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.
+    if (workMap.containsKey(pair)) {
+      bytesToMove += workMap.get(pair).getBytesToCopy();
+    }
+
+    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);
+  }
+
+  /**
+   * 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();
+
+    /**
+     * Returns time when this plan started executing.
+     *
+     * @return Start time in milliseconds.
+     */
+    long getStartTime();
+
+    /**
+     * Number of seconds elapsed.
+     *
+     * @return time in seconds
+     */
+    long getElapsedSeconds();
+
+  }
+
+  /**
+   * Holds references to actual volumes that we will be operating against.
+   */
+  public 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>
+   */
+  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;
+    private long startTime;
+    private long secondsElapsed;
+
+    /**
+     * Constructs diskBalancerMover.
+     *
+     * @param dataset Dataset
+     * @param conf    Configuration
+     */
+    public DiskBalancerMover(FsDatasetSpi dataset, Configuration conf) {
+      this.dataset = dataset;
+      shouldRun = new AtomicBoolean(false);
+
+      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;
+      }
+    }
+
+    /**
+     * Sets Diskmover 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();
+    }
+
+    /**
+     * 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 copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
+      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;
+      }
+
+      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());
+
+            // 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));
+
+            // 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();
+          } 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);
+      }
+    }
+
+    /**
+     * Returns a pointer to the current dataset we are operating against.
+     *
+     * @return FsDatasetSpi
+     */
+    @Override
+    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;
+    }
+  }
+}

+ 11 - 0
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;
 }

+ 72 - 23
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();
@@ -980,6 +958,77 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     return replicaInfo;
   }
 
+  /**
+   * 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.

+ 13 - 0
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;

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerConstants.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;
+
+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";
+
+  /** 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() {
+  }
+}

+ 101 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java

@@ -0,0 +1,101 @@
+/**
+ * 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 {
+  /**
+   * Results returned by the RPC layer of 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,
+    NO_SUCH_PLAN,
+    UNKNOWN_KEY
+  }
+
+  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;
+  }
+}

+ 147 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java

@@ -0,0 +1,147 @@
+/*
+ * 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.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 = NodePlan.parseJson(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.
+   */
+  @Override
+  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);
+  }
+}

+ 437 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java

@@ -0,0 +1,437 @@
+/*
+ * 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.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;
+import org.apache.hadoop.fs.FSDataInputStream;
+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.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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+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.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * 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 int topNodes;
+
+  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.
+    topNodes = 0;
+  }
+
+  /**
+   * Executes the Client Calls.
+   *
+   * @param cmd - CommandLine
+   * @throws Exception
+   */
+  public abstract void execute(CommandLine cmd) throws Exception;
+
+  /**
+   * Gets extended help for this command.
+   */
+  public abstract void printHelp();
+
+  /**
+   * 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);
+
+    setClusterURI(FileSystem.getDefaultUri(getConf()));
+    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() +
+                Path.SEPARATOR + format.format(now));
+      } else {
+        diskBalancerLogs = new Path(DEFAULT_LOG_DIR.toString() +
+            Path.SEPARATOR + format.format(now));
+      }
+    } else {
+      diskBalancerLogs = new Path(path);
+    }
+    if (fs.exists(diskBalancerLogs)) {
+      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());
+    }
+    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.getLongOpt())) {
+        String errMessage = String
+            .format("%nInvalid argument found for command %s : %s%n",
+                commandName, opt.getLongOpt());
+        StringBuilder validArguments = new StringBuilder();
+        validArguments.append(String.format("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.
+   * @throws IOException
+   */
+  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
+   */
+  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;
+  }
+
+  /**
+   * 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;
+  }
+}

+ 115 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java

@@ -0,0 +1,115 @@
+/*
+ * 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.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;
+
+
+/**
+ * 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.");
+  }
+
+  /**
+   * 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 = NodePlan.parseJson(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;
+    }
+  }
+
+
+
+  /**
+   * Gets extended help for this command.
+   */
+  @Override
+  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);
+  }
+}

+ 109 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/HelpCommand.java

@@ -0,0 +1,109 @@
+/*
+ * 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);
+    addValidCommandParameters(DiskBalancer.HELP, "Help Command");
+  }
+
+  /**
+   * 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(), "");
+  }
+
+
+}

+ 271 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/PlanCommand.java

@@ -0,0 +1,271 @@
+/*
+ * 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.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.DiskBalancerVolume;
+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;
+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.
+ * <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 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 " +
+        "we tolerate before diskbalancer starts working.");
+    addValidCommandParameters(DiskBalancer.MAXERROR, "Max errors to tolerate " +
+        "between 2 disks");
+    addValidCommandParameters(DiskBalancer.VERBOSE, "Run plan command in " +
+        "verbose mode.");
+    addValidCommandParameters(DiskBalancer.PLAN, "Plan Command");
+  }
+
+  /**
+   * 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.getOptionValue(DiskBalancer.PLAN) == null) {
+      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);
+
+    // -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.PLAN));
+    }
+    this.thresholdPercentage = getThresholdPercentage(cmd);
+
+    LOG.debug("threshold Percentage is {}", this.thresholdPercentage);
+    setNodesToProcess(node);
+    populatePathNames(node);
+
+    NodePlan plan = null;
+    List<NodePlan> plans = getCluster().computePlan(this.thresholdPercentage);
+    setPlanParams(plans);
+
+    if (plans.size() > 0) {
+      plan = plans.get(0);
+    }
+
+
+    try (FSDataOutputStream beforeStream = create(String.format(
+        DiskBalancer.BEFORE_TEMPLATE,
+        cmd.getOptionValue(DiskBalancer.PLAN)))) {
+      beforeStream.write(getCluster().toJson()
+          .getBytes(StandardCharsets.UTF_8));
+    }
+
+    if (plan != null && plan.getVolumeSetPlans().size() > 0) {
+      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) && 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.
+   *
+   * @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.
+   */
+  @Override
+  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 -plan " +
+        "<hostname> [options]", header, DiskBalancer.getPlanOptions(), footer);
+  }
+
+  /**
+   * 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(
+        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 %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)));
+      }
+    }
+
+    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) {
+          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);
+        }
+      }
+    }
+  }
+}

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/QueryCommand.java

@@ -0,0 +1,104 @@
+/*
+ * 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.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.
+ */
+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);
+    nodeName = nodeName.trim();
+    String nodeAddress = nodeName;
+
+    // if the string is not name:port format use the default port.
+    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();
+      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 %nResult: %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.
+   */
+  @Override
+  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);
+  }
+}

+ 214 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java

@@ -0,0 +1,214 @@
+/**
+ * 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.List;
+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;
+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;
+import com.google.common.collect.Lists;
+
+/**
+ * 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()));
+
+        List<String> volumeList = Lists.newArrayList();
+        for (DiskBalancerVolumeSet vset : dbdn.getVolumeSets().values()) {
+          for (DiskBalancerVolume vol : vset.getVolumes()) {
+            volumeList.add(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));
+          }
+        }
+
+        Collections.sort(volumeList);
+        result.appendln(
+            StringUtils.join(volumeList.toArray(), System.lineSeparator()));
+      }
+    }
+  }
+
+  /**
+   * Prints the help message.
+   */
+  @Override
+  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 -report\n"
+        + "hdfs diskbalancer -report -top 5\n"
+        + "hdfs diskbalancer -report "
+        + "-node {DataNodeID | IP | Hostname}";
+
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.printHelp("hdfs diskbalancer -fs http://namenode.uri " +
+        "-report [options]",
+        header, DiskBalancer.getReportOptions(), footer);
+  }
+}

+ 22 - 0
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;

+ 44 - 0
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();
+}

+ 55 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/ConnectorFactory.java

@@ -0,0 +1,55 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+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 {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(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.debug("Cluster URI : {}" , clusterURI);
+    LOG.debug("scheme : {}" , clusterURI.getScheme());
+    if (clusterURI.getScheme().startsWith("file")) {
+      LOG.debug("Creating a JsonNodeConnector");
+      return new JsonNodeConnector(clusterURI.toURL());
+    } else {
+      LOG.debug("Creating NameNode connector");
+      return new DBNameNodeConnector(clusterURI, conf);
+    }
+  }
+
+  private ConnectorFactory() {
+    // never constructed
+  }
+}

+ 162 - 0
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.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;
+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 {
+  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;
+
+  /**
+   * 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());
+      node.addVolume(volume);
+    }
+
+  }
+}

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/JsonNodeConnector.java

@@ -0,0 +1,78 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+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 {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(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();
+  }
+}

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/connectors/package-info.java

@@ -0,0 +1,37 @@
+/**
+ * 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 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
+ * ask the diskBalancer Cluster to read data from this source.
+ */

+ 391 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java

@@ -0,0 +1,391 @@
+/**
+ * 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.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;
+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.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;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * 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 {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(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;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode> ipList;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode> hostNames;
+  @JsonIgnore
+  private final Map<String, DiskBalancerDataNode>  hostUUID;
+
+  private float threshold;
+
+  /**
+   * Empty Constructor needed by Jackson.
+   */
+  public DiskBalancerCluster() {
+    nodes = new LinkedList<>();
+    exclusionList = new TreeSet<>();
+    inclusionList = new TreeSet<>();
+    ipList = new HashMap<>();
+    hostNames = new HashMap<>();
+    hostUUID = new HashMap<>();
+  }
+
+  /**
+   * Constructs a DiskBalancerCluster.
+   *
+   * @param connector - ClusterConnector
+   * @throws IOException
+   */
+  public DiskBalancerCluster(ClusterConnector connector) throws IOException {
+    this();
+    Preconditions.checkNotNull(connector);
+    clusterConnector = connector;
+  }
+
+  /**
+   * 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.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);
+      }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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(double 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;
+    }
+  }
+
+  /**
+   * Returns a node by UUID.
+   * @param uuid - Node's UUID
+   * @return DiskBalancerDataNode.
+   */
+  public DiskBalancerDataNode getNodeByUUID(String uuid) {
+    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);
+  }
+}

+ 270 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java

@@ -0,0 +1,270 @@
+/**
+ * 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 double 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 (Double.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+        < 0) {
+      return -1;
+    }
+
+    if (Double.compare(this.nodeDataDensity - that.getNodeDataDensity(), 0)
+        == 0) {
+      return 0;
+    }
+
+    if (Double.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 double 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() {
+    double 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(double 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());
+      vSet.setStorageType(volumeSetKey);
+      volumeSets.put(volumeSetKey, vSet);
+    }
+
+    vSet.addVolume(volume);
+    computeNodeDensity();
+  }
+
+  /**
+   * Returns how many volumes are in the DataNode.
+   *
+   * @return int
+   */
+  public int getVolumeCount() {
+    return volumeCount;
+  }
+
+
+}

+ 361 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolume.java

@@ -0,0 +1,361 @@
+/**
+ * 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.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.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 double 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 double getVolumeDataDensity() {
+    return volumeDataDensity;
+  }
+
+  /**
+   * Sets this volume's data density.
+   *
+   * @param volDataDensity - density
+   */
+  public void setVolumeDataDensity(double 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;
+  }
+
+  /**
+   * 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
+   */
+  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;
+  }
+
+}

+ 342 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerVolumeSet.java

@@ -0,0 +1,342 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.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 {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(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 double 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 = truncateDecimals(totalUsed /
+          (double) totalCapacity);
+    }
+
+    for (DiskBalancerVolume volume : volumes) {
+      if (!volume.isFailed() && !volume.isSkip()) {
+        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.
+    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.error(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(double thresholdPercentage) {
+    double threshold = thresholdPercentage / 100.0d;
+
+    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();
+      Double absDensity =
+          truncateDecimals(Math.abs(vol.getVolumeDataDensity()));
+
+      if ((absDensity > 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 double 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 Double.compare(second.getVolumeDataDensity(),
+          first.getVolumeDataDensity());
+    }
+  }
+}

+ 31 - 0
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.
+ */

+ 36 - 0
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.
+ *
+ */

+ 264 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java

@@ -0,0 +1,264 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+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;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(GreedyPlanner.class);
+  private final double 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(double 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.getDataNodeName(), node.getDataNodePort());
+    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:%d took %d ms ",
+            node.getDataNodeName(), node.getDataNodePort(),
+            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 {
+    Preconditions.checkNotNull(vSet);
+    Preconditions.checkNotNull(plan);
+    Preconditions.checkNotNull(node);
+    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: {} highVolume: {}",
+            lowVolume.getPath(), 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("{} Skipping disk from computation. Maximum data size " +
+          "achieved.", lowVolume.getPath());
+      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(" {} Skipping disk from computation. Minimum data size " +
+          "achieved.", highVolume.getPath());
+      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);
+  }
+}

+ 262 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/MoveStep.java

@@ -0,0 +1,262 @@
+/**
+ * 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;
+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.
+ */
+public class MoveStep implements Step {
+  private DiskBalancerVolume sourceVolume;
+  private DiskBalancerVolume destinationVolume;
+  private double idealStorage;
+  private long bytesToMove;
+  private String volumeSetID;
+
+  private long maxDiskErrors;
+  private long tolerancePercent;
+  private long bandwidth;
+
+  /**
+   * 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, double 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 double 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(double 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);
+  }
+
+  /**
+   * Gets Maximum numbers of errors to be tolerated before this
+   * move operation is aborted.
+   * @return  long.
+   */
+  @Override
+  public long getMaxDiskErrors() {
+    return maxDiskErrors;
+  }
+
+  /**
+   * Sets the maximum numbers of Errors to be tolerated before this
+   * step is aborted.
+   * @param maxDiskErrors - long
+   */
+  @Override
+  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.
+   */
+  @Override
+  public long getTolerancePercent() {
+    return tolerancePercent;
+  }
+
+  /**
+   * Sets the tolerance percentage.
+   * @param tolerancePercent  - long
+   */
+  @Override
+  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.
+   */
+  @Override
+  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.
+   */
+  @Override
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
+}

+ 190 - 0
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;
+  }
+}

+ 28 - 0
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;
+}

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/PlannerFactory.java

@@ -0,0 +1,60 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
+    .DiskBalancerDataNode;
+
+/**
+ * Returns a planner based on the user defined tags.
+ */
+public final class PlannerFactory {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(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, double 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
+  }
+}

+ 109 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/Step.java

@@ -0,0 +1,109 @@
+/**
+ * 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
+   */
+  double 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);
+
+  /**
+   * 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();
+
+  /**
+   * 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);
+
+
+}

+ 46 - 0
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;
+ */

+ 475 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DiskBalancer.java

@@ -0,0 +1,475 @@
+/**
+ * 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.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;
+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;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintStream;
+
+/**
+ * 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 {
+  /**
+   * 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";
+  /**
+   * Executes a given plan file on the target datanode.
+   */
+  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";
+  /**
+   * Runs the command in verbose mode.
+   */
+  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";
+  /**
+   * Cancels a running plan.
+   */
+  public static final String CANCEL = "cancel";
+  /**
+   * 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);
+
+  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.
+   *
+   * @param conf
+   */
+  public DiskBalancer(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * 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());
+      res = 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 {
+    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, out);
+  }
+
+  /**
+   * returns the Command Line Options.
+   *
+   * @return Options
+   */
+  private Options getOpts() {
+    Options opts = new Options();
+    addPlanCommands(opts);
+    addHelpCommands(opts);
+    addExecuteCommands(opts);
+    addQueryCommands(opts);
+    addCancelCommands(opts);
+    addReportCommands(opts);
+    return opts;
+  }
+
+  /**
+   * 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.
+   *
+   * @return Options.
+   */
+  private void addPlanCommands(Options opt) {
+
+    Option plan = OptionBuilder.withLongOpt(PLAN)
+        .withDescription("creates a plan for datanode.")
+        .hasArg()
+        .create();
+    getPlanOptions().addOption(plan);
+    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 = 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 = 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 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 = 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()
+        .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 = 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 = 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 = OptionBuilder.withLongOpt(CANCEL)
+        .hasArg()
+        .withDescription("Cancels a running plan using a plan file.")
+        .create();
+    getCancelOptions().addOption(cancel);
+    opt.addOption(cancel);
+
+    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 = OptionBuilder.withLongOpt(REPORT)
+        .withDescription("List nodes that will benefit from running " +
+            "DiskBalancer.")
+        .create();
+    getReportOptions().addOption(report);
+    opt.addOption(report);
+
+    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 =  OptionBuilder.withLongOpt(NODE)
+        .hasArg()
+        .withDescription("Datanode address, " +
+            "it can be DataNodeID, IP or hostname.")
+        .create();
+    getReportOptions().addOption(node);
+    opt.addOption(node);
+  }
+
+  /**
+   * 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
+   * @param opts options of command line
+   * @param out  the output stream used for printing
+   */
+  private int dispatch(CommandLine cmd, Options opts, final PrintStream out)
+      throws Exception {
+    Command currentCommand = null;
+    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.CANCEL)) {
+      currentCommand = new CancelCommand(getConf());
+    }
+
+    if (cmd.hasOption(DiskBalancer.REPORT)) {
+      currentCommand = new ReportCommand(getConf(), out);
+    }
+
+    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);
+    return 0;
+  }
+}

+ 40 - 0
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>

+ 22 - 0
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> -fs <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:

+ 120 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSDiskbalancer.md

@@ -0,0 +1,120 @@
+<!---
+  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 -plan node1.mycluster.com`
+
+ 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.
+
+| 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’s 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.
+
+### Report
+Report command provides detailed report about a node.
+
+`hdfs diskbalancer -fs http://namenode.uri -report -node {DataNodeID | IP | Hostname}`
+
+
+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.|

+ 2 - 1
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);

+ 7 - 0
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;
+  }
+
 }
 

+ 8 - 0
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;
+  }
+
 }

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerResultVerifier.java

@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+/**
+ * Helps in verifying test results.
+ */
+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);
+
+  }
+}

+ 267 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerTestUtil.java

@@ -0,0 +1,267 @@
+/**
+ * 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.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;
+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;
+
+/**
+ * Helper class to create various cluster configrations at run time.
+ */
+public class DiskBalancerTestUtil {
+  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};
+  private 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;
+  }
+
+  /**
+   * 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);
+        }
+      }
+    }
+  }
+}

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestConnectors.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
+ * <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;
+
+/**
+ * Test Class that tests connectors.
+ */
+public class TestConnectors {
+  private MiniDFSCluster cluster;
+  private final int numDatanodes = 3;
+  private final int volumeCount = 2; // default volumes in MiniDFSCluster.
+  private 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());
+  }
+}

+ 227 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDataModels.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 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;
+
+/**
+ * Tests DiskBalancer Data models.
+ */
+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());
+  }
+}

+ 257 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java

@@ -0,0 +1,257 @@
+/**
+ * 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;
+
+/**
+ * Test Disk Balancer.
+ */
+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.
+   * <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.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDiskBalancerEndToEnd() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    final int defaultBlockSize = 100;
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    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";
+    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[]{defaultBlockSize * 2 * fileLen,
+        defaultBlockSize * 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, defaultBlockSize * 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);
+      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
+        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) *
+              defaultBlockSize + 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();
+    }
+  }
+}

+ 323 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java

@@ -0,0 +1,323 @@
+/**
+ * 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.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.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.NodePlan;
+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 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;
+
+/**
+ * Test DiskBalancer RPC.
+ */
+public class TestDiskBalancerRPC {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+
+  @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();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  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, plan.toJson(),
+        false);
+  }
+
+  @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);
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_HASH));
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
+  }
+
+  @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);
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN_VERSION));
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
+  }
+
+  @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);
+    thrown.expect(new DiskBalancerResultVerifier(Result.INVALID_PLAN));
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, "",
+        false);
+  }
+
+  @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, plan.toJson(),
+        false);
+    dataNode.cancelDiskBalancePlan(planHash);
+  }
+
+  @Test
+  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);
+    thrown.expect(new DiskBalancerResultVerifier(Result.NO_SUCH_PLAN));
+    dataNode.cancelDiskBalancePlan(planHash);
+  }
+
+  @Test
+  public void testCancelEmptyPlan() throws Exception {
+    RpcTestHelper rpcTestHelper = new RpcTestHelper().invoke();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = "";
+    NodePlan plan = rpcTestHelper.getPlan();
+    thrown.expect(DiskBalancerException.class);
+    thrown.expect(new DiskBalancerResultVerifier(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 DiskBalancerResultVerifier(Result.UNKNOWN_KEY));
+    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, 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();
+    DataNode dataNode = rpcTestHelper.getDataNode();
+    String planHash = rpcTestHelper.getPlanHash();
+    int planVersion = rpcTestHelper.getPlanVersion();
+    NodePlan plan = rpcTestHelper.getPlan();
+
+    dataNode.submitDiskBalancerPlan(planHash, planVersion, plan.toJson(),
+        false);
+    DiskBalancerWorkStatus status = dataNode.queryDiskBalancerPlan();
+    Assert.assertTrue(status.getResult() == PLAN_UNDER_PROGRESS ||
+        status.getResult() == PLAN_DONE);
+  }
+
+  @Test
+  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 testMoveBlockAcrossVolume() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    final int defaultBlockSize = 100;
+    conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
+    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;
+    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;
+    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;
+    }
+  }
+}

+ 628 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java

@@ -0,0 +1,628 @@
+/*
+ * 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 com.google.common.base.Supplier;
+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.test.GenericTestUtils;
+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.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();
+
+  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.
+   */
+  @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, 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();
+    final DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    executeSubmitPlan(plan, balancer);
+    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);
+  }
+
+  @Test
+  public void testSubmitWithOlderPlan() throws Exception {
+    final long millisecondInAnHour = 1000 * 60 * 60L;
+    MockMoverHelper mockMoverHelper = new MockMoverHelper().invoke();
+    NodePlan plan = mockMoverHelper.getPlan();
+    DiskBalancer balancer = mockMoverHelper.getBalancer();
+
+    plan.setTimeStamp(Time.now() - (32 * millisecondInAnHour));
+    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, 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, 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());
+
+  }
+
+
+  /**
+   * 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();
+    final int numStoragesPerDn = 2;
+    cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(3)
+        .storagesPerDatanode(numStoragesPerDn)
+        .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;
+    }
+
+    /**
+     * 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());
+        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);
+      }
+    }
+
+  }
+}
+

+ 513 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestPlanner.java

@@ -0,0 +1,513 @@
+/**
+ * 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.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;
+
+/**
+ * Test Planner.
+ */
+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(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
+    // 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(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
+    // 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());
+    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"));
+  }
+
+  @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());
+    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"));
+
+  }
+
+  @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.5 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());
+  }
+
+  @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);
+  }
+}

+ 390 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -0,0 +1,390 @@
+/**
+ * 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.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;
+import org.junit.After;
+import org.junit.Before;
+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 {
+    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();
+    }
+  }
+
+  /* test basic report */
+  @Test(timeout = 60000)
+  public void testReportSimple() throws Exception {
+    final String cmdLine = "hdfs diskbalancer -report";
+    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"))));
+
+  }
+
+  /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
+  @Test(timeout = 60000)
+  public void testReportLessThanTotal() throws Exception {
+    final String cmdLine = "hdfs diskbalancer -report -top 32";
+    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"))));
+  }
+
+  /* test more than 64 DataNode(s) as total, e.g., -report -top 128 */
+  @Test(timeout = 60000)
+  public void testReportMoreThanTotal() throws Exception {
+    final String cmdLine = "hdfs diskbalancer -report -top 128";
+    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"))));
+
+  }
+
+  /* test invalid top limit, e.g., -report -top xx */
+  @Test(timeout = 60000)
+  public void testReportInvalidTopLimit() throws Exception {
+    final String cmdLine = "hdfs diskbalancer -report -top xx";
+    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"))));
+  }
+
+  @Test(timeout = 60000)
+  public void testReportNode() throws Exception {
+    final String cmdLine =
+        "hdfs diskbalancer -report -node " +
+            "a87654a9-54c7-4693-8dd9-c9c7021dc340";
+    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/KmHefYNURo"),
+            containsString("0.20 used: 39160240782/200000000000"),
+            containsString("0.80 free: 160839759218/200000000000"))));
+    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("/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/BoBlQFxhfw"),
+            containsString("0.60 used: 477590453390/800000000000"),
+            containsString("0.40 free: 322409546610/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/MXRyYsCz3U"),
+            containsString("0.55 used: 438102096853/800000000000"),
+            containsString("0.45 free: 361897903147/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 testReadClusterFromJson() throws Exception {
+    ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
+        conf);
+    DiskBalancerCluster diskBalancerCluster = new DiskBalancerCluster(
+        jsonConnector);
+    diskBalancerCluster.readClusterInfo();
+    assertEquals(64, diskBalancerCluster.getNodes().size());
+  }
+
+  /* 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, cluster);
+  }
+
+  /* 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> 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);
+
+    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;
+  }
+
+  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
+   */
+  @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();
+    }
+  }
+}

+ 59 - 0
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);
+  }
+}

+ 380 - 0
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
+}

+ 9484 - 0
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-ea1fa7401920",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"1ef67ace-5212-4d58-9733-ddc4b4fbee22",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.9688416,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Mxfcfmb24Y",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":733099315216,
+                     "reserved":45400441265,
+                     "uuid":"86032355-e4e9-4252-b9d6-8995d07e18f2",
+                     "failed":false,
+                     "volumeDataDensity":-0.14702404,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/KmHefYNURo",
+                     "capacity":200000000000,
+                     "storageType":"DISK",
+                     "used":39160240782,
+                     "reserved":16471085439,
+                     "uuid":"7afe62c0-5a1d-493c-ba8b-081f932ff38c",
+                     "failed":false,
+                     "volumeDataDensity":0.6111101,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xx3j3ph3zd",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":289544224916,
+                     "reserved":50287738594,
+                     "uuid":"5e559276-f9f2-46de-885d-a67ed347b84c",
+                     "failed":false,
+                     "volumeDataDensity":-0.003466189,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"4b8ac6ab-7ef1-4a94-949b-536d33ee0125",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/MXRyYsCz3U",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":438102096853,
+                     "reserved":91297963368,
+                     "uuid":"2313aea8-5bbb-48ce-8241-60ea8595b686",
+                     "failed":false,
+                     "volumeDataDensity":-0.036521375,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/BoBlQFxhfw",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":477590453390,
+                     "reserved":34503448616,
+                     "uuid":"cf9b25b7-df73-4c85-9984-ef82c2122100",
+                     "failed":false,
+                     "volumeDataDensity":-0.04224235,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/DtmAygEU6f",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":134602910470,
+                     "reserved":68493586770,
+                     "uuid":"9e1c21da-8912-4f9f-bfe0-4940757d7007",
+                     "failed":false,
+                     "volumeDataDensity":0.17561987,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"7b0f867c-9f71-468c-8375-c98f1185f9f1",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/BGe09Y77dI",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":890446265501,
+                     "reserved":94727741720,
+                     "uuid":"e43f3e31-76a1-4457-9542-a1c286f92bd5",
+                     "failed":false,
+                     "volumeDataDensity":-0.51954484,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/uLOYmVZfWV",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1509592146007,
+                     "reserved":36004846310,
+                     "uuid":"670a51d5-e1fd-4c3f-ad46-c64829b4db61",
+                     "failed":false,
+                     "volumeDataDensity":-0.30455548,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/JX3H8iHggM",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":2782614512957,
+                     "reserved":701630231670,
+                     "uuid":"200f5678-062d-4943-84ec-809673d4cfb7",
+                     "failed":false,
+                     "volumeDataDensity":0.12875721,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"cf6fe4ce-74aa-45eb-9f67-4bfd4e3a859b",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"a87654a9-54c7-4693-8dd9-c9c7021dc340",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.5206845,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/4EI93AjBhe",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":376654298796,
+                     "reserved":296889392334,
+                     "uuid":"3fe392c7-54b5-40e4-88dd-1693a29c7b5e",
+                     "failed":false,
+                     "volumeDataDensity":0.35342154,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/2rrwN8RSZq",
+                     "capacity":9000000000000,
+                     "storageType":"DISK",
+                     "used":4796534876850,
+                     "reserved":1189674800187,
+                     "uuid":"8da8eb33-4b60-4ecd-b8fe-8da7132f618f",
+                     "failed":false,
+                     "volumeDataDensity":-0.039549172,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/OiqIDsclEl",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":747876253720,
+                     "reserved":8371624007,
+                     "uuid":"8846faa3-b340-437a-bd8a-dbfd8af190a2",
+                     "failed":false,
+                     "volumeDataDensity":-0.3701533,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"1e3a03ac-dddf-42ee-8178-6f91c4951666",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/IRAcb8PLzh",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":5700816981867,
+                     "reserved":613785952575,
+                     "uuid":"9a0bd1a1-1f35-4e19-a3c0-769ea5e4dc4c",
+                     "failed":false,
+                     "volumeDataDensity":-0.06694269,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/olVpur3DMa",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":100867725107,
+                     "reserved":27243882425,
+                     "uuid":"46f480ea-fb57-4758-87bb-2390161497e6",
+                     "failed":false,
+                     "volumeDataDensity":0.3350666,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mX4bOI9EgJ",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":118268031388,
+                     "reserved":60394664049,
+                     "uuid":"037c4658-2287-41bb-a727-829c1861b43d",
+                     "failed":false,
+                     "volumeDataDensity":0.54496884,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"d1aa4667-6263-473b-8cb2-6ee3bbed0d46",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/XqJY2kHyGj",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":12411373323,
+                     "reserved":4633315517,
+                     "uuid":"6e034e64-30db-4d04-9cd8-5e8d10f1d02b",
+                     "failed":false,
+                     "volumeDataDensity":0.20286702,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/PJIqWjwCXF",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":31867113288,
+                     "reserved":13978714814,
+                     "uuid":"96179838-be24-406a-8a1f-c1aba535a71b",
+                     "failed":false,
+                     "volumeDataDensity":0.15498045,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DoTVqjLiWF",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":133365133752,
+                     "reserved":14546678850,
+                     "uuid":"beec50c4-086e-4b93-b9ed-c5a4583e7c40",
+                     "failed":false,
+                     "volumeDataDensity":-0.4527349,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"2fb46e18-1987-43e3-aa66-75b9ff4942e7",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"24aa7cf8-e784-4fd0-ba7c-0b3799658554",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.431992,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/zrUge6m9Bp",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":835837400,
+                     "reserved":2755644829,
+                     "uuid":"bd41e3c4-5eae-48aa-a64a-73923d16be33",
+                     "failed":false,
+                     "volumeDataDensity":0.54487175,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/eWMmWB1TSD",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":5196747528815,
+                     "reserved":14241544702,
+                     "uuid":"0f700a35-18b2-4f16-9eff-1f8582f3059b",
+                     "failed":false,
+                     "volumeDataDensity":-0.3147183,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/IDegp6Bgpt",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":143365028248,
+                     "reserved":433016299478,
+                     "uuid":"73b37148-94e8-4fb5-973f-3910cf1e204e",
+                     "failed":false,
+                     "volumeDataDensity":0.5132747,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"6f106dad-6cae-4e2b-9579-3da39353882e",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Cl4ga0zUPq",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":862076154965,
+                     "reserved":109484000703,
+                     "uuid":"4ccf9f68-fa6f-4137-818f-43fa968db7ee",
+                     "failed":false,
+                     "volumeDataDensity":-0.08748478,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/YkxbufsPY8",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":175989011670,
+                     "reserved":44939297176,
+                     "uuid":"e126217a-47bf-4bd4-8258-400261558022",
+                     "failed":false,
+                     "volumeDataDensity":0.19059008,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/rmcuB1rb5H",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":53151659883,
+                     "reserved":6372687687,
+                     "uuid":"ef5fc0dd-3a52-47ff-ae58-d21e0d80115e",
+                     "failed":false,
+                     "volumeDataDensity":0.31288487,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"0b44f7c9-0744-4607-96c9-e42dab2f323e",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/ZkFH0Bu5xS",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":393147260722,
+                     "reserved":31489714265,
+                     "uuid":"375ab090-b2ad-4528-a378-d9432f2c5f04",
+                     "failed":false,
+                     "volumeDataDensity":-0.23374924,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tDD2K6ERSf",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":156118669932,
+                     "reserved":9567804101,
+                     "uuid":"ecb5087b-d643-4ddf-a702-af6a98a3ed8c",
+                     "failed":false,
+                     "volumeDataDensity":0.13946137,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/VKXPeNRzW7",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":246427098905,
+                     "reserved":120824852554,
+                     "uuid":"76f9935b-1851-4778-9736-379c7c677fd1",
+                     "failed":false,
+                     "volumeDataDensity":0.0949572,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"3320cced-5a44-4812-8790-0f8bfefa15cc",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"f9f52788-4fb9-4e08-bf15-804c8af2d62c",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.9747126,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/kqWDq1ccc0",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":175291855838,
+                     "reserved":147548295848,
+                     "uuid":"ae14e380-7c2e-48bf-a9a7-446b901ce85e",
+                     "failed":false,
+                     "volumeDataDensity":0.56038725,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/C4oTKKGyG7",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":4900810091947,
+                     "reserved":368778349582,
+                     "uuid":"a730d4de-4b26-4fb0-8ea7-973f6d6e3d38",
+                     "failed":false,
+                     "volumeDataDensity":-0.041238844,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DuTMn6Tj7A",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":888517673826,
+                     "reserved":89181671360,
+                     "uuid":"6ce14ea1-e9cc-4214-a38c-702e43dfc225",
+                     "failed":false,
+                     "volumeDataDensity":-0.14646214,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"9b7c6a92-00b4-43c0-8538-16bf6a2a6e86",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/aIfhaVUFFU",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1112135820301,
+                     "reserved":1581616458567,
+                     "uuid":"d7fbc3c0-e018-4d0c-93d6-1e4320024f37",
+                     "failed":false,
+                     "volumeDataDensity":0.17863922,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/9UKaEm1hBf",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2805337544126,
+                     "reserved":259804526093,
+                     "uuid":"c338817d-39da-4204-9f3e-e78c1cb80fa9",
+                     "failed":false,
+                     "volumeDataDensity":-0.26326352,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/jRPKC8GFF7",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":277317527378,
+                     "reserved":91201329596,
+                     "uuid":"b191859b-2d1b-4b04-986a-615343d851b7",
+                     "failed":false,
+                     "volumeDataDensity":-0.12696058,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"aa5cad0d-18ee-4fa4-bfc3-de24b1ddf43e",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/SsM7v8DZRX",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":4928882524014,
+                     "reserved":103409074554,
+                     "uuid":"3dcedf8f-eda5-46e0-b7ca-eb10e9abab24",
+                     "failed":false,
+                     "volumeDataDensity":-0.021176994,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ddw3iHJ8NV",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":173025128099,
+                     "reserved":69338127289,
+                     "uuid":"56360e39-55a6-4f27-985b-07f02268aa30",
+                     "failed":false,
+                     "volumeDataDensity":0.29144067,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/QhJKxPPkBp",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":38779686081,
+                     "reserved":17413835528,
+                     "uuid":"12be9ff8-1000-4094-a61a-0e9f021c624b",
+                     "failed":false,
+                     "volumeDataDensity":0.34514335,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"db04e801-6a9b-438a-9227-f60f99d6ea40",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"df18b2b1-1141-4606-acad-e97824948541",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.7231848,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/uf08KiAozw",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":597758270324,
+                     "reserved":176845142858,
+                     "uuid":"96873d1f-2eb6-4ab9-a430-4c3a536b3573",
+                     "failed":false,
+                     "volumeDataDensity":0.38944376,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/PELgPZZsih",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":2495566844910,
+                     "reserved":247891930419,
+                     "uuid":"306d83e0-594e-4ee4-a843-a05e72a7307f",
+                     "failed":false,
+                     "volumeDataDensity":-0.3934049,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/FgOmE7zq5W",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":4776442517189,
+                     "reserved":245904976512,
+                     "uuid":"375e6351-49d4-4977-b873-9edb9e5cfd25",
+                     "failed":false,
+                     "volumeDataDensity":-0.10261077,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"5bdb071b-4cf8-4ad2-8531-a82b501eff5d",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/mhUikMiNt0",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2565568097766,
+                     "reserved":5392322547,
+                     "uuid":"94354f4c-566b-43f5-bba1-85fc1092e23b",
+                     "failed":false,
+                     "volumeDataDensity":-0.1678667,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/g9P96MnzqD",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":135768233816,
+                     "reserved":88750183407,
+                     "uuid":"9e139df9-7562-4ffb-9dfa-401bb88d72bf",
+                     "failed":false,
+                     "volumeDataDensity":0.35872695,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/DyI2e7PW7m",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":47506325199,
+                     "reserved":15449546017,
+                     "uuid":"91a94f2a-da87-46f5-8cd3-105f5016fbf7",
+                     "failed":false,
+                     "volumeDataDensity":0.60759276,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"233dcd45-2748-47de-90d4-312cb6562c91",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/waxJuUV70y",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1747238819233,
+                     "reserved":122126373022,
+                     "uuid":"9becb45a-425d-4e60-9279-1d85dafd6a91",
+                     "failed":false,
+                     "volumeDataDensity":-0.2404204,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/lHUdorGJ8Y",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":343860472121,
+                     "reserved":32983724729,
+                     "uuid":"2fa94611-a813-4e55-b522-f862447f912d",
+                     "failed":false,
+                     "volumeDataDensity":0.33442515,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/3OhfScorUm",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":558666637109,
+                     "reserved":4728811982,
+                     "uuid":"5f46423f-c5f7-4aac-8cf9-370831d35b9d",
+                     "failed":false,
+                     "volumeDataDensity":0.12869328,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"d9a7465c-eff1-4a44-869b-ef8a197c8c82",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"72c4a301-bbe4-415c-a13d-0354584e5c04",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.2112641,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Ri7PdAHBbp",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":128406624977,
+                     "reserved":42448743593,
+                     "uuid":"865c2699-6714-48a4-92a8-3f0f423ae6f7",
+                     "failed":false,
+                     "volumeDataDensity":-0.109736905,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/v6sJmuVnbg",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":57520089052,
+                     "reserved":34610724356,
+                     "uuid":"1e2912dd-9ae2-4830-80e8-9337f086d4ae",
+                     "failed":false,
+                     "volumeDataDensity":0.095750466,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/zdeNAvDlDI",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":833378983501,
+                     "reserved":258665690456,
+                     "uuid":"e9c9369b-58f8-4c3d-abd9-d08bd0265305",
+                     "failed":false,
+                     "volumeDataDensity":-0.0048669875,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"1110b4e2-6b6c-4df6-befe-52a2745b5d0b",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/eySemTSXUJ",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":310866037985,
+                     "reserved":64140124851,
+                     "uuid":"a6a3f0c0-abac-4a6e-a446-9839c4513cf4",
+                     "failed":false,
+                     "volumeDataDensity":-0.28191733,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/MKAUuI45in",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4735229114283,
+                     "reserved":390465749439,
+                     "uuid":"51380f29-ffb3-4c45-a2c2-fabdf17bfe5d",
+                     "failed":false,
+                     "volumeDataDensity":0.021389425,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/FQbXyUk4Qo",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":550745129633,
+                     "reserved":50127818272,
+                     "uuid":"4d703c63-01d4-41ab-b8d1-f25ba6c699c1",
+                     "failed":false,
+                     "volumeDataDensity":-0.090786815,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"accf9a53-8138-43e9-9444-56ddc9e98d52",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/lsVX7TuRkJ",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":1656492911790,
+                     "reserved":800425096140,
+                     "uuid":"b0209bd9-3393-4ef7-9c99-5a3e6f264ecc",
+                     "failed":false,
+                     "volumeDataDensity":0.21565923,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/5Ej1G2KiAx",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":139795586833,
+                     "reserved":10856542374,
+                     "uuid":"a8280c17-2bcf-4020-a563-f069ebc2869f",
+                     "failed":false,
+                     "volumeDataDensity":0.1266205,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/XNdgmP38u9",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":3115498115082,
+                     "reserved":437959172796,
+                     "uuid":"00f46c85-e4b8-4f3f-8066-549a6bc95ad0",
+                     "failed":false,
+                     "volumeDataDensity":-0.26453632,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"28dc9c97-bcbf-4f1e-b941-aa5c0a7ab79a",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"f7e3728f-bdd6-4d4b-a54f-d8795b25e917",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.2569578,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Pj8bPiC09D",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":251907270737,
+                     "reserved":20249812174,
+                     "uuid":"83fd5cd5-7e9c-47ee-98ed-5230767583f2",
+                     "failed":false,
+                     "volumeDataDensity":-0.7497477,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UD0ByWQIhP",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":307399553755,
+                     "reserved":181340462018,
+                     "uuid":"bacc452f-e480-44b1-accb-bd666b69ba12",
+                     "failed":false,
+                     "volumeDataDensity":0.07022525,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/J2PsyBDwIL",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":72534493384,
+                     "reserved":6384174978,
+                     "uuid":"b868da3d-d8f5-42a1-b489-b2d0ef553ede",
+                     "failed":false,
+                     "volumeDataDensity":-0.62408555,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"ec841a32-a390-4df4-90c7-5e2f59ec7fc3",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/3Hit3Nf5vH",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4602225295351,
+                     "reserved":132065635352,
+                     "uuid":"3177db61-79bf-47d6-be56-9000ea0ec412",
+                     "failed":false,
+                     "volumeDataDensity":-0.067928195,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/ailkGi7hf3",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":51118540503,
+                     "reserved":5541516835,
+                     "uuid":"0c676d98-43d9-4aee-a3b1-e4061309f481",
+                     "failed":false,
+                     "volumeDataDensity":0.061000407,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/YGZ8Ny8QgL",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4164845108101,
+                     "reserved":1318497842610,
+                     "uuid":"0e55ac67-38fa-439f-9bcd-4a9b8be902b1",
+                     "failed":false,
+                     "volumeDataDensity":0.05998361,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"384c757c-d095-4d05-bd7f-9454b09dff26",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/dgqUKGwbrH",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":456069059293,
+                     "reserved":37709340005,
+                     "uuid":"747469e5-8541-4b2d-8984-65fe21a257a2",
+                     "failed":false,
+                     "volumeDataDensity":-0.17575485,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Zt98uulaqU",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":798260753694,
+                     "reserved":101988565243,
+                     "uuid":"905d7042-796b-4ac0-8881-9e038b1f18f7",
+                     "failed":false,
+                     "volumeDataDensity":-0.37605155,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Rs4aAu3HqJ",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":2772445220765,
+                     "reserved":708830839324,
+                     "uuid":"3de8cd9e-60b9-4b53-aa42-7e7099264b44",
+                     "failed":false,
+                     "volumeDataDensity":0.07218051,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"47bcb79d-6475-4d77-a666-25f13ad9f92b",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"c0090a41-4d04-4590-9fef-ea76a6bdcd58",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":3.0899496,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Y355FnXt8v",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":53561691468,
+                     "reserved":684332622996,
+                     "uuid":"b74bcb69-e279-4279-b558-cb15a1a83612",
+                     "failed":false,
+                     "volumeDataDensity":0.20126641,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/n7is0YnEQK",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":448637150781,
+                     "reserved":94664942511,
+                     "uuid":"207e8aa4-c7e9-476e-ae70-e20cffa4c0b9",
+                     "failed":false,
+                     "volumeDataDensity":-0.33966082,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/H0kDLnnCb0",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":544306018020,
+                     "reserved":7727289282,
+                     "uuid":"ea8546f8-3734-4a0f-ae84-c94b55a652fe",
+                     "failed":false,
+                     "volumeDataDensity":-0.568839,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"98efb4d1-91d2-4f70-b021-cbecc93297da",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/yAsmwx1bmS",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":5909148439411,
+                     "reserved":67441627458,
+                     "uuid":"b1ab1084-2e3e-43d4-8b4a-b0be9e165d92",
+                     "failed":false,
+                     "volumeDataDensity":-0.15863568,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/5fItDhQLpI",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":532230069899,
+                     "reserved":353080502603,
+                     "uuid":"dda08aa2-8b4b-4ca7-87e4-11788cd08337",
+                     "failed":false,
+                     "volumeDataDensity":0.38521248,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Il2GBdRC6p",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":59947628664,
+                     "reserved":90508831012,
+                     "uuid":"4ceb9d71-0535-41e5-99e9-8528bcac1ebf",
+                     "failed":false,
+                     "volumeDataDensity":0.46862596,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"0521367a-038e-4fa9-88ac-c7edc1edeb34",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/zqCDLHBfDW",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":65584513818,
+                     "reserved":11852045613,
+                     "uuid":"0f48adc8-72ca-4f87-8eeb-7d232da34104",
+                     "failed":false,
+                     "volumeDataDensity":-0.4092155,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Hl349ENUDG",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":1245486845366,
+                     "reserved":1055477885539,
+                     "uuid":"dac752c2-cc2d-4c30-b9ea-3cb33aaa3fbe",
+                     "failed":false,
+                     "volumeDataDensity":0.17803912,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/d7VZ7KA9WK",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":2591368840002,
+                     "reserved":377060184077,
+                     "uuid":"2f407e21-d0cf-4a31-88a9-9ef280ea025a",
+                     "failed":false,
+                     "volumeDataDensity":-0.38045472,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"4e0b777f-1cd7-4058-8bf9-fe7f6604e17d",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"25d8b28d-64ce-434d-8bc1-8c28504fe1b2",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.3189394,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/yXi4VEdU1x",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":335858021133,
+                     "reserved":60032603340,
+                     "uuid":"95ae0460-00fe-4e89-81c5-0eef452e4bab",
+                     "failed":false,
+                     "volumeDataDensity":-0.30753535,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/JCHKiwpqgD",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":1035095512088,
+                     "reserved":208093814253,
+                     "uuid":"e39bbe7c-74d0-4b70-a7f2-c06c9035dbe7",
+                     "failed":false,
+                     "volumeDataDensity":0.03855534,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/fDt3EnRJP3",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":79988558393,
+                     "reserved":53797832660,
+                     "uuid":"0dc6e3fe-4551-4fd7-a3ef-8431f4390bdd",
+                     "failed":false,
+                     "volumeDataDensity":-0.10762024,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"ec36824d-3a8d-4d07-9753-c7c5cd4cdeed",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/0Na0lNsCD8",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":365841853890,
+                     "reserved":104574674229,
+                     "uuid":"020c0285-754f-48a1-8558-885550029900",
+                     "failed":false,
+                     "volumeDataDensity":-0.07884413,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/jOi1QcoIUL",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4938827546806,
+                     "reserved":100584055816,
+                     "uuid":"8e796b26-b2e2-4990-8f83-20f0520b0654",
+                     "failed":false,
+                     "volumeDataDensity":-0.056236863,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/LwcjCyD9FJ",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":30111623805,
+                     "reserved":106886603834,
+                     "uuid":"814ba4f3-6acc-41b0-8aeb-9f5af8ddf8c4",
+                     "failed":false,
+                     "volumeDataDensity":0.6161518,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"a0f590b2-e0ad-415f-9097-63b3b3b53ad9",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/hPSSUrFQSF",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":13950566384,
+                     "reserved":3424678749,
+                     "uuid":"bf37893e-c881-4295-a1be-ef95cfcafa94",
+                     "failed":false,
+                     "volumeDataDensity":0.40636277,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/D8Xaul9lDr",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":15812167088,
+                     "reserved":17932208879,
+                     "uuid":"4bee508c-3683-48b1-8ee0-cfa693a71e8f",
+                     "failed":false,
+                     "volumeDataDensity":0.35814345,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/2b7TRRH218",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":176811967008,
+                     "reserved":3608802895,
+                     "uuid":"804e2273-1ac6-43d0-af37-4a3f062e74be",
+                     "failed":false,
+                     "volumeDataDensity":-0.3494895,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"99e19a77-22d3-4547-a5e7-dea03568b36e",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"45b1ed7c-29ac-4740-b841-79ebbbf4a709",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.2388608,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/iZ5p8MkCNX",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":500875550478,
+                     "reserved":8520310373,
+                     "uuid":"1a29df95-478f-45c1-8c28-58226516a495",
+                     "failed":false,
+                     "volumeDataDensity":-0.3432771,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/BadsdmW96M",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":95002357118,
+                     "reserved":65768929826,
+                     "uuid":"144c7a50-6c4d-4edb-badb-4965a4b915c0",
+                     "failed":false,
+                     "volumeDataDensity":0.32571065,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/3gfENseKD6",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":15687133752,
+                     "reserved":11181337020,
+                     "uuid":"6aa66d78-712d-4276-959e-cbc1a5d60cdb",
+                     "failed":false,
+                     "volumeDataDensity":0.32692096,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"c5482ff7-7d38-48d9-b252-a4ad97406d9f",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/GAuv1pYhAC",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1279952464332,
+                     "reserved":570652500865,
+                     "uuid":"5894ae86-1f31-4f27-93b9-206808a8dd52",
+                     "failed":false,
+                     "volumeDataDensity":-7.8856945E-4,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/NRBhjqpX1h",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":11180837190,
+                     "reserved":26248589964,
+                     "uuid":"181b498e-9943-47d3-a03a-bda15de8859a",
+                     "failed":false,
+                     "volumeDataDensity":0.49616718,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/DF2JTImYbk",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":411605219690,
+                     "reserved":66460232831,
+                     "uuid":"ae0011ea-6fa4-4f3c-9346-de99fc6c01f6",
+                     "failed":false,
+                     "volumeDataDensity":-0.4233235,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"719dfd5e-71ce-4b2d-9439-eaaa4a352e49",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/qPP7zyFTGL",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":478632219059,
+                     "reserved":99587675538,
+                     "uuid":"784e33f0-b481-4270-92a2-8373608a53cb",
+                     "failed":false,
+                     "volumeDataDensity":-0.01644826,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tIkY8CDdPe",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":4720094388018,
+                     "reserved":78606398239,
+                     "uuid":"fbec6b53-7162-4083-b65b-01e6046d639d",
+                     "failed":false,
+                     "volumeDataDensity":-0.013954163,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/zZ8m2s5ov0",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":106214520733,
+                     "reserved":23384083378,
+                     "uuid":"eebadcd9-c25f-48ed-9468-30b2a196a039",
+                     "failed":false,
+                     "volumeDataDensity":0.29227036,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"319986ea-9bd3-4daa-a224-6ae687275eed",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"53641c4d-09b2-43de-b302-e5dd369b107c",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.907052,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/56F6jkRZ0w",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":3525367348780,
+                     "reserved":370576700021,
+                     "uuid":"ca533dd4-98dd-40c9-a379-eb8ee5fc4501",
+                     "failed":false,
+                     "volumeDataDensity":0.1630185,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/2TmDyYpCfU",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":4604022306997,
+                     "reserved":138258447558,
+                     "uuid":"f3990db5-f684-4d41-9a46-3bc5c3c8c726",
+                     "failed":false,
+                     "volumeDataDensity":-0.32189667,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Z33szLlOUW",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":80862103450,
+                     "reserved":56731282655,
+                     "uuid":"77677882-ffcd-41c9-b3ca-7f1a03462689",
+                     "failed":false,
+                     "volumeDataDensity":0.4993887,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"994ba111-85ca-4545-b6e5-4056d0166b21",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/7Tv1eUSpvg",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3358608453519,
+                     "reserved":147907048448,
+                     "uuid":"0029e6b5-ded5-4ed7-85ad-eaaa157f6f4d",
+                     "failed":false,
+                     "volumeDataDensity":7.2419643E-4,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/coMG2VdFnd",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1358496222872,
+                     "reserved":215194877830,
+                     "uuid":"187db856-eac0-4935-a0da-071cfc35ddc0",
+                     "failed":false,
+                     "volumeDataDensity":0.021203965,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/TMRFv0UXwi",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":157442729277,
+                     "reserved":13806355932,
+                     "uuid":"142c0660-ebfd-4c29-89b7-67de74800524",
+                     "failed":false,
+                     "volumeDataDensity":-0.46544778,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"2ce0fd19-7452-469f-8604-92e01bb5fee5",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/OoDFBDGfbk",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":1705148445866,
+                     "reserved":315464777181,
+                     "uuid":"5dee2ed5-4f34-46a2-b81d-6923495b41c4",
+                     "failed":false,
+                     "volumeDataDensity":-0.08241606,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/WYoGgCPlXL",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":115234071073,
+                     "reserved":4289926459,
+                     "uuid":"c9814788-9cb7-4807-b4f4-b024c8d835ed",
+                     "failed":false,
+                     "volumeDataDensity":-0.03604138,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/3y0wmNMWuZ",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":169899562294,
+                     "reserved":179609614656,
+                     "uuid":"aeb35b15-697e-41fa-9f1d-d2161815d93a",
+                     "failed":false,
+                     "volumeDataDensity":0.3169148,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"70546077-8c40-4ca3-b0e5-e1aa23a06aac",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"89c2c18c-ccd8-4a00-9df4-cb88168c4520",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.2200859,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Ym1X66Cjpj",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":99535778968,
+                     "reserved":361138530,
+                     "uuid":"057f5561-74f8-4176-9923-e31bea669d99",
+                     "failed":false,
+                     "volumeDataDensity":-0.20587778,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/5dT3tnYtXD",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":3639962690303,
+                     "reserved":1110966700534,
+                     "uuid":"6a241d9e-cc00-4828-82d1-8e6d555acee1",
+                     "failed":false,
+                     "volumeDataDensity":0.048571825,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/rRCSItCpVw",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":4679593491143,
+                     "reserved":373083786763,
+                     "uuid":"c6f2e195-70d4-4a6b-b321-69765ba2c59f",
+                     "failed":false,
+                     "volumeDataDensity":-0.038556755,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"290a8806-9edf-4040-bf7c-f9839896e422",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/qjji8UQqT4",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":373701425497,
+                     "reserved":18979890740,
+                     "uuid":"6456cd96-90e7-4b67-991e-7e4b4a93a62c",
+                     "failed":false,
+                     "volumeDataDensity":0.06677079,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/3HdD42OrZl",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":6577430459,
+                     "reserved":1231650464,
+                     "uuid":"9ff4201c-6f14-4a97-a0a8-311c0cd440cd",
+                     "failed":false,
+                     "volumeDataDensity":0.6768614,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/YQzApXcWiG",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":813301805000,
+                     "reserved":98576032483,
+                     "uuid":"ae603863-30bf-467d-9344-b6b01ad4e9ae",
+                     "failed":false,
+                     "volumeDataDensity":-0.19228882,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"e96b3a1a-0035-4e2e-99ff-2549a6329ad0",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/xUh3EGOqmk",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":8143096767781,
+                     "reserved":181512406590,
+                     "uuid":"57b8e94e-a65a-4d36-8f10-4e96f210560d",
+                     "failed":false,
+                     "volumeDataDensity":-0.34079188,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/LyAjTzkk5G",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":171748681258,
+                     "reserved":643456880996,
+                     "uuid":"9b05e7d4-668e-4ae5-827f-5da078711c8f",
+                     "failed":false,
+                     "volumeDataDensity":0.55055666,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/12UWnhLHw7",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":271802373937,
+                     "reserved":37040987980,
+                     "uuid":"9cdc3836-1dd3-4bb5-9b0f-ccb5b77eb7b2",
+                     "failed":false,
+                     "volumeDataDensity":0.099809796,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"f7e4b6fc-263e-4e19-942b-348becedfdf4",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"dec916ad-3fee-48e0-980f-96de9167e910",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.5549583,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/jjDiopdCJS",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":5779926221135,
+                     "reserved":770555701530,
+                     "uuid":"93bc25de-6e55-4531-8cc2-14fbc5c67c74",
+                     "failed":false,
+                     "volumeDataDensity":-0.030275762,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/o3zotwsiPg",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":5025421571514,
+                     "reserved":229287569588,
+                     "uuid":"32799090-541d-4b18-8717-5c247c21f7fc",
+                     "failed":false,
+                     "volumeDataDensity":0.026992857,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/wilxtDwJRL",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":40530223645,
+                     "reserved":359468730,
+                     "uuid":"e61118ed-b79c-4f7d-a6a4-861e90dc57fb",
+                     "failed":false,
+                     "volumeDataDensity":0.36245778,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"8b082030-ed9e-47cd-9a4b-4c4f80a32e38",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/VCGtRiuw13",
+                     "capacity":2000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":51359633884,
+                     "reserved":70994192663,
+                     "uuid":"6f0e67a5-fd4b-4d3e-9b8a-104cb0a9fe1e",
+                     "failed":false,
+                     "volumeDataDensity":0.080704264,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/hvGa7oTgLL",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":147226280977,
+                     "reserved":66877941040,
+                     "uuid":"e527759f-0c10-4406-9617-77dd8a4db494",
+                     "failed":false,
+                     "volumeDataDensity":-0.12521093,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/933sT0E7dL",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":376504506550,
+                     "reserved":203936122788,
+                     "uuid":"ca06689c-e684-435a-b3f4-69d936cf89ff",
+                     "failed":false,
+                     "volumeDataDensity":-0.027326003,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"09cf6d36-9910-45fb-9afc-ae7da575deb7",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/piz0EWOApf",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":3116946673947,
+                     "reserved":338818466596,
+                     "uuid":"49a9fe0a-3706-4bfe-bc51-6ad61b8a3b79",
+                     "failed":false,
+                     "volumeDataDensity":0.17777622,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Pe2mbhua98",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":7135455802462,
+                     "reserved":678445480914,
+                     "uuid":"7b8d03a6-ea40-4fc0-9d1d-1db0813e2a88",
+                     "failed":false,
+                     "volumeDataDensity":-0.12810272,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/9wzM6kUUxv",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":45890945160,
+                     "reserved":16705677925,
+                     "uuid":"94c71073-068d-4508-9ac3-2e2def4c5fc9",
+                     "failed":false,
+                     "volumeDataDensity":0.59611183,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"95cf1e18-c4be-4d0a-9f97-217c8c74ca49",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"26b55e98-6a48-4f2b-aa3b-075ab7c5a9a4",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.0915017,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/HLmODHyKMY",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":83660339538,
+                     "reserved":53748559452,
+                     "uuid":"21a460fd-7338-4607-b359-3c0497d08748",
+                     "failed":false,
+                     "volumeDataDensity":0.12627375,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/uJeMESDGrN",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":149969110176,
+                     "reserved":6719833185,
+                     "uuid":"4fad139b-b062-40d3-8fb7-9336164b32c2",
+                     "failed":false,
+                     "volumeDataDensity":0.08468026,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8tG69Oj5GV",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":269215450760,
+                     "reserved":60486529361,
+                     "uuid":"c66d9bd1-d9be-467f-9be9-0ff681b95dc3",
+                     "failed":false,
+                     "volumeDataDensity":-0.14652139,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"f6c9a7c2-c161-4d4f-b269-3aac6aa0cf9d",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/5JuKTbBKiG",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2976252196876,
+                     "reserved":971338471024,
+                     "uuid":"48808e9b-b075-43d3-a51a-58bda627cd45",
+                     "failed":false,
+                     "volumeDataDensity":-0.34130633,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/1awvIk9Kqc",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":629301651608,
+                     "reserved":62188263162,
+                     "uuid":"307710c9-ae74-420f-a70e-899d53a87743",
+                     "failed":false,
+                     "volumeDataDensity":-0.3536622,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/nyzZ83ulsD",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":684431304188,
+                     "reserved":1073057295468,
+                     "uuid":"89952af6-5f6c-43ac-97ed-2bd147310574",
+                     "failed":false,
+                     "volumeDataDensity":0.28198516,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"dc6814fa-7b9a-4fa5-b362-1e00e03ef541",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/oVfXx5aUfy",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":258820277857,
+                     "reserved":527727158401,
+                     "uuid":"1fd9fdaa-39c5-4d69-96e7-b989594755c0",
+                     "failed":false,
+                     "volumeDataDensity":0.031115726,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/AiZNdQrAzY",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":105261472705,
+                     "reserved":531659213739,
+                     "uuid":"768cbd4b-8c5d-4c0a-a62b-ed9487a3f3cf",
+                     "failed":false,
+                     "volumeDataDensity":0.09316031,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/HXMBkFnpBA",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":372737255921,
+                     "reserved":115044767827,
+                     "uuid":"905d670e-f3e9-4d09-bc8e-e1159ed98b97",
+                     "failed":false,
+                     "volumeDataDensity":-0.6327964,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"c599103f-5819-407e-8ac1-818e818d0abd",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"ba39ce1d-216a-4e05-b097-8a0639697412",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.4086019,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/gdN8MgC2Fj",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":6936014971675,
+                     "reserved":818684344495,
+                     "uuid":"c964b876-ac38-4470-9c10-f84bbdde3c34",
+                     "failed":false,
+                     "volumeDataDensity":-0.11593509,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xJQyRUoaAy",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":4598528186272,
+                     "reserved":1105459660165,
+                     "uuid":"741115ee-9118-4293-89ff-3e65e75c2b20",
+                     "failed":false,
+                     "volumeDataDensity":0.06977326,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/rCepshPK3B",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":3332254751868,
+                     "reserved":583586722843,
+                     "uuid":"9067dc3c-ad67-4c7d-8761-5ffd68c871a5",
+                     "failed":false,
+                     "volumeDataDensity":0.09539068,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"fe68752a-4920-4cbf-b8ab-7fcaacc2a880",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/f9DWvbzOya",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":59853340735,
+                     "reserved":6738204064,
+                     "uuid":"4891c1e7-651e-4ab6-844d-69f9c6af954b",
+                     "failed":false,
+                     "volumeDataDensity":0.49207032,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mprFuCElSv",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2943241496652,
+                     "reserved":2624030315,
+                     "uuid":"a897662d-e6f4-4a26-af96-ecc3177f5ba4",
+                     "failed":false,
+                     "volumeDataDensity":-0.040127814,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/qhUTeuRl58",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":401406849793,
+                     "reserved":100276108989,
+                     "uuid":"c654b5b8-7663-4daa-8155-b091da009ae2",
+                     "failed":false,
+                     "volumeDataDensity":0.02684611,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"c38090ef-77a8-4d50-9f4d-ff6c1028dda3",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/qyY0zwOTdJ",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":64028163427,
+                     "reserved":7152523904,
+                     "uuid":"c9dbf8ec-40d3-44ea-809d-bab952bc4754",
+                     "failed":false,
+                     "volumeDataDensity":-0.53336847,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DhdwPi6zMh",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":1217842730539,
+                     "reserved":71635216836,
+                     "uuid":"a6935f20-be68-420b-a955-645660d65bc7",
+                     "failed":false,
+                     "volumeDataDensity":0.0026316643,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/riIykVeEWw",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":109282934514,
+                     "reserved":117112626907,
+                     "uuid":"fe2e838c-d699-452b-8f0a-8c99f5772746",
+                     "failed":false,
+                     "volumeDataDensity":0.032458432,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"61cd3b02-d788-4546-bbc9-9574689f613c",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"1a98d321-dde7-4ac1-bd83-eb2acc13025c",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":3.1012845,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DP6dN7u1IU",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":360229026685,
+                     "reserved":91128583218,
+                     "uuid":"472942e5-f598-4e5f-8840-4b961e533443",
+                     "failed":false,
+                     "volumeDataDensity":-0.17348883,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hOsbkDuW3j",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":808319595048,
+                     "reserved":817079873102,
+                     "uuid":"6883829e-cf6e-4bbc-8a46-8f4928ca9130",
+                     "failed":false,
+                     "volumeDataDensity":0.17872542,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/skHAooYCLr",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":1416065815498,
+                     "reserved":169233804611,
+                     "uuid":"b1be33e7-8794-481d-996b-8172a0db09a3",
+                     "failed":false,
+                     "volumeDataDensity":-0.43879887,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"c3037283-80aa-4f44-ae85-6b1a0e810985",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/ZnMdJrnbTw",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":422075923100,
+                     "reserved":17894740514,
+                     "uuid":"416ea6bd-7545-4d2e-bddb-c1d26a2ee43e",
+                     "failed":false,
+                     "volumeDataDensity":0.2802819,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/0oQSaGV5SX",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":8055856699805,
+                     "reserved":516222224884,
+                     "uuid":"16dd4e53-627b-4340-8f60-7725be791534",
+                     "failed":false,
+                     "volumeDataDensity":-0.050493956,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/A4mUg3sdCd",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":53568052271,
+                     "reserved":76591609541,
+                     "uuid":"38958b7f-1bb3-42da-a138-5cbbbf32b5b4",
+                     "failed":false,
+                     "volumeDataDensity":0.73343015,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"2936fba9-1812-4e30-8e69-1e92201fccb9",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/ShDtBAzYgo",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":327571358762,
+                     "reserved":39191606662,
+                     "uuid":"28295119-0a4f-4484-aa82-334a1f173b55",
+                     "failed":false,
+                     "volumeDataDensity":-0.7357502,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/zOm2GAVcYb",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":844492473140,
+                     "reserved":347905904345,
+                     "uuid":"0d322487-3ce5-49d1-82d1-ed9356d591bf",
+                     "failed":false,
+                     "volumeDataDensity":0.061770707,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UQoGzsyLdm",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":286728595849,
+                     "reserved":38038397906,
+                     "uuid":"73354724-19a8-47b4-93c2-1dc149cf4b14",
+                     "failed":false,
+                     "volumeDataDensity":-0.44854456,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"2835ffbe-4d5b-4a58-b3da-2f620d9a63ca",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"38e885e9-969d-4a5e-ac51-dab86fae88f2",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.9453588,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/UHrlDnYkhD",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":209296012951,
+                     "reserved":69015224324,
+                     "uuid":"ceb130f9-e6f0-40a6-92de-d39817a86a68",
+                     "failed":false,
+                     "volumeDataDensity":0.17602295,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ZGUMmcqXOO",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":4916788842916,
+                     "reserved":174805482429,
+                     "uuid":"8c46c095-6137-4aca-9459-9a6bf1e7d560",
+                     "failed":false,
+                     "volumeDataDensity":-0.035689414,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/SvIGz2wy09",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":476529463213,
+                     "reserved":25391646744,
+                     "uuid":"5ec2be3b-1f00-4919-b9c0-ae9f8a8fac25",
+                     "failed":false,
+                     "volumeDataDensity":0.1931786,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"0d7834bb-5e77-49e6-a524-522fe21e68af",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/yzxtXLD15c",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3760758987327,
+                     "reserved":832445310647,
+                     "uuid":"cc5ae548-e233-44fc-94b3-2efd3250823b",
+                     "failed":false,
+                     "volumeDataDensity":-0.03591937,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/iFr7RuEz39",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":24333104007,
+                     "reserved":46467302870,
+                     "uuid":"32369cdd-2847-4946-bb8d-d26d0956f5ca",
+                     "failed":false,
+                     "volumeDataDensity":0.7976421,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/8q600f1wBD",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":5538174093774,
+                     "reserved":761037163193,
+                     "uuid":"bb052239-c097-4712-b4f9-93454490d986",
+                     "failed":false,
+                     "volumeDataDensity":-0.021204948,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"17771d2c-6f1a-460b-835f-f24da305417c",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/IgGQ9T1iyz",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":2269601579125,
+                     "reserved":184870716204,
+                     "uuid":"996244ed-531b-4085-abc0-0ea7cb1ec40c",
+                     "failed":false,
+                     "volumeDataDensity":-0.059710115,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/fcb04uZemA",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":143452338375,
+                     "reserved":30545254536,
+                     "uuid":"427fccc2-c4bb-478d-8079-cf7181898236",
+                     "failed":false,
+                     "volumeDataDensity":-0.4349146,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Vpe5zEnzqj",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":416947634317,
+                     "reserved":109605866643,
+                     "uuid":"11b46b39-308d-4249-bc37-46a9012f3bc1",
+                     "failed":false,
+                     "volumeDataDensity":0.1910767,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"e277f45b-38a7-4def-967a-cb0ae45a3fcd",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"d5980ad3-463b-4f71-8c67-30bf1a0a7661",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.1122098,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/it4MrGTuxj",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":70946836457,
+                     "reserved":52848492798,
+                     "uuid":"0e7307ab-e6cd-4266-ad5b-b5e47636e277",
+                     "failed":false,
+                     "volumeDataDensity":-0.19787917,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/VXRW8JVWPi",
+                     "capacity":9000000000000,
+                     "storageType":"DISK",
+                     "used":125633242666,
+                     "reserved":1784289195526,
+                     "uuid":"f1a7c6e2-88ad-44c5-88d9-718db9b2b040",
+                     "failed":false,
+                     "volumeDataDensity":0.07176782,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/QYzGNVrLJ7",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":517511663251,
+                     "reserved":55454675822,
+                     "uuid":"8c698b61-dbda-47d8-885c-418ef4089d6e",
+                     "failed":false,
+                     "volumeDataDensity":-0.8611767,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"0b44343b-7925-4d42-b369-92d47e630f53",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/lExzGBzCbb",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":722723942969,
+                     "reserved":158306025086,
+                     "uuid":"7f462758-9c54-423a-a280-33f8636b0a96",
+                     "failed":false,
+                     "volumeDataDensity":-0.005297184,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/RItGcbhDmV",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":6349340367339,
+                     "reserved":911250932266,
+                     "uuid":"066e2ca5-8a29-48ac-a452-95f95d386eb1",
+                     "failed":false,
+                     "volumeDataDensity":-0.04233581,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/DeYFR24NJh",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":386980651960,
+                     "reserved":89615693265,
+                     "uuid":"2e78f752-cadd-44e9-b332-220de3528232",
+                     "failed":false,
+                     "volumeDataDensity":0.37582952,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"e4758d80-7eb7-41c7-a655-b757aa2c243e",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/uKEvQBMrD3",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":71110101277,
+                     "reserved":122606608999,
+                     "uuid":"129b987e-c505-4005-aee5-f2f6f39202b0",
+                     "failed":false,
+                     "volumeDataDensity":0.32763952,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/FJKC5vsJ28",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":179693727160,
+                     "reserved":11183676092,
+                     "uuid":"fa9cb603-857e-42df-8bc2-07f2694904fb",
+                     "failed":false,
+                     "volumeDataDensity":0.1899234,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/eAyUaKQDdq",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":3894159906924,
+                     "reserved":71456269043,
+                     "uuid":"75b957e7-f8af-4c07-9557-a1dbddfaa3e2",
+                     "failed":false,
+                     "volumeDataDensity":-0.04036039,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"cb3241d3-fe93-4dc1-9071-f3c004407ea2",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"3639bb87-ce77-4724-81cf-44ed72c74d6c",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7539215,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/LxCHp4uW2F",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":147363417408,
+                     "reserved":88233911819,
+                     "uuid":"a4ce80d7-f015-4335-a1cc-aade1df1dd2a",
+                     "failed":false,
+                     "volumeDataDensity":0.28140992,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/aQCX6m53zZ",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":496896373612,
+                     "reserved":195999882332,
+                     "uuid":"42f55314-3a8c-456b-bef0-c0cbd907a129",
+                     "failed":false,
+                     "volumeDataDensity":0.021261036,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UbcfsE1MA0",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":428033935546,
+                     "reserved":38449921993,
+                     "uuid":"7327e62e-64c6-45e7-be1c-411a40679044",
+                     "failed":false,
+                     "volumeDataDensity":-0.2880922,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"6040b59b-fb17-414e-b6d0-acfa91e23b0b",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/defr1xaSK2",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":49532044755,
+                     "reserved":8513572607,
+                     "uuid":"44e1cd30-2468-4b47-ad68-d2098a69b48e",
+                     "failed":false,
+                     "volumeDataDensity":0.2840218,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Y4qa0ueq7Z",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":535758186022,
+                     "reserved":91407012466,
+                     "uuid":"5ee1f44a-a0f1-4357-b04c-b21d291e38b9",
+                     "failed":false,
+                     "volumeDataDensity":0.16285509,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/rzOClWQnw0",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4122553346652,
+                     "reserved":196615392496,
+                     "uuid":"13a8361c-d918-450b-868a-c58f6b640699",
+                     "failed":false,
+                     "volumeDataDensity":-0.03282422,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"0af7048d-c286-48db-8acb-d60ae3186b8c",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/i6x775TcMA",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":1636734858263,
+                     "reserved":211361702785,
+                     "uuid":"29722a2f-38e6-449d-9e78-bc0bb1184b90",
+                     "failed":false,
+                     "volumeDataDensity":0.19395629,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/4B7T0OKk9b",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":4373900489821,
+                     "reserved":720296088335,
+                     "uuid":"6d90ccd9-5d27-4431-9dd5-77f5a44604c1",
+                     "failed":false,
+                     "volumeDataDensity":-0.19673479,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/eciKuq19Xp",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":186815812851,
+                     "reserved":31920119103,
+                     "uuid":"7c200d6b-fa5f-4adc-81ae-637bcb4af538",
+                     "failed":false,
+                     "volumeDataDensity":-0.29276603,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"2a4ae6a6-1804-4c8c-8111-b40b545e1c55",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"5935e48e-ff25-4182-b8b7-105df924ae1d",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.6683788,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DFlxjajG2s",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":46399283408,
+                     "reserved":83757115681,
+                     "uuid":"c89afb8d-b991-4feb-a272-44093e63e69f",
+                     "failed":false,
+                     "volumeDataDensity":0.36101678,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/VLqcVMAZdX",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":263157596227,
+                     "reserved":16379064689,
+                     "uuid":"3b2e3ae8-e557-4921-8304-ea0f4d99b098",
+                     "failed":false,
+                     "volumeDataDensity":-0.21349499,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DjUFlcBDoO",
+                     "capacity":200000000000,
+                     "storageType":"DISK",
+                     "used":159398513467,
+                     "reserved":7341345281,
+                     "uuid":"ad092539-f5b1-4469-b092-69306aabcf02",
+                     "failed":false,
+                     "volumeDataDensity":-0.35487387,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"30315b71-9242-4caf-ab09-70de74decce9",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/cPXh0fkSuu",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":594892378338,
+                     "reserved":1539030573,
+                     "uuid":"6b403772-7baa-45ce-9cbf-920590a8c397",
+                     "failed":false,
+                     "volumeDataDensity":-0.39979073,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/BdJ25URxtb",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1271596596727,
+                     "reserved":592112391179,
+                     "uuid":"b42498da-6867-4569-a73c-0be0c8186ca0",
+                     "failed":false,
+                     "volumeDataDensity":0.16344595,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/hOYPLvIqKA",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":822545288175,
+                     "reserved":56210118955,
+                     "uuid":"a4de484c-2947-4876-8e1d-f3b36e1f32e4",
+                     "failed":false,
+                     "volumeDataDensity":-0.5228944,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"8d11e36e-3ce1-4824-803d-6ec4ce5b1060",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/o2VKhzD7jt",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":151462599146,
+                     "reserved":23775668799,
+                     "uuid":"e4e08010-fb4d-4556-ae70-2ae2551213db",
+                     "failed":false,
+                     "volumeDataDensity":-0.07129255,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Qzfz4jr5By",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":376650654451,
+                     "reserved":538817017014,
+                     "uuid":"c969e0d0-16fa-4364-86e2-41544766c1ff",
+                     "failed":false,
+                     "volumeDataDensity":0.04327131,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/doBZG581a6",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":258080760406,
+                     "reserved":96662874915,
+                     "uuid":"d014d9ea-8a50-468a-8aa7-5aceb65cb870",
+                     "failed":false,
+                     "volumeDataDensity":-0.538298,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"af636ccc-57a5-4b52-9f8a-bf17b3e1f299",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"8616920c-abb3-429c-8662-3d0e540633d9",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.0425103,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/dZS8v29UE8",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":22401924238,
+                     "reserved":19420603863,
+                     "uuid":"b0c090bb-6af3-40f7-9f35-0c02eaab3277",
+                     "failed":false,
+                     "volumeDataDensity":0.0796884,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hOnCQ2BQYj",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":999131746242,
+                     "reserved":279795132162,
+                     "uuid":"883871c0-1500-49e7-ad5e-0e990563fa0c",
+                     "failed":false,
+                     "volumeDataDensity":0.089129955,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UetFPgD3b7",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":549262580563,
+                     "reserved":109392735599,
+                     "uuid":"4ae32f01-2664-41c8-886c-bc4d25d599bc",
+                     "failed":false,
+                     "volumeDataDensity":-0.57229733,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"c57a8806-e543-4125-a58c-ec740c7c2d28",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/CRzqWRJlpB",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":283227354640,
+                     "reserved":143161242223,
+                     "uuid":"800704a1-966f-460a-a0e2-d326340307cf",
+                     "failed":false,
+                     "volumeDataDensity":0.23378587,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/I4OwF4anu9",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":461563752534,
+                     "reserved":88460336497,
+                     "uuid":"93139b9e-9297-4ff4-928d-924178f1519d",
+                     "failed":false,
+                     "volumeDataDensity":-0.0897733,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/wqDpCbdFwx",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":418412487602,
+                     "reserved":19156864187,
+                     "uuid":"2e17ed34-e5ad-4b4b-b9bf-9c65d94cc389",
+                     "failed":false,
+                     "volumeDataDensity":-0.20518064,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"fca750cb-a8b8-4b8e-b108-9b18cdd348c5",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/LMFvNJqZRL",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":83487825524,
+                     "reserved":41335116241,
+                     "uuid":"c5dc50c8-bbf4-40f9-a293-ac3f5be7dfa8",
+                     "failed":false,
+                     "volumeDataDensity":0.003336966,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/avVay2xpt6",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":214487745220,
+                     "reserved":26962100186,
+                     "uuid":"46b62724-4236-433c-a2ac-13a3b3976c6c",
+                     "failed":false,
+                     "volumeDataDensity":-0.54944944,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/KgQJgi7hp5",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":10994112829,
+                     "reserved":23123951389,
+                     "uuid":"ada28e1d-c150-4a9a-8259-115a547e7570",
+                     "failed":false,
+                     "volumeDataDensity":0.21986844,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"dd896d7b-cfa9-40cd-b65b-22dd51510631",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"2463d83f-ac6e-4034-868c-f68092f2fcff",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7115581,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/GKdqDaVUsJ",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":20656246643,
+                     "reserved":17890791258,
+                     "uuid":"e7b94721-5b56-48f3-a924-402daa57b0db",
+                     "failed":false,
+                     "volumeDataDensity":-0.08075784,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hKMiqYDEp5",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":155327444709,
+                     "reserved":259433572401,
+                     "uuid":"64ef895b-5678-4079-ac91-1202dfce358b",
+                     "failed":false,
+                     "volumeDataDensity":0.081572965,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/gS9PRBRVtG",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":236546762789,
+                     "reserved":7569636184,
+                     "uuid":"c913814a-db23-424f-8c71-d3562a9a7403",
+                     "failed":false,
+                     "volumeDataDensity":-0.22846942,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"7d112096-f188-4ed6-b18a-724f555ab41d",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DQV1OClW8S",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":50450417113,
+                     "reserved":2516986626,
+                     "uuid":"20652889-e584-4ee9-9b0f-a301a407b13a",
+                     "failed":false,
+                     "volumeDataDensity":0.09429495,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mm0XVVQRuh",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":43346024722,
+                     "reserved":87659036702,
+                     "uuid":"0012af53-4991-404b-9831-0ce51dbce20c",
+                     "failed":false,
+                     "volumeDataDensity":0.17370889,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mfxCJEm0rv",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":254251185591,
+                     "reserved":36511536581,
+                     "uuid":"3320d5dd-ce75-4fb8-b42b-61bf4373fe57",
+                     "failed":false,
+                     "volumeDataDensity":-0.7437227,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"998051b5-8f67-4d15-b962-c8a48cad32f2",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/q7T1Q6ncKo",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":5331777488912,
+                     "reserved":939896617550,
+                     "uuid":"ac6b9172-a12a-4558-a0a0-1922ed2c9455",
+                     "failed":false,
+                     "volumeDataDensity":0.014862657,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/up8HFEwvxB",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":3146313578595,
+                     "reserved":252115514176,
+                     "uuid":"75d838e3-9dd8-4f88-b951-f775bb7157ed",
+                     "failed":false,
+                     "volumeDataDensity":-0.069429636,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/G0jlCyygxG",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":376785461821,
+                     "reserved":9058449424,
+                     "uuid":"a3f33f05-2675-4979-85c9-32607ae5ce61",
+                     "failed":false,
+                     "volumeDataDensity":0.22473913,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"306b474f-2f42-4e01-87d5-ecf799f2e21d",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"907b28cf-c035-4eba-95a2-739246b1f7cc",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.84145,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/no9TMKsWnI",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":2950365716256,
+                     "reserved":647586892340,
+                     "uuid":"d9addde3-47bf-45a5-9f56-89b16c4ba44f",
+                     "failed":false,
+                     "volumeDataDensity":-0.043754756,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Jc7v65gEaT",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":141694041813,
+                     "reserved":12664377521,
+                     "uuid":"02aae2d2-1477-4704-b2a9-1b896e66d53a",
+                     "failed":false,
+                     "volumeDataDensity":0.14164963,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Qt7yhrVWDn",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":42395825223,
+                     "reserved":63078376615,
+                     "uuid":"8313fc07-a6fc-41ec-921e-6cddb6c36560",
+                     "failed":false,
+                     "volumeDataDensity":0.41043383,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"1c46f5d3-a54e-4c77-ac2b-f6ed9aad168c",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Dtt2L64PkX",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":126251765353,
+                     "reserved":33257616870,
+                     "uuid":"02294b61-33eb-49b1-9611-be319e24005d",
+                     "failed":false,
+                     "volumeDataDensity":-0.41240042,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/CAHXD2GYdc",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2731507581584,
+                     "reserved":181559069047,
+                     "uuid":"1bd0f017-cb6c-4f5a-91c6-ab7a9c20c795",
+                     "failed":false,
+                     "volumeDataDensity":-0.1246908,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/yiHPxyhQrY",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1179013035397,
+                     "reserved":276458989700,
+                     "uuid":"5db8075b-8641-40d4-93c4-bfd9ff087fc4",
+                     "failed":false,
+                     "volumeDataDensity":0.13877255,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"4f2beb68-7d0c-4300-b464-b317bbed4eb5",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/r8STzBV95B",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":3367244928801,
+                     "reserved":344806015827,
+                     "uuid":"35457665-482a-46aa-93d3-1c2451191166",
+                     "failed":false,
+                     "volumeDataDensity":-0.21402448,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/7DVXZzrBQ6",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":3447234757542,
+                     "reserved":1219643313250,
+                     "uuid":"5b87c18e-4f9d-4a41-9c29-e354642e4b5d",
+                     "failed":false,
+                     "volumeDataDensity":0.11082685,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/wk4g72Yp2i",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":267457860224,
+                     "reserved":121463537246,
+                     "uuid":"34a27d38-dee0-425b-9275-c3e121d54578",
+                     "failed":false,
+                     "volumeDataDensity":0.24489671,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"630b3bb8-143f-4d33-83e2-e7861d18d57e",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"b093af0d-f500-4994-b82b-f3dc13bfb8fb",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.2460175,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/vinR2u7XLl",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":485110780478,
+                     "reserved":129726562579,
+                     "uuid":"121ca832-7b5f-4400-9dc8-1667fe0d0127",
+                     "failed":false,
+                     "volumeDataDensity":-0.16781425,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/6slCXAL4Pr",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":83809739970,
+                     "reserved":2358301879,
+                     "uuid":"ddbcf44f-65d2-4469-99c0-4563dd2b27bb",
+                     "failed":false,
+                     "volumeDataDensity":0.47208226,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/1h1NYZOD3x",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":2872004125630,
+                     "reserved":928845568705,
+                     "uuid":"9f278f44-db62-47cf-afd9-5dca98c707b7",
+                     "failed":false,
+                     "volumeDataDensity":-0.022602737,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"acfd91d3-d854-4b63-84ea-01e4dd6e46bd",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DLbE0PXODA",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":572568360085,
+                     "reserved":6549430278,
+                     "uuid":"bc126d17-c5fc-46a5-a08b-ff46a55e892e",
+                     "failed":false,
+                     "volumeDataDensity":-0.26364487,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/AzTYkUi6CA",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":47219039946,
+                     "reserved":101888050240,
+                     "uuid":"bf032bd3-0d05-4ef2-b669-c76b4a154d02",
+                     "failed":false,
+                     "volumeDataDensity":0.2982589,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/7UIRgdXHWh",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":37620860894,
+                     "reserved":48725166182,
+                     "uuid":"fb4b06d4-6017-4f1c-a750-2520bf68eff3",
+                     "failed":false,
+                     "volumeDataDensity":0.22748578,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"694c4856-78a0-405c-8014-8fd69cdd56b2",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/24ygP5iN0z",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":3890050271568,
+                     "reserved":730002535987,
+                     "uuid":"6fbccb73-2ee6-4eb1-9dba-79eff324b951",
+                     "failed":false,
+                     "volumeDataDensity":-0.13468617,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xs8g4eNmTC",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":4322836630193,
+                     "reserved":969765010338,
+                     "uuid":"5f6291bc-9c5a-4805-bcbf-8a6815eb66d6",
+                     "failed":false,
+                     "volumeDataDensity":0.05947274,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/NFBgtb1sKG",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":63633626393,
+                     "reserved":39190482607,
+                     "uuid":"021d3a58-0d31-473b-9f8f-985c6791d3b1",
+                     "failed":false,
+                     "volumeDataDensity":0.5999697,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"a0973139-041f-4b58-8ba0-e9d9407c6a88",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"1ad29bef-c853-44ea-8a05-5fba45ff1510",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":3.0332794,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/zRcdMrayjp",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":54698061534,
+                     "reserved":68413050622,
+                     "uuid":"04cfa2cb-4ac4-440f-aed6-3cdb9940a25a",
+                     "failed":false,
+                     "volumeDataDensity":0.43371004,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/CvYKh9O6cl",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":850416334468,
+                     "reserved":38953486799,
+                     "uuid":"5b231983-9f39-465e-856d-3cc8a029bdb0",
+                     "failed":false,
+                     "volumeDataDensity":-0.32443857,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/fOsvFHSmAz",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":196363809255,
+                     "reserved":127277258702,
+                     "uuid":"e2c9d875-d57a-49b8-a3b9-d5f768773c37",
+                     "failed":false,
+                     "volumeDataDensity":0.21758696,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"d387935e-dcca-4466-9b63-233ce14c6e85",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/aionDS6yDb",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1653516872,
+                     "reserved":19819597451,
+                     "uuid":"cfc3bb94-d1ea-427a-844a-e52b049d6284",
+                     "failed":false,
+                     "volumeDataDensity":0.5296742,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/HLTApTT6z9",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3948070834548,
+                     "reserved":964344332714,
+                     "uuid":"19143bf1-a32e-44f7-a948-06bbb5682b20",
+                     "failed":false,
+                     "volumeDataDensity":-0.44272143,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/SmWn3x5Ocj",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":414770721613,
+                     "reserved":166672750447,
+                     "uuid":"1ae25b9a-d359-4e2e-8207-414faa2ad4cb",
+                     "failed":false,
+                     "volumeDataDensity":0.42737457,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"87bd0392-0793-4f46-8f33-6ada0d7004cb",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/ig9rYDZav9",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":44873742770,
+                     "reserved":7455941131,
+                     "uuid":"9f40ee4f-3e7f-4f6d-ae80-43f7265842e1",
+                     "failed":false,
+                     "volumeDataDensity":-0.3688519,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/5UqPAiZdGo",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":765615009700,
+                     "reserved":95116744669,
+                     "uuid":"4b2bbacb-effa-4623-b1e0-05f413fcdea9",
+                     "failed":false,
+                     "volumeDataDensity":0.019185275,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hQh2VslWOQ",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":168079506330,
+                     "reserved":64307271657,
+                     "uuid":"69a59e1d-44ae-4677-ad19-7a8cd71f9e5a",
+                     "failed":false,
+                     "volumeDataDensity":-0.26973662,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"541df125-fe18-46f8-93f6-2e11d201f33c",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"fe148db0-b944-40b4-b98e-f2874b7ac2d5",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.8308225,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/PLrAFawKvO",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":186916676381,
+                     "reserved":58932362942,
+                     "uuid":"08abb575-bb40-454a-b263-3adf698eb6da",
+                     "failed":false,
+                     "volumeDataDensity":-0.16569549,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/CCWxpgc8KX",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":367018020218,
+                     "reserved":86006757616,
+                     "uuid":"5a4130cd-c4a4-4d7c-9954-274c40da662f",
+                     "failed":false,
+                     "volumeDataDensity":-0.10437739,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/fyFBTJtpwI",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":137037668674,
+                     "reserved":21715135349,
+                     "uuid":"91a373fc-64d2-4c1f-af8f-6cfab09f00c5",
+                     "failed":false,
+                     "volumeDataDensity":0.24741429,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"d0ab590c-50dc-4a1e-96bb-7f9a2b38bfb8",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/WoGlp0gOwp",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":498465660882,
+                     "reserved":22380585961,
+                     "uuid":"fa624acc-4c97-49e4-aebc-c62e97c13775",
+                     "failed":false,
+                     "volumeDataDensity":-0.14142862,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/PMmjFzGHI8",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":275646909200,
+                     "reserved":80217120279,
+                     "uuid":"6f98d5d1-21a7-4173-85b6-c80906adf734",
+                     "failed":false,
+                     "volumeDataDensity":0.1633425,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/jcume3r8lk",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":300777645042,
+                     "reserved":45841745040,
+                     "uuid":"c3f62bab-1f81-4241-8373-f22f924587e7",
+                     "failed":false,
+                     "volumeDataDensity":-0.04317853,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"6d258c4f-c333-4a3f-9810-c5556df29d37",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/QWfemIkrwI",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":49558522968,
+                     "reserved":26130913925,
+                     "uuid":"512d099a-ae5a-44df-bb90-dc9d226d0abf",
+                     "failed":false,
+                     "volumeDataDensity":0.17725073,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/VIwFqkuFCq",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":268493331420,
+                     "reserved":11713259520,
+                     "uuid":"95ea89ce-a32d-40dc-9b7e-4ab7d143d440",
+                     "failed":false,
+                     "volumeDataDensity":-0.6215347,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/uttYxXnhhw",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":97056590451,
+                     "reserved":22260207060,
+                     "uuid":"678da188-465c-4cbd-9ab4-92776477869c",
+                     "failed":false,
+                     "volumeDataDensity":0.16660029,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"042659b0-0bf0-49ab-bca3-22f6d2e700cf",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"449ef856-38f0-419e-8138-d15133835aca",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.4551034,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/c80wDhoqGh",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":449335225772,
+                     "reserved":22938216636,
+                     "uuid":"269c0c4e-86e0-45c8-a52a-b3d6ae2dae4f",
+                     "failed":false,
+                     "volumeDataDensity":-0.17516565,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/cj5StzS5V2",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":363724832527,
+                     "reserved":31913771197,
+                     "uuid":"3b24a83f-1407-4665-916e-f1f977f2029b",
+                     "failed":false,
+                     "volumeDataDensity":-0.22143626,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/GWeIsbLm1e",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":40839250445,
+                     "reserved":31436488975,
+                     "uuid":"1f01cbf6-a3a3-44e5-bcfc-adbd410b59d7",
+                     "failed":false,
+                     "volumeDataDensity":0.6146494,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"4847938d-630f-46cd-9e8e-742fddea30d8",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/bBW7vSHyrn",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3317348521031,
+                     "reserved":122235967992,
+                     "uuid":"af9de85b-5d62-4eae-8cbb-21365a07ff35",
+                     "failed":false,
+                     "volumeDataDensity":0.014384538,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/4ohTec8Qiu",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":210184649282,
+                     "reserved":57538640983,
+                     "uuid":"d76aa819-a90d-4990-985b-f5516d540d6d",
+                     "failed":false,
+                     "volumeDataDensity":0.10833201,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/KwPNtnA1yy",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":565614891578,
+                     "reserved":121218975867,
+                     "uuid":"9c915fc1-198a-461e-ae59-7108be55cc55",
+                     "failed":false,
+                     "volumeDataDensity":-0.20814842,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"8fe26427-1fc6-4f34-911a-ef89abc06d42",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/JT3mOFaKEG",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":27736096123,
+                     "reserved":46261532686,
+                     "uuid":"bc6ddd16-1bad-4a63-a86d-067fb419cd82",
+                     "failed":false,
+                     "volumeDataDensity":0.052432247,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/qJyeuebPkR",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":74846421441,
+                     "reserved":9415260606,
+                     "uuid":"f70d342d-92bd-46c3-83bc-65ffa1aa9d73",
+                     "failed":false,
+                     "volumeDataDensity":0.004107997,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/W06QBHlHwD",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":69588697703,
+                     "reserved":28439196348,
+                     "uuid":"78326090-8e59-459b-8959-fee9d06294c9",
+                     "failed":false,
+                     "volumeDataDensity":-0.056446776,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"02cf61e9-c021-46d7-90e3-51a06e1e19fd",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"4d08e3c5-74c1-4297-9395-fe19ca6baca4",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7964681,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/MzDoNYEZ5f",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":4192631660947,
+                     "reserved":975708059129,
+                     "uuid":"aced6628-66f8-46a5-b24e-9f6d61d2c691",
+                     "failed":false,
+                     "volumeDataDensity":-0.041521966,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/kO6v1C9mhe",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":83219524150,
+                     "reserved":25554797432,
+                     "uuid":"c6b54a11-ceda-48f4-abc4-bdfb0be93a50",
+                     "failed":false,
+                     "volumeDataDensity":0.3512041,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/S6WguLgme3",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":4201515890020,
+                     "reserved":344964454560,
+                     "uuid":"e43b5d42-d12e-4cdd-97be-875e677f457a",
+                     "failed":false,
+                     "volumeDataDensity":0.023103476,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"c6a6ad1a-73e2-4c95-a580-11762520dc53",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/mP4iHDoQPK",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":509364731084,
+                     "reserved":74958969576,
+                     "uuid":"180a6169-ff74-4f22-b5a3-b2d609702618",
+                     "failed":false,
+                     "volumeDataDensity":0.2408207,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/5JOJR6pHgh",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":167698162421,
+                     "reserved":121874523980,
+                     "uuid":"2b2aa88e-8764-476c-8cef-6b29220528b0",
+                     "failed":false,
+                     "volumeDataDensity":0.5441642,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/m93KFrV7Gk",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3184214971925,
+                     "reserved":724494725957,
+                     "uuid":"a79b85f2-de3b-4b39-adda-c62d2f550c2b",
+                     "failed":false,
+                     "volumeDataDensity":-0.18066853,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"2823593e-c214-4cb0-97c3-56b15e74bc8a",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/49STDLcelQ",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":584953918067,
+                     "reserved":66867417697,
+                     "uuid":"f40ebc87-dc95-47d2-bd1f-34e8555469fd",
+                     "failed":false,
+                     "volumeDataDensity":-0.27385968,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/wNyyCiWMha",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":604432722023,
+                     "reserved":125463435671,
+                     "uuid":"0f679385-15fc-4e4d-99f1-f00b94b585b7",
+                     "failed":false,
+                     "volumeDataDensity":0.105810314,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/aBDA0rpKtG",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":331754945406,
+                     "reserved":155709084002,
+                     "uuid":"977fa62a-834a-49bb-b097-a402e9072302",
+                     "failed":false,
+                     "volumeDataDensity":0.035315007,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"67d84a86-ad7a-4b8c-a8c4-ee154a7d98bb",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"d6116168-d053-4acc-99cc-48737915b9f9",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.7443852,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/LDhXqq9avp",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":62064681937,
+                     "reserved":1603435543,
+                     "uuid":"328a7931-8ad7-41b1-b728-82be5e193819",
+                     "failed":false,
+                     "volumeDataDensity":-0.33158025,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/NePccO1YOk",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":273227982660,
+                     "reserved":82208484224,
+                     "uuid":"a2cf2744-a0ff-4f45-b325-945d908a9426",
+                     "failed":false,
+                     "volumeDataDensity":0.001478821,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/i5rj7OMwa4",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":199424907577,
+                     "reserved":128913529727,
+                     "uuid":"44ceb44a-11a8-4c99-927e-c20c1de12203",
+                     "failed":false,
+                     "volumeDataDensity":0.04055196,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"25dc010d-62f5-48d3-8367-73c4178ee303",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/2odvi0RIAi",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":513589215184,
+                     "reserved":24122765925,
+                     "uuid":"37ed1b66-6f51-404f-b507-5fed80d14d29",
+                     "failed":false,
+                     "volumeDataDensity":-0.46927056,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/wVo083vV6D",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":230309108738,
+                     "reserved":852583139804,
+                     "uuid":"88ef8ab1-008d-4660-a205-dbf5edd3a8db",
+                     "failed":false,
+                     "volumeDataDensity":0.25315043,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/YFgAlAT66Z",
+                     "capacity":2000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1815830034045,
+                     "reserved":15317961032,
+                     "uuid":"a23e8b85-d855-4e5b-94fb-68c887ee8416",
+                     "failed":false,
+                     "volumeDataDensity":-0.62430763,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"77973440-92d3-4858-80a1-184fe2d83b23",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/fPDzjBIFG3",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":12574238494,
+                     "reserved":31443366467,
+                     "uuid":"1fe5b028-5465-47f9-a65d-b2b5114004b3",
+                     "failed":false,
+                     "volumeDataDensity":0.3059501,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/lWLVUn5Sd6",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":652164260000,
+                     "reserved":478921696300,
+                     "uuid":"a8404874-0c93-4332-b68a-eb28c8f61dd8",
+                     "failed":false,
+                     "volumeDataDensity":0.121864974,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YNDR7c47HA",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":587808957290,
+                     "reserved":98217670080,
+                     "uuid":"e8b01266-982d-44c3-a9b4-65272c2ad21f",
+                     "failed":false,
+                     "volumeDataDensity":-0.5962304,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"4bb28d87-f325-4b1a-b396-e764d55e9d09",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"67e2153a-2813-443d-812c-df55f52fc963",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.6995974,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/9DCdZn2UkL",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":1818418310423,
+                     "reserved":704106064188,
+                     "uuid":"9a26df36-5819-4d1f-8a26-d1d1efd69a0f",
+                     "failed":false,
+                     "volumeDataDensity":0.008547723,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/N98Njpc1Hj",
+                     "capacity":200000000000,
+                     "storageType":"DISK",
+                     "used":148954875352,
+                     "reserved":14038972847,
+                     "uuid":"c7de8f97-bb0c-4037-b717-9d1f8641a05e",
+                     "failed":false,
+                     "volumeDataDensity":-0.24073046,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/airOE15KOs",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":149532092350,
+                     "reserved":3488975441,
+                     "uuid":"4a27d82f-6b00-43e3-8679-2c3c0112c200",
+                     "failed":false,
+                     "volumeDataDensity":0.05596471,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"43546ee9-ab9d-4080-bdd1-6617ef611bc7",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/QEONUzArcz",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":762041359532,
+                     "reserved":26918617908,
+                     "uuid":"dcc8ee3f-1f8b-4d9b-b7cf-8b76ad907b58",
+                     "failed":false,
+                     "volumeDataDensity":-0.15362144,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/EVKmDDKcJg",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":664222939767,
+                     "reserved":101123842966,
+                     "uuid":"d7061db3-ab4b-4472-b499-562f5c2e14d3",
+                     "failed":false,
+                     "volumeDataDensity":0.09314972,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/87nbOuxmHE",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":107449302703,
+                     "reserved":28768745825,
+                     "uuid":"ecb43d46-dc04-4907-9e82-b6151038c470",
+                     "failed":false,
+                     "volumeDataDensity":0.20458806,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"1d439795-3b66-462b-8861-c351c09dd313",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/7qvKhivgAW",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":964741472095,
+                     "reserved":277487175215,
+                     "uuid":"d9201b1c-a548-45c4-af07-0f020ebd9ff1",
+                     "failed":false,
+                     "volumeDataDensity":0.30467427,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/rX3EFIoP4q",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":543074403990,
+                     "reserved":78971919983,
+                     "uuid":"fbe22e5c-a29f-4da8-8947-f8d8c98813fd",
+                     "failed":false,
+                     "volumeDataDensity":-0.3655166,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/kAudnDSHzc",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":3472698207288,
+                     "reserved":557870357810,
+                     "uuid":"e0705837-782a-4d0e-81f6-9926fff72413",
+                     "failed":false,
+                     "volumeDataDensity":-0.27280432,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"487b0ca8-54c2-42c6-b5aa-9d18d4ce23cf",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"2cfa1b53-32be-4bdc-9477-e800ccd5aadb",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7047619,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/ufKGHovv6I",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":71023062800,
+                     "reserved":82072696440,
+                     "uuid":"894ecee8-bb9d-424e-b150-7b230c0c5184",
+                     "failed":false,
+                     "volumeDataDensity":0.5304306,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/5e0YqRQ9gu",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":154830031147,
+                     "reserved":71362026232,
+                     "uuid":"c58dfcc4-9d51-41bc-b804-81653a59cc8a",
+                     "failed":false,
+                     "volumeDataDensity":0.2292454,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/x9nILsFYTY",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":5295857871626,
+                     "reserved":862594325973,
+                     "uuid":"c6f813c8-0e8c-4080-afbc-1a52761a8184",
+                     "failed":false,
+                     "volumeDataDensity":-0.041614592,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"1b701d11-f2b1-4d29-8ffa-c2316eb47471",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/j9a3yuCtcK",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2160351689919,
+                     "reserved":485217130718,
+                     "uuid":"3946a8a3-b622-4866-830f-fd545544a519",
+                     "failed":false,
+                     "volumeDataDensity":0.06436604,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/ij9NvSb8TY",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":624302941381,
+                     "reserved":56229876833,
+                     "uuid":"b1e9f9dd-508a-4945-9da9-4f0871aa4056",
+                     "failed":false,
+                     "volumeDataDensity":-0.29074734,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/jvzXcXweeg",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3947929732302,
+                     "reserved":243305185110,
+                     "uuid":"2771d841-0c22-4771-8b7d-5aa2a82f741d",
+                     "failed":false,
+                     "volumeDataDensity":-0.006784737,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"45d98a9d-cbbb-44d4-97da-b5ddb2ca8b33",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/XlMDhDEQ6Q",
+                     "capacity":5000000000000,
+                     "storageType":"SSD",
+                     "used":942437861127,
+                     "reserved":213117753098,
+                     "uuid":"8afd0539-3dc7-421b-8eb4-12363126d66b",
+                     "failed":false,
+                     "volumeDataDensity":0.18780802,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/TcVPDROIZK",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":3404030905977,
+                     "reserved":656287538350,
+                     "uuid":"42a432e5-9bc5-4c29-828e-50ab7a440a32",
+                     "failed":false,
+                     "volumeDataDensity":-0.15191185,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/BcTNET2eXF",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":58576859094,
+                     "reserved":79617361711,
+                     "uuid":"4779f595-4f5f-4172-837b-d7f13fd2ca70",
+                     "failed":false,
+                     "volumeDataDensity":0.20185325,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"0438faf7-71e0-483d-89bf-f382f797cd1f",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"c3c0ce3d-cd92-4d2e-9ffe-af0cf8df38aa",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.599162,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/tuCCiEs88z",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":6032939764350,
+                     "reserved":805179832136,
+                     "uuid":"71ab9517-7212-478e-b8b2-afe11ac9e5e8",
+                     "failed":false,
+                     "volumeDataDensity":-0.15437216,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/g5etwPklc3",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":235458832286,
+                     "reserved":109114607839,
+                     "uuid":"59dd4636-f875-422e-a955-7b04639f20d0",
+                     "failed":false,
+                     "volumeDataDensity":0.5217807,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/y2UsAeSWX5",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":236830383610,
+                     "reserved":47622212240,
+                     "uuid":"594ee613-8ca9-451e-af4d-c5c008bb7202",
+                     "failed":false,
+                     "volumeDataDensity":0.5708235,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"cce5f7f8-81e2-49fb-9ff7-e44f0fc06947",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/JLg2oLqJ7u",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":217426502155,
+                     "reserved":1882589536,
+                     "uuid":"2b42119a-1143-4427-bbc8-579e5cef8476",
+                     "failed":false,
+                     "volumeDataDensity":0.0017999113,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/8Zmxe2I20k",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":252478994526,
+                     "reserved":78184827644,
+                     "uuid":"0e1003f5-b246-4bfc-8f0d-10606d5c8893",
+                     "failed":false,
+                     "volumeDataDensity":-0.036536634,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Oiqmbormri",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":239166821613,
+                     "reserved":56310799226,
+                     "uuid":"9c00ca57-1ab2-409a-aa53-17772efeb4c3",
+                     "failed":false,
+                     "volumeDataDensity":0.02976942,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"75f7fb25-505e-4120-85bb-4b53b870de5a",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Ta0BhtykpQ",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":103809404322,
+                     "reserved":29689615732,
+                     "uuid":"e89d6f8f-2462-4060-9bbc-bf8b46717cc0",
+                     "failed":false,
+                     "volumeDataDensity":0.51419723,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/TDh9yqFPWC",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":8393719094,
+                     "reserved":18316304819,
+                     "uuid":"7790a093-f2b6-4c58-b7b9-7d0cf5bbfa8e",
+                     "failed":false,
+                     "volumeDataDensity":0.6394402,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Xl0RiELDid",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":5735320626859,
+                     "reserved":641390581171,
+                     "uuid":"0dc8e0c7-3a2c-4fb2-a2ab-1d4da3b8fbac",
+                     "failed":false,
+                     "volumeDataDensity":-0.13044244,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"9208cf2b-731c-4f60-bef6-5e311f6a6025",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"522d4e99-0717-487c-b128-a08c8862b356",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.3865048,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/5VbzmpdSGx",
+                     "capacity":200000000000,
+                     "storageType":"DISK",
+                     "used":159804004561,
+                     "reserved":7103293568,
+                     "uuid":"482360d2-b1e6-4560-a3cf-7f61740c7c6c",
+                     "failed":false,
+                     "volumeDataDensity":-0.30313277,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tsKwA2VsTG",
+                     "capacity":600000000000,
+                     "storageType":"DISK",
+                     "used":357096046825,
+                     "reserved":23060392003,
+                     "uuid":"f6959029-5a34-4a3d-a599-c14337cfeaa2",
+                     "failed":false,
+                     "volumeDataDensity":-0.09363818,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/v0N3DwUGl3",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":300260161969,
+                     "reserved":14260951227,
+                     "uuid":"8969bab2-40da-4003-bb98-fc791de8c6c1",
+                     "failed":false,
+                     "volumeDataDensity":0.1431733,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"be603be0-6dc1-446f-9323-1e1d7361e5a4",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/az4bNaCUg6",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":274281000008,
+                     "reserved":4037653256,
+                     "uuid":"b0a0e0dc-821b-436f-ba2b-eed1bb9af9b9",
+                     "failed":false,
+                     "volumeDataDensity":-0.19002163,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/qvfSz4qi1T",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":5974186792693,
+                     "reserved":373462404122,
+                     "uuid":"13ce842c-6400-4a3b-9c54-187177efe0b9",
+                     "failed":false,
+                     "volumeDataDensity":-0.16483355,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/8MoQxKUeOg",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1630407732325,
+                     "reserved":227987322727,
+                     "uuid":"014dfaf8-ca7f-4fc9-bc63-2143d29308ac",
+                     "failed":false,
+                     "volumeDataDensity":0.30448312,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"6b6895e1-7ca6-45db-9e51-f1ff9ea91c55",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/tvTL9bG41x",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":1795850671087,
+                     "reserved":294667635452,
+                     "uuid":"63e59f0f-425e-414d-b9c3-b01afbd0155a",
+                     "failed":false,
+                     "volumeDataDensity":0.07330987,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/GcsUGfUNl8",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":2214957228606,
+                     "reserved":361800185279,
+                     "uuid":"43b77f16-d594-4ce4-8605-b7d1edd82d16",
+                     "failed":false,
+                     "volumeDataDensity":-0.11324525,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/9xSZ57lSsQ",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":130882362943,
+                     "reserved":30780073246,
+                     "uuid":"320c808a-4fe1-4444-a29c-c4236ab90b39",
+                     "failed":false,
+                     "volumeDataDensity":6.66976E-4,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"65565c14-d262-405b-b371-7367f2743c52",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"c34a3d22-9520-4fb0-aec6-479ef5727cc6",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.9818475,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/K69VDEMa8d",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":260298870112,
+                     "reserved":20975505444,
+                     "uuid":"fcc01f9a-f6b7-4941-82f0-218ac9e3d455",
+                     "failed":false,
+                     "volumeDataDensity":-0.36487025,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/T1PPBHLDVY",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":512101497544,
+                     "reserved":39755350344,
+                     "uuid":"e2e13d6b-66b1-4d84-b190-2252c18eddaa",
+                     "failed":false,
+                     "volumeDataDensity":-0.20760512,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DtARJCRe1R",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":178299248060,
+                     "reserved":165557321397,
+                     "uuid":"d68040c5-854a-44ae-9c66-73899c12ab3e",
+                     "failed":false,
+                     "volumeDataDensity":0.3252506,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"b758abd8-149e-4e7b-bb7d-d6dffa603bdf",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Smmv1LDgv1",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":113035998410,
+                     "reserved":123896782036,
+                     "uuid":"25047e37-7766-42a9-93ce-6d3b6ac18028",
+                     "failed":false,
+                     "volumeDataDensity":-0.0883665,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/pqfgtRlxaS",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":95362068485,
+                     "reserved":3787535477,
+                     "uuid":"fc6f7057-7f21-4c04-9b2a-aed10f0e688b",
+                     "failed":false,
+                     "volumeDataDensity":-0.93388224,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/IXV3tTITdY",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":278105766766,
+                     "reserved":378745040245,
+                     "uuid":"f188d1ab-bae3-46fb-b700-4e42f8d036b6",
+                     "failed":false,
+                     "volumeDataDensity":0.020788264,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"8ff7470a-8cbe-439d-944b-d7de161a130d",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/yuTYoiXDJL",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":89932728357,
+                     "reserved":5857136285,
+                     "uuid":"ea341376-31f7-4d72-8cd0-51467a84f95c",
+                     "failed":false,
+                     "volumeDataDensity":-0.69036305,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xUHOVmABmV",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":424027550008,
+                     "reserved":28076605943,
+                     "uuid":"aa926450-ea1d-481c-a508-4030e2d2b777",
+                     "failed":false,
+                     "volumeDataDensity":-0.22139671,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/s2ERPLdkkI",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":270536780138,
+                     "reserved":4763104814,
+                     "uuid":"65f14973-769a-4558-8c55-b660b2552d0c",
+                     "failed":false,
+                     "volumeDataDensity":0.12932484,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"305b082e-7a26-4a32-ac1f-286d51bbaf2c",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"c81839d8-f6de-44a9-8701-bf5a85f3ab48",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.4677161,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/F25ToDzgYW",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":3114088543292,
+                     "reserved":822217688386,
+                     "uuid":"3f1f84ce-a198-4e34-a5e5-ab22e0d02491",
+                     "failed":false,
+                     "volumeDataDensity":-0.110342205,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hMREDyUSBF",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":814399520094,
+                     "reserved":439774638431,
+                     "uuid":"e39cbabc-44e7-4f5d-8320-7589139e0385",
+                     "failed":false,
+                     "volumeDataDensity":0.16498706,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/c2hLGDUouh",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":81874121858,
+                     "reserved":52611376971,
+                     "uuid":"b12bf5bb-285d-4bf5-93fe-183c45b398c6",
+                     "failed":false,
+                     "volumeDataDensity":0.210732,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"e744e034-8563-4ebe-9a9a-ab34e62b32be",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/RhPI39QUar",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":434373574937,
+                     "reserved":176752713646,
+                     "uuid":"0b9d78d0-7b78-4cc1-a3b8-fdecb86e29d7",
+                     "failed":false,
+                     "volumeDataDensity":-0.15777981,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/4jeOzk3Zep",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":258834580109,
+                     "reserved":72021820618,
+                     "uuid":"ef3175af-2879-479e-a57d-f3167033a7f9",
+                     "failed":false,
+                     "volumeDataDensity":0.014301896,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/6CCmM3jA3s",
+                     "capacity":900000000000,
+                     "storageType":"RAM_DISK",
+                     "used":208180545886,
+                     "reserved":14081742247,
+                     "uuid":"4a547d50-d771-4359-aaf7-d03ed2b2c4ee",
+                     "failed":false,
+                     "volumeDataDensity":0.13486613,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"508f395b-ff1a-4b0b-bd5e-fc97dd200a62",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/sEgihzTtlc",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1202775683633,
+                     "reserved":344252891515,
+                     "uuid":"95837d7c-790e-4fe7-a76b-b35ef938ff24",
+                     "failed":false,
+                     "volumeDataDensity":-0.022966683,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Ereit6r54D",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":3005724981337,
+                     "reserved":691216155935,
+                     "uuid":"762572d9-bf54-485d-ae28-db42a8038f12",
+                     "failed":false,
+                     "volumeDataDensity":-0.20494968,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/v5w1o8qknl",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":411412854276,
+                     "reserved":397097904340,
+                     "uuid":"ee14476d-8dc2-4bc1-af14-b431b351805d",
+                     "failed":false,
+                     "volumeDataDensity":0.44679058,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"3ee1531f-9d5e-4ac8-b677-d47a80b08e81",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"cf0e2c34-3416-45ce-af7a-e372de399488",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.2987883,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/9ob13dXtAg",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":409132414024,
+                     "reserved":33858765897,
+                     "uuid":"11986bee-0abf-4173-bf82-0a340a82603c",
+                     "failed":false,
+                     "volumeDataDensity":0.20765616,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/lI2vWDRvjF",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":534218417475,
+                     "reserved":159537203196,
+                     "uuid":"dab6b97d-ac8b-41b4-9f10-107419fafcc4",
+                     "failed":false,
+                     "volumeDataDensity":-0.52330065,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DLDbppmAwf",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":2801589026950,
+                     "reserved":557730912485,
+                     "uuid":"a61fa90e-3608-4bb2-9f57-bfe8951d77fa",
+                     "failed":false,
+                     "volumeDataDensity":-0.065630406,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"f26da74b-973b-464c-9a9d-f04db0a3f7a8",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/GAbWa6pSkG",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":210732797181,
+                     "reserved":87264796087,
+                     "uuid":"fc833cfc-eecf-4d5c-b446-4f98f32b2a5f",
+                     "failed":false,
+                     "volumeDataDensity":-0.06694296,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/dza2Hk3t7O",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":991253520105,
+                     "reserved":456690483601,
+                     "uuid":"68f41a1d-7439-4e8a-99dc-16b527078fa9",
+                     "failed":false,
+                     "volumeDataDensity":-0.0027750432,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/DQsZ61ta9B",
+                     "capacity":600000000000,
+                     "storageType":"RAM_DISK",
+                     "used":95525746639,
+                     "reserved":71522637197,
+                     "uuid":"d9dfa004-2896-4c34-8eda-3afa38bb7326",
+                     "failed":false,
+                     "volumeDataDensity":0.09622197,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"d1c35f4e-d0d3-44bf-8b5d-e4cc390aa5e3",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/lJE7eEK40D",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":1004481119081,
+                     "reserved":923093918329,
+                     "uuid":"b7854c2e-e0f0-41a5-845f-7e81eb85ae3f",
+                     "failed":false,
+                     "volumeDataDensity":0.09972711,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/az6T2YAENO",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1023544354371,
+                     "reserved":11063144767,
+                     "uuid":"6452d970-6bae-4537-be9a-392b4c4a20e0",
+                     "failed":false,
+                     "volumeDataDensity":-0.21703869,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/6BOY4sMWWD",
+                     "capacity":4000000000000,
+                     "storageType":"SSD",
+                     "used":1213784520502,
+                     "reserved":171936147976,
+                     "uuid":"df5c5794-b4a9-4b2c-bd77-6cde1e705344",
+                     "failed":false,
+                     "volumeDataDensity":-0.01949519,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"8568ce8c-324d-4778-8fd4-6f77eb300c40",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"6457c582-814c-4ff6-bfe2-ee5b4c5e1dad",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7780728,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/I7OLXCZKDy",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":233181578473,
+                     "reserved":165386762044,
+                     "uuid":"88333a6e-b359-4f16-800d-dcde68ec14a1",
+                     "failed":false,
+                     "volumeDataDensity":-0.056648225,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/wIV4v52QWW",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":149437167989,
+                     "reserved":64349156920,
+                     "uuid":"02ca2fd1-687b-4769-84f8-431813f2c08a",
+                     "failed":false,
+                     "volumeDataDensity":-0.18444344,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/uBMs6EG2Gg",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":93005137592,
+                     "reserved":46361850300,
+                     "uuid":"00b77184-b4d5-493f-9964-2ea0b9a4d223",
+                     "failed":false,
+                     "volumeDataDensity":0.13736448,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"4e847819-2997-42eb-9368-1f683e8c8a9f",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/p1monmQKu7",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4322395683324,
+                     "reserved":170881256787,
+                     "uuid":"28aace5b-12f4-4859-90c5-773240779d6d",
+                     "failed":false,
+                     "volumeDataDensity":-0.34049642,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/nxgXPaPPp2",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1349285575709,
+                     "reserved":1595406216849,
+                     "uuid":"50c5b554-b520-4d22-b65c-4489359bad36",
+                     "failed":false,
+                     "volumeDataDensity":0.37235004,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/g3jsLXie5Y",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4889789468751,
+                     "reserved":189378284553,
+                     "uuid":"d8b03fb0-1298-451d-b048-9e311cf6a886",
+                     "failed":false,
+                     "volumeDataDensity":-0.16339236,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"fc42724f-b7c5-402d-844b-7cd121d8c359",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/sTxtWwCZsZ",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":1939860994876,
+                     "reserved":22699443597,
+                     "uuid":"de09cc1e-0274-4df9-8420-7dc297962e26",
+                     "failed":false,
+                     "volumeDataDensity":0.06748012,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/bP8giZ8SXb",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":247855163640,
+                     "reserved":83000493159,
+                     "uuid":"2b6942c6-8cfa-4112-afb5-48674231ed3a",
+                     "failed":false,
+                     "volumeDataDensity":-0.1958456,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ytWWsvNvnr",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1054692596098,
+                     "reserved":59862465022,
+                     "uuid":"979122f3-d479-4c13-a196-5e3b99cfe37f",
+                     "failed":false,
+                     "volumeDataDensity":-0.2600522,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"23838651-b500-4721-9e3d-62c078f6d424",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"24c3c54d-09e4-4193-b9b5-e6418edece63",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.842497,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/xjDyS3PsrS",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":5005382638550,
+                     "reserved":238293662562,
+                     "uuid":"8562de1a-8c92-4371-85d2-4ab1bfce1c85",
+                     "failed":false,
+                     "volumeDataDensity":-0.14782047,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/NnUujdpBfY",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":887176380489,
+                     "reserved":110357025019,
+                     "uuid":"0ba005ed-d4b8-4ab8-a906-990e13e6b2a2",
+                     "failed":false,
+                     "volumeDataDensity":0.41099402,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DXrrSPVUyA",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":2633260765991,
+                     "reserved":260175845115,
+                     "uuid":"dfa492ac-d73b-4b5c-b15b-359290d639a5",
+                     "failed":false,
+                     "volumeDataDensity":-0.36867183,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"8954f4b5-2c25-429a-92a8-6dc837cee6ad",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/9j5FeE4EgZ",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":265833173139,
+                     "reserved":79479355923,
+                     "uuid":"0fd7d684-a7e7-44db-9b1b-f2c32f5a1de5",
+                     "failed":false,
+                     "volumeDataDensity":-0.47464928,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/6glY2hyr1K",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1219164804409,
+                     "reserved":710345983850,
+                     "uuid":"0faa4c57-0b52-4e64-8ca1-44d78e8375c9",
+                     "failed":false,
+                     "volumeDataDensity":0.010432512,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/mLaOXvQvNt",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":598936233861,
+                     "reserved":479111834057,
+                     "uuid":"22361e14-20ec-49e2-ac39-f407befda8e3",
+                     "failed":false,
+                     "volumeDataDensity":0.025021166,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"232bde58-63c2-4c63-a291-06a6cc06fbb8",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/mJTq6LrwOM",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":5055573510899,
+                     "reserved":190420088997,
+                     "uuid":"cb8a1b6b-9820-4900-9db7-9c58323568a8",
+                     "failed":false,
+                     "volumeDataDensity":-0.056824803,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/l5Ggf7AKGt",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":1228278251722,
+                     "reserved":475259459618,
+                     "uuid":"1ed9b6f5-4784-4535-ab24-7fa766684e34",
+                     "failed":false,
+                     "volumeDataDensity":0.19909912,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hfv5x3inHq",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":648239895813,
+                     "reserved":23273885077,
+                     "uuid":"62d73599-237b-4169-ab98-e15af3ab015b",
+                     "failed":false,
+                     "volumeDataDensity":-0.14898384,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"87c5cdf0-94f5-4aa7-9c3f-9c280867c44f",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"ca028312-4cb5-4996-a78c-ea5f6a895355",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.68413,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/LvDh0avZJK",
+                     "capacity":8000000000000,
+                     "storageType":"DISK",
+                     "used":2533772562644,
+                     "reserved":849498499125,
+                     "uuid":"b981554f-394f-4ccb-9b67-ab1f8c6043e6",
+                     "failed":false,
+                     "volumeDataDensity":-0.053794026,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/O3lh7XIsk3",
+                     "capacity":9000000000000,
+                     "storageType":"DISK",
+                     "used":2368768883927,
+                     "reserved":737681387671,
+                     "uuid":"009fe6e4-7da7-4559-802a-3ef043f14c5e",
+                     "failed":false,
+                     "volumeDataDensity":0.013859481,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/dXWl6Hgv8I",
+                     "capacity":1000000000000,
+                     "storageType":"DISK",
+                     "used":515508953,
+                     "reserved":99470843074,
+                     "uuid":"e706596f-4cf7-4474-9938-f9a766ad9b96",
+                     "failed":false,
+                     "volumeDataDensity":0.29998246,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"18e8fda4-737e-4256-bdab-23b19938af6b",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/cy6VjRg5Ak",
+                     "capacity":700000000000,
+                     "storageType":"RAM_DISK",
+                     "used":408511709341,
+                     "reserved":130764628389,
+                     "uuid":"e3536d89-8f42-42a8-be5b-3f63efae0870",
+                     "failed":false,
+                     "volumeDataDensity":-0.16029501,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/N9fpinXN4L",
+                     "capacity":200000000000,
+                     "storageType":"RAM_DISK",
+                     "used":75650181968,
+                     "reserved":39816290541,
+                     "uuid":"a87fef59-9e13-4b6f-bc84-bcf50c10d3fd",
+                     "failed":false,
+                     "volumeDataDensity":0.085083574,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/ICxD0fXA51",
+                     "capacity":4000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1988808064896,
+                     "reserved":292443533732,
+                     "uuid":"e02810d7-4cfc-4fa1-a6ba-26425051d9bb",
+                     "failed":false,
+                     "volumeDataDensity":0.02093476,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"3ec5d241-44f6-4a03-954a-bcb9d2b38941",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/giwmLFM2mq",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":5680666726426,
+                     "reserved":318600401994,
+                     "uuid":"a1683187-913a-48ce-9987-3ece357fa9d5",
+                     "failed":false,
+                     "volumeDataDensity":-0.025424957,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/xp3fORdQNt",
+                     "capacity":200000000000,
+                     "storageType":"SSD",
+                     "used":18409290753,
+                     "reserved":5182692440,
+                     "uuid":"91c0d262-0b77-43bc-b6c9-c08bff27e632",
+                     "failed":false,
+                     "volumeDataDensity":0.73030096,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8kvQqPzwvw",
+                     "capacity":100000000000,
+                     "storageType":"SSD",
+                     "used":49708472895,
+                     "reserved":6270793774,
+                     "uuid":"40639b3c-3f1a-4ea9-9187-cfc6bfe8b404",
+                     "failed":false,
+                     "volumeDataDensity":0.29445475,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"4b1a3ed6-955a-4f39-b73f-81db4ae5672c",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"9be46dfb-4f36-4384-9ea0-cfd32cc1cdf1",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.8776151,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/1UZLeUtyB3",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":531394514291,
+                     "reserved":159020823078,
+                     "uuid":"49f6159e-ccee-4a32-a530-92a5c9112abc",
+                     "failed":false,
+                     "volumeDataDensity":0.09939708,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/LOFBMQ8MJI",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":407397503460,
+                     "reserved":106978159815,
+                     "uuid":"a1618737-e4b3-419d-8fa4-224ef9e789f7",
+                     "failed":false,
+                     "volumeDataDensity":-0.40054232,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/0I6ge4OWxQ",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":130309987837,
+                     "reserved":1667991000,
+                     "uuid":"ed49da77-536a-49a6-85e7-22d023c48b42",
+                     "failed":false,
+                     "volumeDataDensity":-0.15035185,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"8e58b503-f5e4-4412-abcc-2bcf1df0620b",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/aDp2GXRxTZ",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":46924325814,
+                     "reserved":11247226994,
+                     "uuid":"2e1343b8-b1fe-4faa-a1db-6f7b76d2f3b5",
+                     "failed":false,
+                     "volumeDataDensity":-0.012383401,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/NcNRhgKrgT",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":839571985989,
+                     "reserved":237608243343,
+                     "uuid":"e139dd60-85c7-4b6a-bf3c-0be785b23902",
+                     "failed":false,
+                     "volumeDataDensity":0.21239549,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/fuUBGTyCum",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3146483856784,
+                     "reserved":40209381618,
+                     "uuid":"cc30e235-096a-46fb-a1e8-90663120bbe0",
+                     "failed":false,
+                     "volumeDataDensity":-0.11807364,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"3c301ebb-3376-4e52-8716-8926ae162acf",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/znqn6nQeMJ",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":276594028362,
+                     "reserved":16150769691,
+                     "uuid":"0bfcd978-de0d-46cf-a0de-a83dd7250afb",
+                     "failed":false,
+                     "volumeDataDensity":-0.46191937,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/0WD1qTCWa6",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":2648356591029,
+                     "reserved":1184824986492,
+                     "uuid":"22ffe225-907c-44c6-9289-19acf87e244d",
+                     "failed":false,
+                     "volumeDataDensity":0.057098955,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/DJFYOFU8lT",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":482707281228,
+                     "reserved":50202994124,
+                     "uuid":"c4035d77-cdec-4b94-966d-b1db6536a8a2",
+                     "failed":false,
+                     "volumeDataDensity":-0.365453,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"5fd97c37-3dde-486f-92a7-e0604af98301",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"2826e3cc-5492-4bd7-99ff-036375f4b5fc",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.4974133,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/DRDVJX4QOQ",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":1420719370401,
+                     "reserved":321799004704,
+                     "uuid":"57d98771-dc1d-4825-9866-c87867f7c937",
+                     "failed":false,
+                     "volumeDataDensity":-0.39067495,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/jfJLjC7dDe",
+                     "capacity":400000000000,
+                     "storageType":"DISK",
+                     "used":7374370267,
+                     "reserved":26886972977,
+                     "uuid":"4198c7c6-20a7-4483-adb4-a6205eb941e4",
+                     "failed":false,
+                     "volumeDataDensity":0.43613347,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/YErtqEmQec",
+                     "capacity":5000000000000,
+                     "storageType":"DISK",
+                     "used":1688796394304,
+                     "reserved":214498206041,
+                     "uuid":"397be114-292a-48e0-9204-464ecdeefb0e",
+                     "failed":false,
+                     "volumeDataDensity":0.10299942,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"fcce00d3-9bcf-49a8-b763-8f567ddb38f3",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Ak8DhXDRHY",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3917069106831,
+                     "reserved":673016988007,
+                     "uuid":"5b85031f-ee9d-49d6-8994-02907b86460a",
+                     "failed":false,
+                     "volumeDataDensity":-0.05019337,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/25H4EpzjLL",
+                     "capacity":3000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1694325399477,
+                     "reserved":203062034276,
+                     "uuid":"0f1a8b91-7574-46be-bcd2-b9c49b8eb0aa",
+                     "failed":false,
+                     "volumeDataDensity":0.07935393,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/6vWkYKOPM6",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":231936704406,
+                     "reserved":95161972117,
+                     "uuid":"e0ffa8ee-ebaa-4635-bf01-41adcc1bf34b",
+                     "failed":false,
+                     "volumeDataDensity":0.11222029,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"bbd8f9ca-4e5d-4e92-becc-a4fc633d280c",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/KqFrZqAfTg",
+                     "capacity":600000000000,
+                     "storageType":"SSD",
+                     "used":3799750945,
+                     "reserved":72807198306,
+                     "uuid":"10a8b4c3-9375-4248-b740-c6c3139434c9",
+                     "failed":false,
+                     "volumeDataDensity":0.11730838,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UM4jBTDMLz",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":195465513549,
+                     "reserved":69821202357,
+                     "uuid":"586cb3c8-8842-4a24-a44c-5c43909000f1",
+                     "failed":false,
+                     "volumeDataDensity":-0.18565881,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/VZshgmgc3a",
+                     "capacity":3000000000000,
+                     "storageType":"SSD",
+                     "used":245125610146,
+                     "reserved":588422984947,
+                     "uuid":"5510e7bc-f73e-4e8b-8dc5-ae5106a2028b",
+                     "failed":false,
+                     "volumeDataDensity":0.022870533,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"ca0ff029-f7c2-4d0b-bff5-684b2f2113ad",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"f6ae3791-e71e-434d-875d-7fddd9d4154f",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.7024343,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/4q6FTP6nKJ",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":389332413827,
+                     "reserved":242484711410,
+                     "uuid":"040a7e60-2e8d-40d0-93e5-9250446e0a90",
+                     "failed":false,
+                     "volumeDataDensity":-0.054613054,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/H0L1xNRXBC",
+                     "capacity":700000000000,
+                     "storageType":"DISK",
+                     "used":13743559292,
+                     "reserved":136258748265,
+                     "uuid":"0a68944a-4115-4d38-92fd-46dcd13be0f5",
+                     "failed":false,
+                     "volumeDataDensity":0.14253208,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ZBAqTHylyv",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":31033688837,
+                     "reserved":20415432978,
+                     "uuid":"d94ad7ae-1c70-42e6-ad8a-43f66bb92f94",
+                     "failed":false,
+                     "volumeDataDensity":0.055911988,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"4618796f-bbaf-4451-87b0-ba6dd6941057",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/WO8gx5MyoU",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":55353661446,
+                     "reserved":17362704277,
+                     "uuid":"606a3bcd-5d15-4b56-a18f-d3f6d75f7bb2",
+                     "failed":false,
+                     "volumeDataDensity":0.08374053,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/9wmJvTzkIC",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":97322556843,
+                     "reserved":57718241541,
+                     "uuid":"a8fdcf6c-cc63-4828-ad71-2a5b92987777",
+                     "failed":false,
+                     "volumeDataDensity":0.53353274,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/2qTIRk4oag",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4807711074923,
+                     "reserved":942482439683,
+                     "uuid":"e8a88733-8d04-4ada-b3be-9fbb6347dedd",
+                     "failed":false,
+                     "volumeDataDensity":-0.040097475,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"de80a80b-d684-4eda-b238-3c67a0e7e5b1",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/V8V1RtxJiU",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":146863802780,
+                     "reserved":2417671565,
+                     "uuid":"f62da453-475d-4b77-9cd6-54ab10959e5c",
+                     "failed":false,
+                     "volumeDataDensity":0.09437369,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/UFnRgnNwgJ",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":82737313810,
+                     "reserved":102784307064,
+                     "uuid":"7acbeee3-d0bc-4a1d-b0b7-e97b17a2b28a",
+                     "failed":false,
+                     "volumeDataDensity":0.18623808,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/E0DDCRktoO",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":312351740354,
+                     "reserved":17357147345,
+                     "uuid":"eb76d51e-3ebf-411a-8d73-3a30fc7b4039",
+                     "failed":false,
+                     "volumeDataDensity":-0.5113948,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"3368df8d-69ab-4bd5-ae65-49bf5470120b",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"9c42aaf3-d260-44a9-a0a5-62a902f18ec4",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.0329072,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/0d1xYdTU2D",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":379893439823,
+                     "reserved":20183585344,
+                     "uuid":"2ab97270-7928-4e70-8bec-17562e1fa64f",
+                     "failed":false,
+                     "volumeDataDensity":-0.49135858,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/s8JtfocM8C",
+                     "capacity":9000000000000,
+                     "storageType":"DISK",
+                     "used":2190226517897,
+                     "reserved":912517943740,
+                     "uuid":"85e090dc-d05f-4268-88d6-24182e2c9d1b",
+                     "failed":false,
+                     "volumeDataDensity":0.02957207,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/3dEJOQ1FxQ",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":29301391279,
+                     "reserved":13778914918,
+                     "uuid":"f57ea963-b1ac-4763-a8dd-768fdae79d57",
+                     "failed":false,
+                     "volumeDataDensity":-0.03945127,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"dee2275d-b6bf-43b1-9281-407975f95240",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Ui1DFe6y1N",
+                     "capacity":8000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":5335635574616,
+                     "reserved":977066691472,
+                     "uuid":"c76552d2-f2ff-43fa-af86-9957e938d465",
+                     "failed":false,
+                     "volumeDataDensity":-0.015427172,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/MM3Mw1ADOP",
+                     "capacity":100000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1643943602,
+                     "reserved":10574686255,
+                     "uuid":"8294f85f-e875-4a75-8824-6f8ce46a306a",
+                     "failed":false,
+                     "volumeDataDensity":0.72593397,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/hTh2VQ4sU1",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":276538678658,
+                     "reserved":70122312076,
+                     "uuid":"70ae52b5-25b1-4cf8-be60-089dd8a528ff",
+                     "failed":false,
+                     "volumeDataDensity":0.10102123,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"77c7cc55-1580-4e73-8f6a-91305b9792a1",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/38ubWsTIDF",
+                     "capacity":9000000000000,
+                     "storageType":"SSD",
+                     "used":4741955392627,
+                     "reserved":1176832690107,
+                     "uuid":"5d9911d1-5a23-428e-81e7-c5a80e8829da",
+                     "failed":false,
+                     "volumeDataDensity":-0.08912867,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/XVPRUly9PN",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":4919637716231,
+                     "reserved":1310443448135,
+                     "uuid":"29d3da97-3cbe-45cb-9a63-db63b9a58230",
+                     "failed":false,
+                     "volumeDataDensity":-0.21840662,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/CVDe50cyS7",
+                     "capacity":7000000000000,
+                     "storageType":"SSD",
+                     "used":1300619941121,
+                     "reserved":309790355249,
+                     "uuid":"1364a771-081d-4884-971c-b8dbc5a9a843",
+                     "failed":false,
+                     "volumeDataDensity":0.32260752,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"8b7058c3-d06e-4bd4-91cd-8562ded78029",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"a96d7301-fa41-4ab2-be09-f60e28387588",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":0.99035704,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/hzxV5xKCDS",
+                     "capacity":900000000000,
+                     "storageType":"DISK",
+                     "used":566897512614,
+                     "reserved":88543988273,
+                     "uuid":"f88efc3f-edb5-4da3-a2df-2863f9cbf816",
+                     "failed":false,
+                     "volumeDataDensity":0.056499183,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/hm8BlrgYwD",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":4070126678672,
+                     "reserved":1082313141593,
+                     "uuid":"236379e5-22aa-429e-b406-66a8f3515c78",
+                     "failed":false,
+                     "volumeDataDensity":-0.07253379,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/eHZ3SwY7zw",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":218315058442,
+                     "reserved":99224898672,
+                     "uuid":"76ef030c-bd1a-4565-96ca-70339a94ce28",
+                     "failed":false,
+                     "volumeDataDensity":0.44358316,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"681e20e4-2d9c-4122-b5e9-655447056b63",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/UMYl86r3ll",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":756822018736,
+                     "reserved":105953370475,
+                     "uuid":"3560b5b3-5815-4121-aa56-a3291a331c1b",
+                     "failed":false,
+                     "volumeDataDensity":-0.022150815,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/44DsBicamM",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":271986547991,
+                     "reserved":94802356791,
+                     "uuid":"a799d200-147b-49d7-8724-8b2b9da434f1",
+                     "failed":false,
+                     "volumeDataDensity":0.15311801,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/WDKdGPynpl",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4701128662774,
+                     "reserved":1348491936994,
+                     "uuid":"9c5c1243-4859-4d3d-a4b0-afe768084caf",
+                     "failed":false,
+                     "volumeDataDensity":-0.0074739456,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"1b3c8218-b2e3-4338-928d-8cff6f94e44a",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/RSw0VgA6L3",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":202661307624,
+                     "reserved":109672974573,
+                     "uuid":"fd279e69-1577-4870-bdd0-8139d56dcd84",
+                     "failed":false,
+                     "volumeDataDensity":0.14846039,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8aDMcLsS7C",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":114310170796,
+                     "reserved":27042635746,
+                     "uuid":"1abb8f2f-9577-49da-97d4-6852dbb95334",
+                     "failed":false,
+                     "volumeDataDensity":0.07297984,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/8TeyDIMcM2",
+                     "capacity":8000000000000,
+                     "storageType":"SSD",
+                     "used":4008914040757,
+                     "reserved":66611280530,
+                     "uuid":"7c7d5059-cdc5-4ea5-b15a-72beddb8bcdc",
+                     "failed":false,
+                     "volumeDataDensity":-0.013557911,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"cf5f3976-126b-4d2c-9170-8db26247c7e1",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"cc7b040b-3166-44f1-92c7-4a3cc66a38c8",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.4673262,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/c3BeVN6YKR",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":459682064993,
+                     "reserved":321238192533,
+                     "uuid":"04552f91-404f-46fc-aa1d-b5a9db4c25e5",
+                     "failed":false,
+                     "volumeDataDensity":0.098892145,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/aoNdcmhUrm",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":1196422348978,
+                     "reserved":337939144550,
+                     "uuid":"b8d00f27-8925-4554-9b5d-c62582fc1717",
+                     "failed":false,
+                     "volumeDataDensity":-0.14686763,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/PtA5if0NOS",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":39760668531,
+                     "reserved":10954042131,
+                     "uuid":"9922b91c-f09f-4755-86dc-b39c4932cd26",
+                     "failed":false,
+                     "volumeDataDensity":-0.26667875,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"9b24ff48-9047-4c6c-83d0-20d73cb00330",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/Rqsi4sf6CF",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":475960164387,
+                     "reserved":136217238691,
+                     "uuid":"794d3f0a-78c8-4193-82c7-a52f910e6aa8",
+                     "failed":false,
+                     "volumeDataDensity":-0.0062925816,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/2uneKEZcoH",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":2955898141467,
+                     "reserved":484073474089,
+                     "uuid":"5ab13ac7-f579-4ea1-8075-67a42c838209",
+                     "failed":false,
+                     "volumeDataDensity":-0.10982394,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/xYqHX5DVg0",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":40582226815,
+                     "reserved":5051153944,
+                     "uuid":"49f42fec-12ab-4cbd-b78e-3da0b483b80d",
+                     "failed":false,
+                     "volumeDataDensity":0.50393754,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"a5db0ca5-4103-4b2d-8a88-77ca8484da87",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/BWknznWbvx",
+                     "capacity":800000000000,
+                     "storageType":"SSD",
+                     "used":550430902514,
+                     "reserved":153268555535,
+                     "uuid":"1f3a7078-52f5-4284-828f-08cee5591c0f",
+                     "failed":false,
+                     "volumeDataDensity":-0.053911865,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/wkDwU8X0z0",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":418582737735,
+                     "reserved":65049487209,
+                     "uuid":"5fd25abf-d9f8-441a-b7eb-ddf2fc7cd79c",
+                     "failed":false,
+                     "volumeDataDensity":-0.16518402,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/ENqKDInMQD",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":628312771158,
+                     "reserved":77972655040,
+                     "uuid":"843d3c88-17df-4af9-9675-8c65bc1b693b",
+                     "failed":false,
+                     "volumeDataDensity":0.11573768,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"f86ad41a-4f59-408d-b657-2bae9484507e",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"f34b049b-0c02-4a31-b28e-3627d1c0976a",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.236092,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/8oV8UgYGox",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":766401252006,
+                     "reserved":315298886051,
+                     "uuid":"4be8a57f-8396-4160-a4c8-a635ecf086cc",
+                     "failed":false,
+                     "volumeDataDensity":0.027637154,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/yqHSnDtWaM",
+                     "capacity":3000000000000,
+                     "storageType":"DISK",
+                     "used":1369025932575,
+                     "reserved":468713801108,
+                     "uuid":"70fcc0fe-d8c4-474b-afc7-3f464f09b117",
+                     "failed":false,
+                     "volumeDataDensity":-0.058286607,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/fDlkCJbD5i",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":1546415412287,
+                     "reserved":586101293152,
+                     "uuid":"646cd8fa-f594-4182-9a25-08c9c8e906b6",
+                     "failed":false,
+                     "volumeDataDensity":0.029579043,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"cece7d49-91e6-4a21-be9a-4c4ccb5d2845",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/3CWMkkCKDT",
+                     "capacity":6000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":154143399990,
+                     "reserved":582928460583,
+                     "uuid":"243aa26e-97c2-466d-9eab-ad0f01d4f9ec",
+                     "failed":false,
+                     "volumeDataDensity":0.5216353,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/wIPoLFtVlG",
+                     "capacity":500000000000,
+                     "storageType":"RAM_DISK",
+                     "used":398499638461,
+                     "reserved":59552856543,
+                     "uuid":"ee9652bc-08ad-4903-8a6b-903c9a4166d9",
+                     "failed":false,
+                     "volumeDataDensity":-0.35467112,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/lxee8m95qD",
+                     "capacity":9000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":7403991256306,
+                     "reserved":393287914200,
+                     "uuid":"5e1381b6-ff75-4ac3-a27c-9a9470cc2b73",
+                     "failed":false,
+                     "volumeDataDensity":-0.31016737,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"53a416d6-c409-4b7e-a369-886c3e61a603",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/UomuCY9mZL",
+                     "capacity":300000000000,
+                     "storageType":"SSD",
+                     "used":7075332787,
+                     "reserved":38361849616,
+                     "uuid":"d51146c4-2098-4d31-a140-11c3e4b526ef",
+                     "failed":false,
+                     "volumeDataDensity":0.6445687,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/XhUTqlE5uC",
+                     "capacity":6000000000000,
+                     "storageType":"SSD",
+                     "used":4191076733975,
+                     "reserved":309523123411,
+                     "uuid":"8d70e59c-4551-4174-9f9e-62c3d7713ae2",
+                     "failed":false,
+                     "volumeDataDensity":-0.06489599,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/S7SyNZOhN4",
+                     "capacity":1000000000000,
+                     "storageType":"SSD",
+                     "used":399200248671,
+                     "reserved":106855141348,
+                     "uuid":"feb1f6b0-ae00-4f76-bbdf-ea3082cc5167",
+                     "failed":false,
+                     "volumeDataDensity":0.22465086,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"6eebd075-8289-4660-b964-7f3ae13665b3",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"55d239cf-1110-4e6c-9264-1d4cc6cecf2d",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.516453,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/jLAotVwYph",
+                     "capacity":2000000000000,
+                     "storageType":"DISK",
+                     "used":226427156704,
+                     "reserved":258205105607,
+                     "uuid":"064d14eb-703e-4261-900a-80c306f79feb",
+                     "failed":false,
+                     "volumeDataDensity":0.5853263,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/bOTcL0EctR",
+                     "capacity":7000000000000,
+                     "storageType":"DISK",
+                     "used":5090013635868,
+                     "reserved":1349044277203,
+                     "uuid":"9175c7f6-056f-487c-8eba-bdc4e94e8895",
+                     "failed":false,
+                     "volumeDataDensity":-0.18541223,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/42NINXIh7r",
+                     "capacity":100000000000,
+                     "storageType":"DISK",
+                     "used":36190526146,
+                     "reserved":9931325176,
+                     "uuid":"fbe8c742-bb91-4523-80ae-57118510d625",
+                     "failed":false,
+                     "volumeDataDensity":0.3135124,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"7299e8de-c863-4aae-9cd5-4fb5bbeae46a",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/pE1cU3mhGq",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":618146869309,
+                     "reserved":38104819037,
+                     "uuid":"044f2728-4be3-412a-9508-d9f3e7ac09e9",
+                     "failed":false,
+                     "volumeDataDensity":-0.23169369,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/7TZRkDHlu4",
+                     "capacity":800000000000,
+                     "storageType":"RAM_DISK",
+                     "used":457251409607,
+                     "reserved":144573798690,
+                     "uuid":"b67be8d7-d916-469c-9d79-eb686d36fd1c",
+                     "failed":false,
+                     "volumeDataDensity":-0.118005514,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/2kpGsF7zwE",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":220089329484,
+                     "reserved":182312846335,
+                     "uuid":"6a9210cb-5c53-45dd-adfc-0088451fb3f8",
+                     "failed":false,
+                     "volumeDataDensity":0.3104735,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"d1e4893e-6410-43fd-8e20-b43cec55b899",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/mEQ4eZtqfT",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":242393808618,
+                     "reserved":98738163774,
+                     "uuid":"ff0751af-0e5c-48a3-9715-6dd3269970fe",
+                     "failed":false,
+                     "volumeDataDensity":0.35496005,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/FNjmvHByGa",
+                     "capacity":700000000000,
+                     "storageType":"SSD",
+                     "used":600729090204,
+                     "reserved":88388224700,
+                     "uuid":"091cbce5-1ae7-498f-ad64-05d1a2fab49f",
+                     "failed":false,
+                     "volumeDataDensity":-0.22410464,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/Exmu0sW3fR",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":376350541637,
+                     "reserved":4285829706,
+                     "uuid":"b8466214-9069-4d72-8b35-2da12e8a6bce",
+                     "failed":false,
+                     "volumeDataDensity":-0.19296473,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"1725d1ec-26b2-46dd-b934-621dbd1ccfc4",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"81de5348-9cb3-47d2-a55a-7a6d3f48246f",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":1.5446163,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/RGyedRQ4au",
+                     "capacity":4000000000000,
+                     "storageType":"DISK",
+                     "used":246497951282,
+                     "reserved":154740776536,
+                     "uuid":"b4e223ce-0927-4bed-83e9-625f1a3c7072",
+                     "failed":false,
+                     "volumeDataDensity":0.18134677,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/CU1SjiJDyq",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":1941013140335,
+                     "reserved":330297433087,
+                     "uuid":"e9787d84-3840-43fc-ad4b-1a61a86987ed",
+                     "failed":false,
+                     "volumeDataDensity":-0.096897185,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/J0Fkdd7L4i",
+                     "capacity":300000000000,
+                     "storageType":"DISK",
+                     "used":217598898278,
+                     "reserved":16230202803,
+                     "uuid":"aded9210-c057-492b-bbe8-863c240b544e",
+                     "failed":false,
+                     "volumeDataDensity":-0.52136374,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"61a86616-e756-4296-8578-c08e2b873fd0",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/D7G7WXgzy7",
+                     "capacity":1000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":515115572402,
+                     "reserved":199986153741,
+                     "uuid":"9812ef87-659a-42f1-9f7c-628bba6b3ccb",
+                     "failed":false,
+                     "volumeDataDensity":0.117491126,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/UzNhchQYAf",
+                     "capacity":2000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":1674855619430,
+                     "reserved":63410078490,
+                     "uuid":"7d9a7267-7190-4124-8765-1c834d2dffc0",
+                     "failed":false,
+                     "volumeDataDensity":-0.103473365,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/Gye6bboH5q",
+                     "capacity":5000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":3333403714741,
+                     "reserved":482123874652,
+                     "uuid":"3e958107-8993-49a6-b41a-53a4d069c619",
+                     "failed":false,
+                     "volumeDataDensity":0.02354902,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"14985428-1e64-4225-a554-caca11457901",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/lLEB3Q6Dv2",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":1623054060496,
+                     "reserved":368857268982,
+                     "uuid":"24ec393a-6194-4c6c-9a52-1b24186b19c6",
+                     "failed":false,
+                     "volumeDataDensity":-0.09246057,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/sMy3l38lIM",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":237953573338,
+                     "reserved":37708834635,
+                     "uuid":"b7fca31b-fe1a-438a-a188-2a46e891785f",
+                     "failed":false,
+                     "volumeDataDensity":0.2457785,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/tO8pXmt0HJ",
+                     "capacity":400000000000,
+                     "storageType":"SSD",
+                     "used":281850666250,
+                     "reserved":19287738675,
+                     "uuid":"1ea5799f-a854-4760-8c44-082a7e3c017b",
+                     "failed":false,
+                     "volumeDataDensity":0.16225588,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"371f9332-d67f-4dbf-8247-70530eaa1c49",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"f7e4f501-4f51-494e-978b-5ec2197dcf6e",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      },
+      {
+         "nodeDataDensity":2.6868234,
+         "volumeSets":{
+            "DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/NdSgARFeqd",
+                     "capacity":6000000000000,
+                     "storageType":"DISK",
+                     "used":3283447905937,
+                     "reserved":959764405773,
+                     "uuid":"38a66d07-974b-4108-9653-dd429c99b3db",
+                     "failed":false,
+                     "volumeDataDensity":-0.086735725,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/cCHPTo0AgY",
+                     "capacity":800000000000,
+                     "storageType":"DISK",
+                     "used":88109254124,
+                     "reserved":11335016655,
+                     "uuid":"8e016ac0-22a6-4244-8eaa-2c4b00b781ca",
+                     "failed":false,
+                     "volumeDataDensity":0.45299214,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/cYoRTJKkHO",
+                     "capacity":500000000000,
+                     "storageType":"DISK",
+                     "used":148854625135,
+                     "reserved":94901264160,
+                     "uuid":"6c248d16-6a13-4782-9517-68c971e69d38",
+                     "failed":false,
+                     "volumeDataDensity":0.19725892,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"DISK",
+               "setID":"6d784637-5ee9-420f-8aab-8de1f30dd891",
+               "transient":false
+            },
+            "RAM_DISK":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/tPV53wXjyE",
+                     "capacity":400000000000,
+                     "storageType":"RAM_DISK",
+                     "used":16354321415,
+                     "reserved":28025661744,
+                     "uuid":"fcb6e5ac-7636-4375-915c-6924eda93843",
+                     "failed":false,
+                     "volumeDataDensity":0.5853394,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/wS0xrPattZ",
+                     "capacity":300000000000,
+                     "storageType":"RAM_DISK",
+                     "used":9443387551,
+                     "reserved":38628426165,
+                     "uuid":"f6c5a61b-45d6-43a3-8301-61967a04c365",
+                     "failed":false,
+                     "volumeDataDensity":0.5931756,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  },
+                  {
+                     "path":"/tmp/disk/ifwQkpFb7P",
+                     "capacity":7000000000000,
+                     "storageType":"RAM_DISK",
+                     "used":4328024660448,
+                     "reserved":714892629361,
+                     "uuid":"aa8cd40f-18c7-4937-a279-817064b8c0bf",
+                     "failed":false,
+                     "volumeDataDensity":-0.05931014,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":true
+                  }
+               ],
+               "storageType":"RAM_DISK",
+               "setID":"05527db9-9d57-441f-a559-88642e509047",
+               "transient":true
+            },
+            "SSD":{
+               "volumes":[
+                  {
+                     "path":"/tmp/disk/LF15aXVG5h",
+                     "capacity":900000000000,
+                     "storageType":"SSD",
+                     "used":325755211560,
+                     "reserved":124650527286,
+                     "uuid":"5690cc50-0c9e-4ac8-9866-c4663a3634f4",
+                     "failed":false,
+                     "volumeDataDensity":-0.074249774,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/E2aftFsIFf",
+                     "capacity":2000000000000,
+                     "storageType":"SSD",
+                     "used":315893295255,
+                     "reserved":313904666808,
+                     "uuid":"683bf3c8-2b22-4b7b-97ea-f076fc1c4a42",
+                     "failed":false,
+                     "volumeDataDensity":0.1585381,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  },
+                  {
+                     "path":"/tmp/disk/X9ULQGTwzV",
+                     "capacity":500000000000,
+                     "storageType":"SSD",
+                     "used":361125842200,
+                     "reserved":62331918619,
+                     "uuid":"e69b185a-27f7-464b-9c7b-c94e04807679",
+                     "failed":false,
+                     "volumeDataDensity":-0.4792234,
+                     "skip":false,
+                     "readOnly":false,
+                     "transient":false
+                  }
+               ],
+               "storageType":"SSD",
+               "setID":"d4f49420-0bb3-45ef-afff-7a5518d4f2a6",
+               "transient":false
+            }
+         },
+         "dataNodeUUID":"bfc200d7-920a-4944-9b3c-f0f7c211cb35",
+         "dataNodeIP":null,
+         "dataNodePort":0,
+         "dataNodeName":null,
+         "volumeCount":9
+      }
+   ],
+   "outPutPath":null,
+   "threshold":0.0
+}