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