|
@@ -27,21 +27,28 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.NameNodeProxies;
|
|
|
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
-
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.rules.Timeout;
|
|
@@ -287,4 +294,82 @@ public class TestStripedINodeFile {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests when choosing blocks on file creation of EC striped mode should
|
|
|
+ * ignore storage policy if that is not suitable. Supported storage policies
|
|
|
+ * for EC Striped mode are HOT, COLD and ALL_SSD. For all other policies set
|
|
|
+ * will be ignored and considered default policy.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testUnsuitableStoragePoliciesWithECStripedMode()
|
|
|
+ throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ int defaultStripedBlockSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE
|
|
|
+ * 4;
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, defaultStripedBlockSize);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY,
|
|
|
+ false);
|
|
|
+
|
|
|
+ // start 10 datanodes
|
|
|
+ int numOfDatanodes = 10;
|
|
|
+ int storagesPerDatanode = 2;
|
|
|
+ long capacity = 10 * defaultStripedBlockSize;
|
|
|
+ long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
|
|
|
+ for (int i = 0; i < numOfDatanodes; i++) {
|
|
|
+ for (int j = 0; j < storagesPerDatanode; j++) {
|
|
|
+ capacities[i][j] = capacity;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(numOfDatanodes).storagesPerDatanode(storagesPerDatanode)
|
|
|
+ .storageTypes(
|
|
|
+ new StorageType[][] { { StorageType.SSD, StorageType.DISK },
|
|
|
+ { StorageType.SSD, StorageType.DISK },
|
|
|
+ { StorageType.SSD, StorageType.DISK },
|
|
|
+ { StorageType.SSD, StorageType.DISK },
|
|
|
+ { StorageType.SSD, StorageType.DISK },
|
|
|
+ { StorageType.DISK, StorageType.SSD },
|
|
|
+ { StorageType.DISK, StorageType.SSD },
|
|
|
+ { StorageType.DISK, StorageType.SSD },
|
|
|
+ { StorageType.DISK, StorageType.SSD },
|
|
|
+ { StorageType.DISK, StorageType.SSD } })
|
|
|
+ .storageCapacities(capacities).build();
|
|
|
+
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ // set "/foo" directory with ONE_SSD storage policy.
|
|
|
+ ClientProtocol client = NameNodeProxies.createProxy(conf,
|
|
|
+ cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
|
|
|
+ String fooDir = "/foo";
|
|
|
+ client.mkdirs(fooDir, new FsPermission((short) 777), true);
|
|
|
+ client.setStoragePolicy(fooDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+ // set an EC policy on "/foo" directory
|
|
|
+ client.setErasureCodingPolicy(fooDir, null);
|
|
|
+
|
|
|
+ // write file to fooDir
|
|
|
+ final String barFile = "/foo/bar";
|
|
|
+ long fileLen = 20 * defaultStripedBlockSize;
|
|
|
+ DFSTestUtil.createFile(cluster.getFileSystem(), new Path(barFile),
|
|
|
+ fileLen, (short) 3, 0);
|
|
|
+
|
|
|
+ // verify storage types and locations
|
|
|
+ LocatedBlocks locatedBlocks = client.getBlockLocations(barFile, 0,
|
|
|
+ fileLen);
|
|
|
+ for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
|
|
|
+ for (StorageType type : lb.getStorageTypes()) {
|
|
|
+ Assert.assertEquals(StorageType.DISK, type);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|