HDFS-9469. DiskBalancer: Add Planner. (Contributed by Anu Engineer)

This commit is contained in:
Arpit Agarwal 2016-01-07 14:45:56 -08:00
parent e325c6ade9
commit 5724a10316
12 changed files with 1792 additions and 5 deletions

View File

@ -11,3 +11,5 @@ HDFS-1312 Change Log
HDFS-9611. DiskBalancer: Replace htrace json imports with jackson. HDFS-9611. DiskBalancer: Replace htrace json imports with jackson.
(Anu Engineer via Arpit Agarwal) (Anu Engineer via Arpit Agarwal)
HDFS-9469. DiskBalancer: Add Planner. (Anu Engineer via Arpit Agarwal)

View File

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

View File

@ -0,0 +1,259 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.diskbalancer.planner;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
.DiskBalancerDataNode;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
.DiskBalancerVolumeSet;
import org.apache.hadoop.util.Time;
import java.util.Iterator;
import java.util.List;
import java.util.TreeSet;
/**
* Greedy Planner is a simple planner that computes the largest possible move at
* any point of time given a volumeSet.
* <p/>
* This is done by choosing the disks with largest amount of data above and
* below the idealStorage and then a move is scheduled between them.
*/
public class GreedyPlanner implements Planner {
public static final long MB = 1024L * 1024L;
public static final long GB = MB * 1024L;
public static final long TB = GB * 1024L;
static final Log LOG = LogFactory.getLog(GreedyPlanner.class);
private final float threshold;
/**
* Constructs a greedy planner.
*
* @param threshold - Disk tolerance that we are ok with
* @param node - node on which this planner is operating upon
*/
public GreedyPlanner(float threshold, DiskBalancerDataNode node) {
this.threshold = threshold;
}
/**
* Computes a node plan for the given node.
*
* @return NodePlan
* @throws Exception
*/
@Override
public NodePlan plan(DiskBalancerDataNode node) throws Exception {
long startTime = Time.monotonicNow();
NodePlan plan = new NodePlan(node.getDataNodeName(),
node.getDataNodePort());
LOG.info("Starting plan for Node : " + node.getDataNodeUUID());
while (node.isBalancingNeeded(this.threshold)) {
for (DiskBalancerVolumeSet vSet : node.getVolumeSets().values()) {
balanceVolumeSet(node, vSet, plan);
}
}
long endTime = Time.monotonicNow();
String message = String
.format("Compute Plan for Node : %s took %d ms ",
node.getDataNodeUUID(), endTime - startTime);
LOG.info(message);
return plan;
}
/**
* Computes Steps to make a DiskBalancerVolumeSet Balanced.
*
* @param node
* @param vSet - DiskBalancerVolumeSet
* @param plan - NodePlan
*/
public void balanceVolumeSet(DiskBalancerDataNode node,
DiskBalancerVolumeSet vSet, NodePlan plan)
throws Exception {
DiskBalancerVolumeSet currentSet = new DiskBalancerVolumeSet(vSet);
while (currentSet.isBalancingNeeded(this.threshold)) {
removeSkipVolumes(currentSet);
DiskBalancerVolume lowVolume = currentSet.getSortedQueue().first();
DiskBalancerVolume highVolume = currentSet.getSortedQueue().last();
Step nextStep = null;
// ok both volumes bytes used are in the range that we expect
// Then we create a move request.
if (!lowVolume.isSkip() && !highVolume.isSkip()) {
nextStep = computeMove(currentSet, lowVolume, highVolume);
} else {
LOG.debug("Skipping compute move. lowVolume :" + lowVolume.getPath());
LOG.debug("Skipping compute move. highVolume :" + highVolume.getPath());
}
applyStep(nextStep, currentSet, lowVolume, highVolume);
if (nextStep != null) {
LOG.debug("Step : " + nextStep.toString());
plan.addStep(nextStep);
}
}
String message = String
.format("Disk Volume set %s Type : %s plan completed.",
currentSet.getSetID(),
currentSet.getVolumes().get(0).getStorageType());
plan.setNodeName(node.getDataNodeName());
plan.setNodeUUID(node.getDataNodeUUID());
plan.setTimeStamp(Time.now());
plan.setPort(node.getDataNodePort());
LOG.info(message);
}
/**
* Apply steps applies the current step on to a volumeSet so that we can
* compute next steps until we reach the desired goals.
*
* @param nextStep - nextStep or Null
* @param currentSet - Current Disk BalancerVolume Set we are operating upon
* @param lowVolume - volume
* @param highVolume - volume
*/
private void applyStep(Step nextStep, DiskBalancerVolumeSet currentSet,
DiskBalancerVolume lowVolume,
DiskBalancerVolume highVolume) throws Exception {
long used;
if (nextStep != null) {
used = lowVolume.getUsed() + nextStep.getBytesToMove();
lowVolume.setUsed(used);
used = highVolume.getUsed() - nextStep.getBytesToMove();
highVolume.setUsed(used);
}
// since the volume data changed , we need to recompute the DataDensity.
currentSet.computeVolumeDataDensity();
}
/**
* Computes a data move from the largest disk we have to smallest disk.
*
* @param currentSet - Current Disk Set we are working with
* @param lowVolume - Low Data Capacity Volume
* @param highVolume - High Data Capacity Volume
* @return Step
*/
private Step computeMove(DiskBalancerVolumeSet currentSet,
DiskBalancerVolume lowVolume,
DiskBalancerVolume highVolume) {
// Compute how many bytes we can move. First Compute the maximum that
// low Volume Can receive, then compute maximum high volume can give
// Then take the minimum of those two numbers that is the bytesToMove.
long maxLowVolumeCanReceive = (long) (
(currentSet.getIdealUsed() * lowVolume.computeEffectiveCapacity()) -
lowVolume.getUsed());
// This disk cannot take any more data from any disk.
// Remove it from our computation matrix.
if (maxLowVolumeCanReceive <= 0) {
LOG.debug(lowVolume.getPath() +
" Skipping disk from computation. Maximum data size " +
"achieved.");
lowVolume.setSkip(true);
}
long maxHighVolumeCanGive = highVolume.getUsed() -
(long) (currentSet.getIdealUsed() *
highVolume.computeEffectiveCapacity());
// This volume cannot give any more data, remove it from the
// computation matrix
if (maxHighVolumeCanGive <= 0) {
LOG.debug(highVolume.getPath() +
" Skipping disk from computation. Minimum data size " +
"achieved.");
highVolume.setSkip(true);
}
long bytesToMove = Math.min(maxLowVolumeCanReceive, maxHighVolumeCanGive);
Step nextStep = null;
if (bytesToMove > 0) {
// Create a new step
nextStep = new MoveStep(highVolume, currentSet.getIdealUsed(), lowVolume,
bytesToMove, currentSet.getSetID());
LOG.debug(nextStep.toString());
}
return nextStep;
}
/**
* Skips this volume if needed.
*
* @param currentSet - Current Disk set
* @param volume - Volume
*/
private void skipVolume(DiskBalancerVolumeSet currentSet,
DiskBalancerVolume volume) {
String message = String.format(
"Skipping volume. Volume : %s " +
"Type : %s Target " +
"Number of bytes : %f lowVolume dfsUsed : %d. Skipping this " +
"volume from all future balancing calls.", volume.getPath(),
volume.getStorageType(),
currentSet.getIdealUsed() * volume.getCapacity(), volume.getUsed());
volume.setSkip(true);
LOG.debug(message);
}
// Removes all volumes which are part of the volumeSet but skip flag is set.
private void removeSkipVolumes(DiskBalancerVolumeSet currentSet) {
List<DiskBalancerVolume> volumeList = currentSet.getVolumes();
Iterator<DiskBalancerVolume> volumeIterator = volumeList.iterator();
while (volumeIterator.hasNext()) {
DiskBalancerVolume vol = volumeIterator.next();
if (vol.isSkip() || vol.isFailed()) {
currentSet.removeVolume(vol);
}
}
currentSet.computeVolumeDataDensity();
}
/**
* This function is used only for debugging purposes to ensure queue looks
* correct.
*
* @param queue - Queue
*/
private void printQueue(TreeSet<DiskBalancerVolume> queue) {
String format = String.format("First Volume : %s, DataDensity : %f",
queue.first().getPath(),
queue.first().getVolumeDataDensity());
LOG.info(format);
format = String
.format("Last Volume : %s, DataDensity : %f%n", queue.last().getPath(),
queue.last().getVolumeDataDensity());
LOG.info(format);
}
}

