|
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.StorageType.DEFAULT;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
- import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
@@ -30,15 +29,18 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
import org.apache.hadoop.hdfs.*;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
|
|
+ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
+ import java.io.IOException;
|
|
|
+
|
|
|
public class TestQuotaByStorageType {
|
|
|
|
|
|
private static final int BLOCKSIZE = 1024;
|
|
|
private static final short REPLICATION = 3;
|
|
|
- static final long seed = 0L;
|
|
|
+ private static final long seed = 0L;
|
|
|
private static final Path dir = new Path("/TestQuotaByStorageType");
|
|
|
|
|
|
private Configuration conf;
|
|
@@ -219,7 +221,6 @@ public class TestQuotaByStorageType {
|
|
|
// Verify space consumed and remaining quota
|
|
|
long ssdConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
|
|
|
.getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
- ;
|
|
|
assertEquals(file1Len, ssdConsumed);
|
|
|
|
|
|
// move file from foo to bar
|
|
@@ -356,7 +357,6 @@ public class TestQuotaByStorageType {
|
|
|
assertTrue(fnode.isQuotaSet());
|
|
|
long currentSSDConsumed = fnode.asDirectory().getDirectoryWithQuotaFeature()
|
|
|
.getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
- ;
|
|
|
assertEquals(file1Len, currentSSDConsumed);
|
|
|
|
|
|
// Create the 2nd file of size BLOCKSIZE under child directory and expect quota exceeded exception
|
|
@@ -399,11 +399,19 @@ public class TestQuotaByStorageType {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Both traditional space quota and the storage type quota for SSD are set and
|
|
|
+ * not exceeded.
|
|
|
+ */
|
|
|
@Test(timeout = 60000)
|
|
|
public void testQuotaByStorageTypeWithTraditionalQuota() throws Exception {
|
|
|
final Path foo = new Path(dir, "foo");
|
|
|
dfs.mkdirs(foo);
|
|
|
+
|
|
|
+ dfs.setStoragePolicy(foo, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+ dfs.setQuotaByStorageType(foo, StorageType.SSD, BLOCKSIZE * 10);
|
|
|
dfs.setQuota(foo, Long.MAX_VALUE - 1, REPLICATION * BLOCKSIZE * 10);
|
|
|
+
|
|
|
INode fnode = fsdir.getINode4Write(foo.toString());
|
|
|
assertTrue(fnode.isDirectory());
|
|
|
assertTrue(fnode.isQuotaSet());
|
|
@@ -434,6 +442,76 @@ public class TestQuotaByStorageType {
|
|
|
counts.getDiskSpace());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Both traditional space quota and the storage type quota for SSD are set and
|
|
|
+ * exceeded. expect DSQuotaExceededException is thrown as we check traditional
|
|
|
+ * space quota first and then storage type quota.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testQuotaByStorageTypeAndTraditionalQuotaException1()
|
|
|
+ throws Exception {
|
|
|
+ testQuotaByStorageTypeOrTraditionalQuotaExceededCase(
|
|
|
+ 4 * REPLICATION, 4, 5, REPLICATION);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Both traditional space quota and the storage type quota for SSD are set and
|
|
|
+ * SSD quota is exceeded but traditional space quota is not exceeded.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testQuotaByStorageTypeAndTraditionalQuotaException2()
|
|
|
+ throws Exception {
|
|
|
+ testQuotaByStorageTypeOrTraditionalQuotaExceededCase(
|
|
|
+ 5 * REPLICATION, 4, 5, REPLICATION);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Both traditional space quota and the storage type quota for SSD are set and
|
|
|
+ * traditional space quota is exceeded but SSD quota is not exceeded.
|
|
|
+ */
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testQuotaByStorageTypeAndTraditionalQuotaException3()
|
|
|
+ throws Exception {
|
|
|
+ testQuotaByStorageTypeOrTraditionalQuotaExceededCase(
|
|
|
+ 4 * REPLICATION, 5, 5, REPLICATION);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testQuotaByStorageTypeOrTraditionalQuotaExceededCase(
|
|
|
+ long storageSpaceQuotaInBlocks, long ssdQuotaInBlocks,
|
|
|
+ long testFileLenInBlocks, short replication) throws Exception {
|
|
|
+ final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
+ final Path testDir = new Path(dir, METHOD_NAME);
|
|
|
+
|
|
|
+ dfs.mkdirs(testDir);
|
|
|
+ dfs.setStoragePolicy(testDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+
|
|
|
+ final long ssdQuota = BLOCKSIZE * ssdQuotaInBlocks;
|
|
|
+ final long storageSpaceQuota = BLOCKSIZE * storageSpaceQuotaInBlocks;
|
|
|
+
|
|
|
+ dfs.setQuota(testDir, Long.MAX_VALUE - 1, storageSpaceQuota);
|
|
|
+ dfs.setQuotaByStorageType(testDir, StorageType.SSD, ssdQuota);
|
|
|
+
|
|
|
+ INode testDirNode = fsdir.getINode4Write(testDir.toString());
|
|
|
+ assertTrue(testDirNode.isDirectory());
|
|
|
+ assertTrue(testDirNode.isQuotaSet());
|
|
|
+
|
|
|
+ Path createdFile = new Path(testDir, "created_file.data");
|
|
|
+ long fileLen = testFileLenInBlocks * BLOCKSIZE;
|
|
|
+
|
|
|
+ try {
|
|
|
+ DFSTestUtil.createFile(dfs, createdFile, BLOCKSIZE / 16,
|
|
|
+ fileLen, BLOCKSIZE, replication, seed);
|
|
|
+ fail("Should have failed with DSQuotaExceededException or " +
|
|
|
+ "QuotaByStorageTypeExceededException ");
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.info("Got expected exception ", t);
|
|
|
+ long currentSSDConsumed = testDirNode.asDirectory().getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
+ assertEquals(Math.min(ssdQuota, storageSpaceQuota/replication),
|
|
|
+ currentSSDConsumed);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 60000)
|
|
|
public void testQuotaByStorageTypeWithSnapshot() throws Exception {
|
|
|
final Path sub1 = new Path(dir, "Sub1");
|
|
@@ -521,4 +599,105 @@ public class TestQuotaByStorageType {
|
|
|
.getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
assertEquals(newFile1Len, ssdConsumed);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQuotaByStorageTypePersistenceInEditLog() throws IOException {
|
|
|
+ final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
+ final Path testDir = new Path(dir, METHOD_NAME);
|
|
|
+ Path createdFile1 = new Path(testDir, "created_file1.data");
|
|
|
+ dfs.mkdirs(testDir);
|
|
|
+
|
|
|
+ // set storage policy on testDir to ONESSD
|
|
|
+ dfs.setStoragePolicy(testDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+
|
|
|
+ // set quota by storage type on testDir
|
|
|
+ final long SSD_QUOTA = BLOCKSIZE * 4;
|
|
|
+ dfs.setQuotaByStorageType(testDir, StorageType.SSD, SSD_QUOTA);
|
|
|
+ INode testDirNode = fsdir.getINode4Write(testDir.toString());
|
|
|
+ assertTrue(testDirNode.isDirectory());
|
|
|
+ assertTrue(testDirNode.isQuotaSet());
|
|
|
+
|
|
|
+ // Create file of size 2 * BLOCKSIZE under testDir
|
|
|
+ long file1Len = BLOCKSIZE * 2;
|
|
|
+ int bufLen = BLOCKSIZE / 16;
|
|
|
+ DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
|
|
|
+
|
|
|
+ // Verify SSD consumed before namenode restart
|
|
|
+ long ssdConsumed = testDirNode.asDirectory().getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
+ assertEquals(file1Len, ssdConsumed);
|
|
|
+
|
|
|
+ // Restart namenode to make sure the editlog is correct
|
|
|
+ cluster.restartNameNode(true);
|
|
|
+
|
|
|
+ INode testDirNodeAfterNNRestart = fsdir.getINode4Write(testDir.toString());
|
|
|
+ // Verify quota is still set
|
|
|
+ assertTrue(testDirNode.isDirectory());
|
|
|
+ assertTrue(testDirNode.isQuotaSet());
|
|
|
+
|
|
|
+ QuotaCounts qc = testDirNodeAfterNNRestart.getQuotaCounts();
|
|
|
+ assertEquals(SSD_QUOTA, qc.getTypeSpace(StorageType.SSD));
|
|
|
+ for (StorageType t: StorageType.getTypesSupportingQuota()) {
|
|
|
+ if (t != StorageType.SSD) {
|
|
|
+ assertEquals(HdfsConstants.QUOTA_RESET, qc.getTypeSpace(t));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ long ssdConsumedAfterNNRestart = testDirNodeAfterNNRestart.asDirectory()
|
|
|
+ .getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
+ assertEquals(file1Len, ssdConsumedAfterNNRestart);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testQuotaByStorageTypePersistenceInFsImage() throws IOException {
|
|
|
+ final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
+ final Path testDir = new Path(dir, METHOD_NAME);
|
|
|
+ Path createdFile1 = new Path(testDir, "created_file1.data");
|
|
|
+ dfs.mkdirs(testDir);
|
|
|
+
|
|
|
+ // set storage policy on testDir to ONESSD
|
|
|
+ dfs.setStoragePolicy(testDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
|
|
|
+
|
|
|
+ // set quota by storage type on testDir
|
|
|
+ final long SSD_QUOTA = BLOCKSIZE * 4;
|
|
|
+ dfs.setQuotaByStorageType(testDir, StorageType.SSD, SSD_QUOTA);
|
|
|
+ INode testDirNode = fsdir.getINode4Write(testDir.toString());
|
|
|
+ assertTrue(testDirNode.isDirectory());
|
|
|
+ assertTrue(testDirNode.isQuotaSet());
|
|
|
+
|
|
|
+ // Create file of size 2 * BLOCKSIZE under testDir
|
|
|
+ long file1Len = BLOCKSIZE * 2;
|
|
|
+ int bufLen = BLOCKSIZE / 16;
|
|
|
+ DFSTestUtil.createFile(dfs, createdFile1, bufLen, file1Len, BLOCKSIZE, REPLICATION, seed);
|
|
|
+
|
|
|
+ // Verify SSD consumed before namenode restart
|
|
|
+ long ssdConsumed = testDirNode.asDirectory().getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
+ assertEquals(file1Len, ssdConsumed);
|
|
|
+
|
|
|
+ // Restart the namenode with checkpoint to make sure fsImage is correct
|
|
|
+ dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
|
|
|
+ dfs.saveNamespace();
|
|
|
+ dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
|
|
|
+ cluster.restartNameNode(true);
|
|
|
+
|
|
|
+ INode testDirNodeAfterNNRestart = fsdir.getINode4Write(testDir.toString());
|
|
|
+ assertTrue(testDirNode.isDirectory());
|
|
|
+ assertTrue(testDirNode.isQuotaSet());
|
|
|
+
|
|
|
+ QuotaCounts qc = testDirNodeAfterNNRestart.getQuotaCounts();
|
|
|
+ assertEquals(SSD_QUOTA, qc.getTypeSpace(StorageType.SSD));
|
|
|
+ for (StorageType t: StorageType.getTypesSupportingQuota()) {
|
|
|
+ if (t != StorageType.SSD) {
|
|
|
+ assertEquals(HdfsConstants.QUOTA_RESET, qc.getTypeSpace(t));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ long ssdConsumedAfterNNRestart = testDirNodeAfterNNRestart.asDirectory()
|
|
|
+ .getDirectoryWithQuotaFeature()
|
|
|
+ .getSpaceConsumed().getTypeSpaces().get(StorageType.SSD);
|
|
|
+ assertEquals(file1Len, ssdConsumedAfterNNRestart);
|
|
|
+
|
|
|
+ }
|
|
|
}
|