HDFS-9850. DiskBalancer: Explore removing references to FsVolumeSpi. Contributed by Manoj Govindassamy.

This commit is contained in:
Anu Engineer 2016-09-27 21:35:06 -07:00
parent 6437ba18c5
commit 03f519a757
2 changed files with 300 additions and 68 deletions

View File

@ -22,6 +22,8 @@ import com.google.common.base.Preconditions;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi
.FsVolumeReferences;
import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -41,6 +43,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.Charset; import java.nio.charset.Charset;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.LinkedList; import java.util.LinkedList;
@ -192,7 +195,30 @@ public class DiskBalancer {
} }
/** /**
* Returns the Current Work Status of a submitted Plan. * Get FsVolume by volume UUID.
* @param fsDataset
* @param volUuid
* @return FsVolumeSpi
*/
private static FsVolumeSpi getFsVolume(final FsDatasetSpi<?> fsDataset,
final String volUuid) {
FsVolumeSpi fsVolume = null;
try (FsVolumeReferences volumeReferences =
fsDataset.getFsVolumeReferences()) {
for (int i = 0; i < volumeReferences.size(); i++) {
if (volumeReferences.get(i).getStorageID().equals(volUuid)) {
fsVolume = volumeReferences.get(i);
break;
}
}
} catch (IOException e) {
LOG.warn("Disk Balancer - Error when closing volume references: ", e);
}
return fsVolume;
}
/**
* Returns the current work status of a previously submitted Plan.
* *
* @return DiskBalancerWorkStatus. * @return DiskBalancerWorkStatus.
* @throws DiskBalancerException * @throws DiskBalancerException
@ -214,8 +240,8 @@ public class DiskBalancer {
for (Map.Entry<VolumePair, DiskBalancerWorkItem> entry : for (Map.Entry<VolumePair, DiskBalancerWorkItem> entry :
workMap.entrySet()) { workMap.entrySet()) {
DiskBalancerWorkEntry workEntry = new DiskBalancerWorkEntry( DiskBalancerWorkEntry workEntry = new DiskBalancerWorkEntry(
entry.getKey().getSource().getBasePath(), entry.getKey().getSourceVolBasePath(),
entry.getKey().getDest().getBasePath(), entry.getKey().getDestVolBasePath(),
entry.getValue()); entry.getValue());
status.addWorkEntry(workEntry); status.addWorkEntry(workEntry);
} }
@ -269,12 +295,7 @@ public class DiskBalancer {
lock.lock(); lock.lock();
try { try {
checkDiskBalancerEnabled(); checkDiskBalancerEnabled();
Map<String, String> pathMap = new HashMap<>(); return JsonUtil.toJsonString(getStorageIDToVolumeBasePathMap());
Map<String, FsVolumeSpi> volMap = getStorageIDToVolumeMap();
for (Map.Entry<String, FsVolumeSpi> entry : volMap.entrySet()) {
pathMap.put(entry.getKey(), entry.getValue().getBasePath());
}
return JsonUtil.toJsonString(pathMap);
} catch (DiskBalancerException ex) { } catch (DiskBalancerException ex) {
throw ex; throw ex;
} catch (IOException e) { } catch (IOException e) {
@ -434,47 +455,52 @@ public class DiskBalancer {
// Cleanup any residual work in the map. // Cleanup any residual work in the map.
workMap.clear(); workMap.clear();
Map<String, FsVolumeSpi> pathMap = getStorageIDToVolumeMap(); Map<String, String> storageIDToVolBasePathMap =
getStorageIDToVolumeBasePathMap();
for (Step step : plan.getVolumeSetPlans()) { for (Step step : plan.getVolumeSetPlans()) {
String sourceuuid = step.getSourceVolume().getUuid(); String sourceVolUuid = step.getSourceVolume().getUuid();
String destinationuuid = step.getDestinationVolume().getUuid(); String destVolUuid = step.getDestinationVolume().getUuid();
FsVolumeSpi sourceVol = pathMap.get(sourceuuid); String sourceVolBasePath = storageIDToVolBasePathMap.get(sourceVolUuid);
if (sourceVol == null) { if (sourceVolBasePath == null) {
LOG.error("Disk Balancer - Unable to find source volume. submitPlan " + final String errMsg = "Disk Balancer - Unable to find volume: "
"failed."); + step.getSourceVolume().getPath() + ". SubmitPlan failed.";
throw new DiskBalancerException("Unable to find source volume.", LOG.error(errMsg);
throw new DiskBalancerException(errMsg,
DiskBalancerException.Result.INVALID_VOLUME); DiskBalancerException.Result.INVALID_VOLUME);
} }
FsVolumeSpi destVol = pathMap.get(destinationuuid); String destVolBasePath = storageIDToVolBasePathMap.get(destVolUuid);
if (destVol == null) { if (destVolBasePath == null) {
LOG.error("Disk Balancer - Unable to find destination volume. " + final String errMsg = "Disk Balancer - Unable to find volume: "
"submitPlan failed."); + step.getDestinationVolume().getPath() + ". SubmitPlan failed.";
throw new DiskBalancerException("Unable to find destination volume.", LOG.error(errMsg);
throw new DiskBalancerException(errMsg,
DiskBalancerException.Result.INVALID_VOLUME); DiskBalancerException.Result.INVALID_VOLUME);
} }
createWorkPlan(sourceVol, destVol, step); VolumePair volumePair = new VolumePair(sourceVolUuid,
sourceVolBasePath, destVolUuid, destVolBasePath);
createWorkPlan(volumePair, step);
} }
} }
/** /**
* Returns a path to Volume Map. * Returns volume UUID to volume base path map.
* *
* @return Map * @return Map
* @throws DiskBalancerException * @throws DiskBalancerException
*/ */
private Map<String, FsVolumeSpi> getStorageIDToVolumeMap() private Map<String, String> getStorageIDToVolumeBasePathMap()
throws DiskBalancerException { throws DiskBalancerException {
Map<String, FsVolumeSpi> pathMap = new HashMap<>(); Map<String, String> storageIDToVolBasePathMap = new HashMap<>();
FsDatasetSpi.FsVolumeReferences references; FsDatasetSpi.FsVolumeReferences references;
try { try {
try(AutoCloseableLock lock = this.dataset.acquireDatasetLock()) { try(AutoCloseableLock lock = this.dataset.acquireDatasetLock()) {
references = this.dataset.getFsVolumeReferences(); references = this.dataset.getFsVolumeReferences();
for (int ndx = 0; ndx < references.size(); ndx++) { for (int ndx = 0; ndx < references.size(); ndx++) {
FsVolumeSpi vol = references.get(ndx); FsVolumeSpi vol = references.get(ndx);
pathMap.put(vol.getStorageID(), vol); storageIDToVolBasePathMap.put(vol.getStorageID(), vol.getBasePath());
} }
references.close(); references.close();
} }
@ -483,7 +509,7 @@ public class DiskBalancer {
throw new DiskBalancerException("Internal error", ex, throw new DiskBalancerException("Internal error", ex,
DiskBalancerException.Result.INTERNAL_ERROR); DiskBalancerException.Result.INTERNAL_ERROR);
} }
return pathMap; return storageIDToVolBasePathMap;
} }
/** /**
@ -513,26 +539,24 @@ public class DiskBalancer {
/** /**
* Insert work items to work map. * Insert work items to work map.
* * @param volumePair - VolumePair
* @param source - Source vol * @param step - Move Step
* @param dest - destination volume
* @param step - Move Step
*/ */
private void createWorkPlan(FsVolumeSpi source, FsVolumeSpi dest, private void createWorkPlan(final VolumePair volumePair, Step step)
Step step) throws DiskBalancerException { throws DiskBalancerException {
if (volumePair.getSourceVolUuid().equals(volumePair.getDestVolUuid())) {
if (source.getStorageID().equals(dest.getStorageID())) { final String errMsg = "Disk Balancer - Source and destination volumes " +
LOG.info("Disk Balancer - source & destination volumes are same."); "are same: " + volumePair.getSourceVolUuid();
throw new DiskBalancerException("source and destination volumes are " + LOG.warn(errMsg);
"same.", DiskBalancerException.Result.INVALID_MOVE); throw new DiskBalancerException(errMsg,
DiskBalancerException.Result.INVALID_MOVE);
} }
VolumePair pair = new VolumePair(source, dest);
long bytesToMove = step.getBytesToMove(); long bytesToMove = step.getBytesToMove();
// In case we have a plan with more than // In case we have a plan with more than
// one line of same <source, dest> // one line of same VolumePair
// we compress that into one work order. // we compress that into one work order.
if (workMap.containsKey(pair)) { if (workMap.containsKey(volumePair)) {
bytesToMove += workMap.get(pair).getBytesToCopy(); bytesToMove += workMap.get(volumePair).getBytesToCopy();
} }
DiskBalancerWorkItem work = new DiskBalancerWorkItem(bytesToMove, 0); DiskBalancerWorkItem work = new DiskBalancerWorkItem(bytesToMove, 0);
@ -542,7 +566,7 @@ public class DiskBalancer {
work.setBandwidth(step.getBandwidth()); work.setBandwidth(step.getBandwidth());
work.setTolerancePercent(step.getTolerancePercent()); work.setTolerancePercent(step.getTolerancePercent());
work.setMaxDiskErrors(step.getMaxDiskErrors()); work.setMaxDiskErrors(step.getMaxDiskErrors());
workMap.put(pair, work); workMap.put(volumePair, work);
} }
/** /**
@ -591,39 +615,63 @@ public class DiskBalancer {
} }
/** /**
* Holds references to actual volumes that we will be operating against. * Holds source and dest volumes UUIDs and their BasePaths
* that disk balancer will be operating against.
*/ */
public static class VolumePair { public static class VolumePair {
private final FsVolumeSpi source; private final String sourceVolUuid;
private final FsVolumeSpi dest; private final String destVolUuid;
private final String sourceVolBasePath;
private final String destVolBasePath;
/** /**
* Constructs a volume pair. * Constructs a volume pair.
* * @param sourceVolUuid - Source Volume
* @param source - Source Volume * @param sourceVolBasePath - Source Volume Base Path
* @param dest - Destination Volume * @param destVolUuid - Destination Volume
* @param destVolBasePath - Destination Volume Base Path
*/ */
public VolumePair(FsVolumeSpi source, FsVolumeSpi dest) { public VolumePair(final String sourceVolUuid,
this.source = source; final String sourceVolBasePath, final String destVolUuid,
this.dest = dest; final String destVolBasePath) {
this.sourceVolUuid = sourceVolUuid;
this.sourceVolBasePath = sourceVolBasePath;
this.destVolUuid = destVolUuid;
this.destVolBasePath = destVolBasePath;
} }
/** /**
* gets source volume. * Gets source volume UUID.
* *
* @return volume * @return UUID String
*/ */
public FsVolumeSpi getSource() { public String getSourceVolUuid() {
return source; return sourceVolUuid;
} }
/** /**
* Gets Destination volume. * Gets source volume base path.
* * @return String
* @return volume.
*/ */
public FsVolumeSpi getDest() { public String getSourceVolBasePath() {
return dest; return sourceVolBasePath;
}
/**
* Gets destination volume UUID.
*
* @return UUID String
*/
public String getDestVolUuid() {
return destVolUuid;
}
/**
* Gets desitnation volume base path.
*
* @return String
*/
public String getDestVolBasePath() {
return destVolBasePath;
} }
@Override @Override
@ -636,13 +684,21 @@ public class DiskBalancer {
} }
VolumePair that = (VolumePair) o; VolumePair that = (VolumePair) o;
return source.equals(that.source) && dest.equals(that.dest); return sourceVolUuid.equals(that.sourceVolUuid)
&& sourceVolBasePath.equals(that.sourceVolBasePath)
&& destVolUuid.equals(that.destVolUuid)
&& destVolBasePath.equals(that.destVolBasePath);
} }
@Override @Override
public int hashCode() { public int hashCode() {
int result = source.getBasePath().hashCode(); final int primeNum = 31;
result = 31 * result + dest.getBasePath().hashCode(); final List<String> volumeStrList = Arrays.asList(sourceVolUuid,
sourceVolBasePath, destVolUuid, destVolBasePath);
int result = 1;
for (String str : volumeStrList) {
result = (result * primeNum) + str.hashCode();
}
return result; return result;
} }
} }
@ -932,8 +988,28 @@ public class DiskBalancer {
*/ */
@Override @Override
public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) { public void copyBlocks(VolumePair pair, DiskBalancerWorkItem item) {
FsVolumeSpi source = pair.getSource(); String sourceVolUuid = pair.getSourceVolUuid();
FsVolumeSpi dest = pair.getDest(); String destVolUuuid = pair.getDestVolUuid();
// When any of the DiskBalancerWorkItem volumes are not
// available, return after setting error in item.
FsVolumeSpi source = getFsVolume(this.dataset, sourceVolUuid);
if (source == null) {
final String errMsg = "Disk Balancer - Unable to find source volume: "
+ pair.getDestVolBasePath();
LOG.error(errMsg);
item.setErrMsg(errMsg);
return;
}
FsVolumeSpi dest = getFsVolume(this.dataset, destVolUuuid);
if (dest == null) {
final String errMsg = "Disk Balancer - Unable to find dest volume: "
+ pair.getDestVolBasePath();
LOG.error(errMsg);
item.setErrMsg(errMsg);
return;
}
List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>(); List<FsVolumeSpi.BlockIterator> poolIters = new LinkedList<>();
startTime = Time.now(); startTime = Time.now();
item.setStartTime(startTime); item.setStartTime(startTime);
@ -977,7 +1053,7 @@ public class DiskBalancer {
// we are not able to find any blocks to copy. // we are not able to find any blocks to copy.
if (block == null) { if (block == null) {
LOG.error("No source blocks, exiting the copy. Source: {}, " + LOG.error("No source blocks, exiting the copy. Source: {}, " +
"dest:{}", source.getBasePath(), dest.getBasePath()); "Dest:{}", source.getBasePath(), dest.getBasePath());
this.setExitFlag(); this.setExitFlag();
continue; continue;
} }

View File

@ -20,6 +20,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.ReconfigurationException;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
@ -27,9 +28,15 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.balancer.TestBalancer; import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DiskBalancer;
import org.apache.hadoop.hdfs.server.datanode.DiskBalancer.DiskBalancerMover;
import org.apache.hadoop.hdfs.server.datanode.DiskBalancer.VolumePair;
import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkItem;
import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus; import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus;
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.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
@ -41,18 +48,30 @@ import org.apache.hadoop.hdfs.server.diskbalancer.datamodel.DiskBalancerVolume;
import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan; import org.apache.hadoop.hdfs.server.diskbalancer.planner.NodePlan;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
/** /**
* Test Disk Balancer. * Test Disk Balancer.
@ -185,6 +204,47 @@ public class TestDiskBalancer {
} }
} }
/**
* Test disk balancer behavior when one of the disks involved
* in balancing operation is removed after submitting the plan.
* @throws Exception
*/
@Test
public void testDiskBalancerWhenRemovingVolumes() throws Exception {
Configuration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
final int blockCount = 100;
final int blockSize = 1024;
final int diskCount = 2;
final int dataNodeCount = 1;
final int dataNodeIndex = 0;
final int sourceDiskIndex = 0;
MiniDFSCluster cluster = new ClusterBuilder()
.setBlockCount(blockCount)
.setBlockSize(blockSize)
.setDiskCount(diskCount)
.setNumDatanodes(dataNodeCount)
.setConf(conf)
.build();
try {
DataMover dataMover = new DataMover(cluster, dataNodeIndex,
sourceDiskIndex, conf, blockSize, blockCount);
dataMover.moveDataToSourceDisk();
NodePlan plan = dataMover.generatePlan();
dataMover.executePlanDuringDiskRemove(plan);
dataMover.verifyAllVolumesHaveData();
dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10);
} catch (Exception e) {
Assert.fail("Unexpected exception: " + e);
} finally {
cluster.shutdown();
}
}
/** /**
* Sets alll Disks capacity to size specified. * Sets alll Disks capacity to size specified.
* *
@ -446,6 +506,102 @@ public class TestDiskBalancer {
}, 1000, 100000); }, 1000, 100000);
} }
public void executePlanDuringDiskRemove(NodePlan plan) throws
IOException, TimeoutException, InterruptedException {
CountDownLatch createWorkPlanLatch = new CountDownLatch(1);
CountDownLatch removeDiskLatch = new CountDownLatch(1);
AtomicInteger errorCount = new AtomicInteger(0);
LOG.info("FSDataSet: " + node.getFSDataset());
final FsDatasetSpi<?> fsDatasetSpy = Mockito.spy(node.getFSDataset());
doAnswer(new Answer<Object>() {
public Object answer(InvocationOnMock invocation) {
try {
node.getFSDataset().moveBlockAcrossVolumes(
(ExtendedBlock)invocation.getArguments()[0],
(FsVolumeSpi) invocation.getArguments()[1]);
} catch (Exception e) {
errorCount.incrementAndGet();
}
return null;
}
}).when(fsDatasetSpy).moveBlockAcrossVolumes(
any(ExtendedBlock.class), any(FsVolumeSpi.class));
DiskBalancerMover diskBalancerMover = new DiskBalancerMover(
fsDatasetSpy, conf);
diskBalancerMover.setRunnable();
DiskBalancerMover diskBalancerMoverSpy = Mockito.spy(diskBalancerMover);
doAnswer(new Answer<Object>() {
public Object answer(InvocationOnMock invocation) {
createWorkPlanLatch.countDown();
LOG.info("Waiting for the disk removal!");
try {
removeDiskLatch.await();
} catch (InterruptedException e) {
LOG.info("Encountered " + e);
}
LOG.info("Got disk removal notification, resuming copyBlocks!");
diskBalancerMover.copyBlocks((VolumePair)(invocation
.getArguments()[0]), (DiskBalancerWorkItem)(invocation
.getArguments()[1]));
return null;
}
}).when(diskBalancerMoverSpy).copyBlocks(
any(VolumePair.class), any(DiskBalancerWorkItem.class));
DiskBalancer diskBalancer = new DiskBalancer(node.getDatanodeUuid(),
conf, diskBalancerMoverSpy);
List<String> oldDirs = new ArrayList<String>(node.getConf().
getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
final String newDirs = oldDirs.get(0);
LOG.info("Reconfigure newDirs:" + newDirs);
Thread reconfigThread = new Thread() {
public void run() {
try {
LOG.info("Waiting for work plan creation!");
createWorkPlanLatch.await();
LOG.info("Work plan created. Removing disk!");
assertThat(
"DN did not update its own config", node.
reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, newDirs),
is(node.getConf().get(DFS_DATANODE_DATA_DIR_KEY)));
Thread.sleep(1000);
LOG.info("Removed disk!");
removeDiskLatch.countDown();
} catch (ReconfigurationException | InterruptedException e) {
Assert.fail("Unexpected error while reconfiguring: " + e);
}
}
};
reconfigThread.start();
String planJson = plan.toJson();
String planID = DigestUtils.shaHex(planJson);
diskBalancer.submitPlan(planID, 1, PLAN_FILE, planJson, false);
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
try {
LOG.info("Work Status: " + diskBalancer.
queryWorkStatus().toJsonString());
Result result = diskBalancer.queryWorkStatus().getResult();
return (result == Result.PLAN_DONE);
} catch (IOException e) {
return false;
}
}
}, 1000, 100000);
assertTrue("Disk balancer operation hit max errors!", errorCount.get() <
DFSConfigKeys.DFS_DISK_BALANCER_MAX_DISK_ERRORS_DEFAULT);
createWorkPlanLatch.await();
removeDiskLatch.await();
}
/** /**
* Verifies the Plan Execution has been done. * Verifies the Plan Execution has been done.
*/ */