|
@@ -25,16 +25,25 @@ import static org.junit.Assume.assumeTrue;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Random;
|
|
|
+import java.util.UUID;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
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.MiniDFSCluster.NameNodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
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.impl.FsVolumeImpl;
|
|
|
import org.apache.hadoop.test.PathUtils;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+
|
|
|
/**
|
|
|
* Tests MiniDFS cluster setup/teardown and isolation.
|
|
|
* Every instance is brought up with a new data dir, to ensure that
|
|
@@ -78,6 +87,116 @@ public class TestMiniDFSCluster {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Tests storage capacity setting still effective after cluster restart.
|
|
|
+ */
|
|
|
+ @Test(timeout=100000)
|
|
|
+ public void testClusterSetStorageCapacity() throws Throwable {
|
|
|
+
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ final int numDatanodes = 1;
|
|
|
+ final int defaultBlockSize = 1024;
|
|
|
+ final int blocks = 100;
|
|
|
+ final int blocksSize = 1024;
|
|
|
+ final int fileLen = blocks * blocksSize;
|
|
|
+ final long capcacity = defaultBlockSize * 2 * fileLen;
|
|
|
+ final long[] capacities = new long[] {capcacity, 2 * capcacity};
|
|
|
+
|
|
|
+ final MiniDFSCluster cluster = newCluster(
|
|
|
+ conf,
|
|
|
+ numDatanodes,
|
|
|
+ capacities,
|
|
|
+ defaultBlockSize,
|
|
|
+ fileLen);
|
|
|
+ verifyStorageCapacity(cluster, capacities);
|
|
|
+
|
|
|
+ /* restart all data nodes */
|
|
|
+ cluster.restartDataNodes();
|
|
|
+ cluster.waitActive();
|
|
|
+ verifyStorageCapacity(cluster, capacities);
|
|
|
+
|
|
|
+ /* restart all name nodes */
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ cluster.waitActive();
|
|
|
+ verifyStorageCapacity(cluster, capacities);
|
|
|
+
|
|
|
+ /* restart all name nodes firstly and data nodes then */
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ cluster.restartDataNodes();
|
|
|
+ cluster.waitActive();
|
|
|
+ verifyStorageCapacity(cluster, capacities);
|
|
|
+
|
|
|
+ /* restart all data nodes firstly and name nodes then */
|
|
|
+ cluster.restartDataNodes();
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ cluster.waitActive();
|
|
|
+ verifyStorageCapacity(cluster, capacities);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyStorageCapacity(
|
|
|
+ final MiniDFSCluster cluster,
|
|
|
+ final long[] capacities) throws IOException {
|
|
|
+
|
|
|
+ FsVolumeImpl source = null;
|
|
|
+ FsVolumeImpl dest = null;
|
|
|
+
|
|
|
+ /* verify capacity */
|
|
|
+ for (int i = 0; i < cluster.getDataNodes().size(); i++) {
|
|
|
+ final DataNode dnNode = cluster.getDataNodes().get(i);
|
|
|
+ try (FsDatasetSpi.FsVolumeReferences refs = dnNode.getFSDataset()
|
|
|
+ .getFsVolumeReferences()) {
|
|
|
+ source = (FsVolumeImpl) refs.get(0);
|
|
|
+ dest = (FsVolumeImpl) refs.get(1);
|
|
|
+ assertEquals(capacities[0], source.getCapacity());
|
|
|
+ assertEquals(capacities[1], dest.getCapacity());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private MiniDFSCluster newCluster(
|
|
|
+ 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();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ final short replicationFactor = (short) 1;
|
|
|
+ final Random r = new Random();
|
|
|
+ FileSystem fs = cluster.getFileSystem(0);
|
|
|
+ DFSTestUtil.createFile(
|
|
|
+ fs,
|
|
|
+ filePath,
|
|
|
+ fileLen,
|
|
|
+ replicationFactor,
|
|
|
+ r.nextLong());
|
|
|
+ DFSTestUtil.waitReplication(fs, filePath, replicationFactor);
|
|
|
+
|
|
|
+ return cluster;
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=100000)
|
|
|
public void testIsClusterUpAfterShutdown() throws Throwable {
|
|
|
Configuration conf = new HdfsConfiguration();
|