View File

@ -0,0 +1,181 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.diskbalancer.planner;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
import org.apache.hadoop.util.StringUtils;
/**
* Move step is a step that planner can execute that will move data from one
* volume to another.
*/
public class MoveStep implements Step {
private DiskBalancerVolume sourceVolume;
private DiskBalancerVolume destinationVolume;
private float idealStorage;
private long bytesToMove;
private String volumeSetID;
/**
* Constructs a MoveStep for the volume set.
*
* @param sourceVolume - Source Disk
* @param idealStorage - Ideal Storage Value for this disk set
* @param destinationVolume - Destination dis
* @param bytesToMove - number of bytes to move
* @param volumeSetID - a diskBalancer generated id.
*/
public MoveStep(DiskBalancerVolume sourceVolume, float idealStorage,
DiskBalancerVolume destinationVolume, long bytesToMove,
String volumeSetID) {
this.destinationVolume = destinationVolume;
this.idealStorage = idealStorage;
this.sourceVolume = sourceVolume;
this.bytesToMove = bytesToMove;
this.volumeSetID = volumeSetID;
}
/**
* Empty Constructor for JSON serialization.
*/
public MoveStep() {
}
/**
* Returns number of bytes to move.
*
* @return - long
*/
@Override
public long getBytesToMove() {
return bytesToMove;
}
/**
* Gets the destination volume.
*
* @return - volume
*/
@Override
public DiskBalancerVolume getDestinationVolume() {
return destinationVolume;
}
/**
* Gets the IdealStorage.
*
* @return float
*/
@Override
public float getIdealStorage() {
return idealStorage;
}
/**
* Gets Source Volume.
*
* @return -- Source Volume
*/
@Override
public DiskBalancerVolume getSourceVolume() {
return sourceVolume;
}
/**
* Gets a volume Set ID.
*
* @return String
*/
@Override
public String getVolumeSetID() {
return volumeSetID;
}
/**
* Set source volume.
*
* @param sourceVolume - volume
*/
public void setSourceVolume(DiskBalancerVolume sourceVolume) {
this.sourceVolume = sourceVolume;
}
/**
* Sets destination volume.
*
* @param destinationVolume - volume
*/
public void setDestinationVolume(DiskBalancerVolume destinationVolume) {
this.destinationVolume = destinationVolume;
}
/**
* Sets Ideal Storage.
*
* @param idealStorage - ideal Storage
*/
public void setIdealStorage(float idealStorage) {
this.idealStorage = idealStorage;
}
/**
* Sets bytes to move.
*
* @param bytesToMove - number of bytes
*/
public void setBytesToMove(long bytesToMove) {
this.bytesToMove = bytesToMove;
}
/**
* Sets volume id.
*
* @param volumeSetID - volume ID
*/
public void setVolumeSetID(String volumeSetID) {
this.volumeSetID = volumeSetID;
}
/**
* Returns a string representation of the object.
*
* @return a string representation of the object.
*/
@Override
public String toString() {
return String.format("%s\t %s\t %s\t %s%n",
this.getSourceVolume().getPath(),
this.getDestinationVolume().getPath(),
getSizeString(this.getBytesToMove()),
this.getDestinationVolume().getStorageType());
}
/**
* Returns human readable move sizes.
*
* @param size - bytes being moved.
* @return String
*/
@Override
public String getSizeString(long size) {
return StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1);
}
}

View File

@ -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;
}
}

View File

@ -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;
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.diskbalancer.planner;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
.DiskBalancerDataNode;
/**
* Returns a planner based on the user defined tags.
*/
public final class PlannerFactory {
static final Log LOG = LogFactory.getLog(PlannerFactory.class);
public static final String GREEDY_PLANNER = "greedyPlanner";
/**
* Gets a planner object.
* @param plannerName - name of the planner.
* @param node - Datanode.
* @param threshold - percentage
* @return Planner
*/
public static Planner getPlanner(String plannerName,
DiskBalancerDataNode node, float threshold) {
if (plannerName.equals(GREEDY_PLANNER)) {
if (LOG.isDebugEnabled()) {
String message = String
.format("Creating a %s for Node : %s IP : %s ID : %s",
GREEDY_PLANNER, node.getDataNodeName(), node.getDataNodeIP(),
node.getDataNodeUUID());
LOG.debug(message);
}
return new GreedyPlanner(threshold, node);
}
throw new IllegalArgumentException("Unrecognized planner name : " +
plannerName);
}
private PlannerFactory() {
// Never constructed
}
}

View File

@ -0,0 +1,68 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.diskbalancer.planner;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
/**
* A step in the plan.
*/
public interface Step {
/**
* Return the number of bytes to move.
*
* @return bytes
*/
long getBytesToMove();
/**
* Gets the destination volume.
*
* @return - volume
*/
DiskBalancerVolume getDestinationVolume();
/**
* Gets the IdealStorage.
*
* @return idealStorage
*/
float getIdealStorage();
/**
* Gets Source Volume.
*
* @return -- Source Volume
*/
DiskBalancerVolume getSourceVolume();
/**
* Gets a volume Set ID.
*
* @return String
*/
String getVolumeSetID();
/**
* Returns a String representation of the Step Size.
*
* @return String
*/
String getSizeString(long size);
}

View File

@ -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;
*/

View File

@ -33,9 +33,9 @@
* Helper class to create various cluster configrations at run time. * Helper class to create various cluster configrations at run time.
*/ */
public class DiskBalancerTestUtil { public class DiskBalancerTestUtil {
// we modeling disks here, hence HDD style units public static final long MB = 1024 * 1024L;
public static final long GB = 1000000000L; public static final long GB = MB * 1024L;
public static final long TB = 1000000000000L; public static final long TB = GB * 1024L;
private static int[] diskSizes = private static int[] diskSizes =
{1, 2, 3, 4, 5, 6, 7, 8, 9, 100, 200, 300, 400, 500, 600, 700, 800, 900}; {1, 2, 3, 4, 5, 6, 7, 8, 9, 100, 200, 300, 400, 500, 600, 700, 800, 900};
Random rand; Random rand;

View File

@ -0,0 +1,462 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.hdfs.server.diskbalancer;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ClusterConnector;
import org.apache.hadoop.hdfs.server.diskbalancer.connectors.ConnectorFactory;
import org.apache.hadoop.hdfs.server.diskbalancer.connectors.NullConnector;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerCluster;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
.DiskBalancerDataNode;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
.DiskBalancerVolumeSet;
import org.apache.hadoop.hdfs.server.diskbalancer.planner.GreedyPlanner;
import org.apache.hadoop.hdfs.server.diskbalancer.planner.MoveStep;
import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
import org.apache.hadoop.hdfs.server.diskbalancer.planner.Step;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URI;
import java.util.List;
import java.util.UUID;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
public class TestPlanner {
static final Logger LOG =
LoggerFactory.getLogger(TestPlanner.class);
@Test
public void TestGreedyPlannerBalanceVolumeSet() throws Exception {
URI clusterJson = getClass()
.getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
null);
DiskBalancerCluster cluster = new DiskBalancerCluster(jsonConnector);
cluster.readClusterInfo();
Assert.assertEquals(3, cluster.getNodes().size());
cluster.setNodesToProcess(cluster.getNodes());
DiskBalancerDataNode node = cluster.getNodes().get(0);
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
}
@Test
public void TestGreedyPlannerComputePlan() throws Exception {
URI clusterJson = getClass()
.getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI();
ClusterConnector jsonConnector = ConnectorFactory.getCluster(clusterJson,
null);
DiskBalancerCluster cluster = new DiskBalancerCluster(jsonConnector);
cluster.readClusterInfo();
Assert.assertEquals(3, cluster.getNodes().size());
cluster.setNodesToProcess(cluster.getNodes());
List<NodePlan> plan = cluster.computePlan(10.0f);
Assert.assertNotNull(plan);
}
private DiskBalancerVolume createVolume(String path, int capacityInGB,
int usedInGB) {
DiskBalancerTestUtil util = new DiskBalancerTestUtil();
DiskBalancerVolume volume = util.createRandomVolume(StorageType.SSD);
volume.setPath(path);
volume.setCapacity(capacityInGB * DiskBalancerTestUtil.GB);
volume.setReserved(0);
volume.setUsed(usedInGB * DiskBalancerTestUtil.GB);
return volume;
}
@Test
public void TestGreedyPlannerNoNodeCluster() throws Exception {
GreedyPlanner planner = new GreedyPlanner(10.0f, null);
assertNotNull(planner);
}
@Test
public void TestGreedyPlannerNoVolumeTest() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
List<NodePlan> planList = cluster.computePlan(10.0f);
assertNotNull(planList);
}
@Test
public void TestGreedyPlannerOneVolumeNoPlanTest() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
node.addVolume(volume30);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
// With a single volume we should not have any plans for moves.
assertEquals(0, plan.getVolumeSetPlans().size());
}
@Test
public void TestGreedyPlannerTwoVolume() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
DiskBalancerVolume volume10 = createVolume("volume10", 100, 10);
node.addVolume(volume10);
node.addVolume(volume30);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
// We should have only one planned move from
// volume30 to volume10 of 10 GB Size.
assertEquals(1, plan.getVolumeSetPlans().size());
Step step = plan.getVolumeSetPlans().get(0);
assertEquals("volume30", step.getSourceVolume().getPath());
assertEquals("volume10", step.getDestinationVolume().getPath());
assertEquals("10 G", step.getSizeString(step.getBytesToMove()));
}
/**
* In this test we pass 3 volumes with 30, 20 and 10 GB of data used. We
* expect the planner to print out 20 GB on each volume.
* <p/>
* That is the plan should say move 10 GB from volume30 to volume10.
*/
@Test
public void TestGreedyPlannerEqualizeData() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
DiskBalancerVolume volume30 = createVolume("volume30", 100, 30);
DiskBalancerVolume volume20 = createVolume("volume20", 100, 20);
DiskBalancerVolume volume10 = createVolume("volume10", 100, 10);
node.addVolume(volume10);
node.addVolume(volume20);
node.addVolume(volume30);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
// We should have only one planned move from
// volume30 to volume10 of 10 GB Size.
assertEquals(1, plan.getVolumeSetPlans().size());
Step step = plan.getVolumeSetPlans().get(0);
assertEquals("volume30", step.getSourceVolume().getPath());
assertEquals("volume10", step.getDestinationVolume().getPath());
assertEquals("10 G", step.getSizeString(step.getBytesToMove()));
}
@Test
public void TestGreedyPlannerEqualDisksNoMoves() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
// All disks have same capacity of data
DiskBalancerVolume volume1 = createVolume("volume1", 100, 30);
DiskBalancerVolume volume2 = createVolume("volume2", 100, 30);
DiskBalancerVolume volume3 = createVolume("volume3", 100, 30);
node.addVolume(volume1);
node.addVolume(volume2);
node.addVolume(volume3);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
// since we have same size of data in all disks , we should have
// no moves planned.
assertEquals(0, plan.getVolumeSetPlans().size());
}
@Test
public void TestGreedyPlannerMoveFromSingleDisk() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
// All disks have same capacity of data
DiskBalancerVolume volume1 = createVolume("volume100", 200, 100);
DiskBalancerVolume volume2 = createVolume("volume0-1", 200, 0);
DiskBalancerVolume volume3 = createVolume("volume0-2", 200, 0);
node.addVolume(volume1);
node.addVolume(volume2);
node.addVolume(volume3);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
// We should see 2 move plans. One from volume100 to volume0-1
// and another from volume100 to volume0-2
assertEquals(2, plan.getVolumeSetPlans().size());
Step step = plan.getVolumeSetPlans().get(0);
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
step = plan.getVolumeSetPlans().get(1);
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("33.3 G", step.getSizeString(step.getBytesToMove()));
}
@Test
public void TestGreedyPlannerThresholdTest() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
DiskBalancerVolume volume1 = createVolume("volume100", 1000, 100);
DiskBalancerVolume volume2 = createVolume("volume0-1", 300, 0);
DiskBalancerVolume volume3 = createVolume("volume0-2", 300, 0);
node.addVolume(volume1);
node.addVolume(volume2);
node.addVolume(volume3);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner planner = new GreedyPlanner(10.0f, node);
NodePlan plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort
());
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
//We should see NO moves since the total data on the volume100
// is less than or equal to threashold value that we pass, which is 10%
assertEquals(0, plan.getVolumeSetPlans().size());
// for this new planner we are passing 1% as as threshold value
// hence planner must move data if possible.
GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
NodePlan newPlan = new NodePlan(node.getDataNodeName(), node
.getDataNodePort());
newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), newPlan);
assertEquals(2, newPlan.getVolumeSetPlans().size());
// Move size should say move 19 GB
// Here is how the math works out.
// TotalCapacity = 1000 + 300 + 300 = 1600 GB
// TotolUsed = 100
// Expected data% on each disk = 0.0625
// On Disk (volume0-1) = 300 * 0.0625 - 18.75 -- We round it up
// in the display string -- hence 18.8 GB, it will be same on volume 2 too.
// since they are equal sized disks with same used capacity
Step step = newPlan.getVolumeSetPlans().get(0);
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
step = newPlan.getVolumeSetPlans().get(1);
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("18.8 G", step.getSizeString(step.getBytesToMove()));
}
@Test
public void TestGreedyPlannerPlanWithDifferentDiskSizes() throws Exception {
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
DiskBalancerVolume volume1 = createVolume("volume100", 1000, 100);
DiskBalancerVolume volume2 = createVolume("volume0-1", 500, 0);
DiskBalancerVolume volume3 = createVolume("volume0-2", 250, 0);
node.addVolume(volume1);
node.addVolume(volume2);
node.addVolume(volume3);
nullConnector.addNode(node);
cluster.readClusterInfo();
Assert.assertEquals(1, cluster.getNodes().size());
GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
NodePlan newPlan = new NodePlan(node.getDataNodeName(), node
.getDataNodePort());
newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), newPlan);
assertEquals(2, newPlan.getVolumeSetPlans().size());
// Move size should say move 26.6 GB and 13.3 GB
// Here is how the math works out.
// TotalCapacity = 1000 + 500 + 250 = 1750 GB
// TotolUsed = 100
// Expected data% on each disk = 0.05714
// On Disk (volume0-1) = 500 * 0.05714 = 28.57
// on Voulume0-2 = 300 * 0.05714 = 14.28
for (Step step : newPlan.getVolumeSetPlans()) {
if (step.getDestinationVolume().getPath().equals("volume0-1")) {
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("28.6 G",
step.getSizeString(step.getBytesToMove()));
}
if (step.getDestinationVolume().getPath().equals("volume0-2")) {
assertEquals("volume100", step.getSourceVolume().getPath());
assertEquals("14.3 G",
step.getSizeString(step.getBytesToMove()));
}
}
Step step = newPlan.getVolumeSetPlans().get(0);
assertEquals(0.05714f, step.getIdealStorage(), 0.001f);
}
@Test
public void TestLoadsCorrectClusterConnector() throws Exception {
ClusterConnector connector = ConnectorFactory.getCluster(getClass()
.getResource("/diskBalancer/data-cluster-3node-3disk.json").toURI()
, null);
assertEquals(connector.getClass().toString(),
"class org.apache.hadoop.hdfs.server.diskbalancer.connectors." +
"JsonNodeConnector");
}
@Test
public void TestPlannerScale() throws Exception {
final int diskCount = 256; // it is rare to see more than 48 disks
DiskBalancerTestUtil util = new DiskBalancerTestUtil();
DiskBalancerVolumeSet vSet =
util.createRandomVolumeSet(StorageType.SSD, diskCount);
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
int diskNum = 0;
for (DiskBalancerVolume vol : vSet.getVolumes()) {
vol.setPath("volume" + diskNum++);
node.addVolume(vol);
}
nullConnector.addNode(node);
cluster.readClusterInfo();
GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
NodePlan newPlan = new NodePlan(node.getDataNodeName(),
node.getDataNodePort());
newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"),
newPlan);
// Assuming that our random disks at least generated one step
assertTrue("No Steps Generated from random disks, very unlikely",
newPlan.getVolumeSetPlans().size() > 0);
assertTrue("Steps Generated less than disk count - false",
newPlan.getVolumeSetPlans().size() < diskCount);
LOG.info("Number of steps are : %d%n", newPlan.getVolumeSetPlans().size());
}
@Test
public void TestNodePlanSerialize() throws Exception {
final int diskCount = 12;
DiskBalancerTestUtil util = new DiskBalancerTestUtil();
DiskBalancerVolumeSet vSet =
util.createRandomVolumeSet(StorageType.SSD, diskCount);
NullConnector nullConnector = new NullConnector();
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
DiskBalancerDataNode node =
new DiskBalancerDataNode(UUID.randomUUID().toString());
int diskNum = 0;
for (DiskBalancerVolume vol : vSet.getVolumes()) {
vol.setPath("volume" + diskNum++);
node.addVolume(vol);
}
nullConnector.addNode(node);
cluster.readClusterInfo();
GreedyPlanner newPlanner = new GreedyPlanner(01.0f, node);
NodePlan newPlan = new NodePlan(node.getDataNodeName(),
node.getDataNodePort());
newPlanner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"),
newPlan);
String planString = newPlan.toJson();
assertNotNull(planString);
NodePlan copy = NodePlan.parseJson(planString);
assertNotNull(copy);
assertEquals(newPlan.getVolumeSetPlans().size(),
copy.getVolumeSetPlans().size());
}
}

View File

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