HDFS-10478. DiskBalancer: resolve volume path names. Contributed by Anu Engineer.
This commit is contained in:
parent
47dcb0f952
commit
64ccb232cc
|
@ -23,6 +23,10 @@ import org.apache.commons.lang.StringUtils;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.server.diskbalancer.DiskBalancerConstants;
|
||||
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
|
||||
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolumeSet;
|
||||
import org.apache.hadoop.hdfs.tools.DiskBalancer;
|
||||
import org.apache.hadoop.hdfs.server.diskbalancer.datamodel
|
||||
.DiskBalancerDataNode;
|
||||
|
@ -32,7 +36,9 @@ import org.codehaus.jackson.map.ObjectMapper;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Class that implements Plan Command.
|
||||
|
@ -111,7 +117,10 @@ public class PlanCommand extends Command {
|
|||
cmd.getOptionValue(DiskBalancer.PLAN));
|
||||
}
|
||||
this.thresholdPercentage = getThresholdPercentage(cmd);
|
||||
|
||||
LOG.debug("threshold Percentage is {}", this.thresholdPercentage);
|
||||
setNodesToProcess(node);
|
||||
populatePathNames(node);
|
||||
|
||||
List<NodePlan> plans = getCluster().computePlan(this.thresholdPercentage);
|
||||
setPlanParams(plans);
|
||||
|
@ -137,6 +146,32 @@ public class PlanCommand extends Command {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Reads the Physical path of the disks we are balancing. This is needed to
|
||||
* make the disk balancer human friendly and not used in balancing.
|
||||
*
|
||||
* @param node - Disk Balancer Node.
|
||||
*/
|
||||
private void populatePathNames(DiskBalancerDataNode node) throws IOException {
|
||||
String dnAddress = node.getDataNodeIP() + ":" + node.getDataNodePort();
|
||||
ClientDatanodeProtocol dnClient = getDataNodeProxy(dnAddress);
|
||||
String volumeNameJson = dnClient.getDiskBalancerSetting(
|
||||
DiskBalancerConstants.DISKBALANCER_VOLUME_NAME);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, String> volumeMap =
|
||||
mapper.readValue(volumeNameJson, HashMap.class);
|
||||
for (DiskBalancerVolumeSet set : node.getVolumeSets().values()) {
|
||||
for (DiskBalancerVolume vol : set.getVolumes()) {
|
||||
if (volumeMap.containsKey(vol.getUuid())) {
|
||||
vol.setPath(volumeMap.get(vol.getUuid()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets extended help for this command.
|
||||
*
|
||||
|
@ -198,9 +233,11 @@ public class PlanCommand extends Command {
|
|||
for (NodePlan plan : plans) {
|
||||
for (Step step : plan.getVolumeSetPlans()) {
|
||||
if (this.bandwidth > 0) {
|
||||
LOG.debug("Setting bandwidth to {}", this.bandwidth);
|
||||
step.setBandwidth(this.bandwidth);
|
||||
}
|
||||
if (this.maxError > 0) {
|
||||
LOG.debug("Setting max error to {}", this.maxError);
|
||||
step.setMaxDiskErrors(this.maxError);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -155,7 +155,6 @@ class DBNameNodeConnector implements ClusterConnector {
|
|||
.READ_ONLY_SHARED) || report.isFailed());
|
||||
volume.setStorageType(storage.getStorageType().name());
|
||||
volume.setIsTransient(storage.getStorageType().isTransient());
|
||||
//volume.setPath(storage.getVolumePath());
|
||||
node.addVolume(volume);
|
||||
}
|
||||
|
||||
|
|
|
@ -456,4 +456,51 @@ public class TestPlanner {
|
|||
assertEquals(newPlan.getVolumeSetPlans().size(),
|
||||
copy.getVolumeSetPlans().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGreedyPlannerLargeDisksWithData() throws Exception {
|
||||
NullConnector nullConnector = new NullConnector();
|
||||
DiskBalancerCluster cluster = new DiskBalancerCluster(nullConnector);
|
||||
|
||||
DiskBalancerDataNode node =
|
||||
new DiskBalancerDataNode(UUID.randomUUID().toString());
|
||||
|
||||
// All disks have same capacity of data
|
||||
DiskBalancerVolume volume1 = createVolume("volume1", 1968, 88);
|
||||
DiskBalancerVolume volume2 = createVolume("volume2", 1968, 88);
|
||||
DiskBalancerVolume volume3 = createVolume("volume3", 1968, 111);
|
||||
DiskBalancerVolume volume4 = createVolume("volume4", 1968, 111);
|
||||
DiskBalancerVolume volume5 = createVolume("volume5", 1968, 30);
|
||||
DiskBalancerVolume volume6 = createVolume("volume6", 1563, 30);
|
||||
DiskBalancerVolume volume7 = createVolume("volume7", 1563, 30);
|
||||
DiskBalancerVolume volume8 = createVolume("volume8", 1563, 30);
|
||||
DiskBalancerVolume volume9 = createVolume("volume9", 1563, 210);
|
||||
|
||||
|
||||
|
||||
|
||||
node.addVolume(volume1);
|
||||
node.addVolume(volume2);
|
||||
node.addVolume(volume3);
|
||||
|
||||
node.addVolume(volume4);
|
||||
node.addVolume(volume5);
|
||||
node.addVolume(volume6);
|
||||
|
||||
node.addVolume(volume7);
|
||||
node.addVolume(volume8);
|
||||
node.addVolume(volume9);
|
||||
|
||||
|
||||
nullConnector.addNode(node);
|
||||
cluster.readClusterInfo();
|
||||
Assert.assertEquals(1, cluster.getNodes().size());
|
||||
|
||||
GreedyPlanner planner = new GreedyPlanner(1.0f, node);
|
||||
NodePlan plan = new NodePlan(node.getDataNodeName(),
|
||||
node.getDataNodePort());
|
||||
planner.balanceVolumeSet(node, node.getVolumeSets().get("SSD"), plan);
|
||||
|
||||
assertTrue(plan.getVolumeSetPlans().size() > 2);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue