|
@@ -18,10 +18,20 @@
|
|
|
package org.apache.hadoop.hdfs.server.diskbalancer;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+
|
|
|
+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.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
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.impl.FsVolumeImpl;
|
|
|
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;
|
|
@@ -29,9 +39,12 @@ 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 static org.junit.Assert.assertTrue;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.util.Random;
|
|
|
import java.util.UUID;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
/**
|
|
|
* Helper class to create various cluster configrations at run time.
|
|
@@ -242,6 +255,65 @@ public class DiskBalancerTestUtil {
|
|
|
return count;
|
|
|
}
|
|
|
|
|
|
+ public static MiniDFSCluster newImbalancedCluster(
|
|
|
+ final Configuration conf,
|
|
|
+ final int numDatanodes,
|
|
|
+ final long[] storageCapacities,
|
|
|
+ final int defaultBlockSize,
|
|
|
+ final int fileLen)
|
|
|
+ throws IOException, InterruptedException, TimeoutException {
|
|
|
+ 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 String fileName = "/" + UUID.randomUUID().toString();
|
|
|
+ final Path filePath = new Path(fileName);
|
|
|
+
|
|
|
+ Preconditions.checkNotNull(storageCapacities);
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ storageCapacities.length == 2,
|
|
|
+ "need to specify capacities for two storages.");
|
|
|
+
|
|
|
+ // Write a file and restart the cluster
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(numDatanodes)
|
|
|
+ .storageCapacities(storageCapacities)
|
|
|
+ .storageTypes(new StorageType[]{StorageType.DISK, StorageType.DISK})
|
|
|
+ .storagesPerDatanode(2)
|
|
|
+ .build();
|
|
|
+ FsVolumeImpl source = null;
|
|
|
+ FsVolumeImpl dest = null;
|
|
|
+
|
|
|
+ cluster.waitActive();
|
|
|
+ Random r = new Random();
|
|
|
+ FileSystem fs = cluster.getFileSystem(0);
|
|
|
+ TestBalancer.createFile(cluster, filePath, fileLen, (short) 1, 0);
|
|
|
+
|
|
|
+ DFSTestUtil.waitReplication(fs, filePath, (short) 1);
|
|
|
+ cluster.restartDataNodes();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ // Get the data node and move all data to one disk.
|
|
|
+ for (int i = 0; i < numDatanodes; i++) {
|
|
|
+ DataNode dnNode = cluster.getDataNodes().get(i);
|
|
|
+ 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();
|
|
|
+
|
|
|
+ return cluster;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Moves all blocks to the destination volume.
|
|
|
*
|