فهرست منبع

HDFS-7776. Adding additional unit tests for Quota By Storage Type. (Contributed by Xiaoyu Yao)

Arpit Agarwal 10 سال پیش
والد
کامیت
e9c5d7ee71

+ 16 - 5
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -292,9 +292,6 @@ Trunk (Unreleased)
     HDFS-7721. The HDFS BlockScanner may run fast during the first hour
     (cmccabe)
 
-    HDFS-7751. Fix TestHDFSCLI for quota with storage type.  (Xiaoyu Yao
-    via szetszwo)
-
     HDFS-7670. HDFS Quota guide has typos, incomplete command lines
     (Brahma Reddy Battula via aw)
 
@@ -340,8 +337,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-6133. Add a feature for replica pinning so that a pinned replica
     will not be moved by Balancer/Mover.  (zhaoyunjiong via szetszwo)
 
-    HDFS-7584. Enable Quota Support for Storage Types. (Xiaoyu Yao via
-    Arpit Agarwal)
+    HDFS-7584. Enable Quota Support for Storage Types (See breakdown of
+    tasks below)
 
   IMPROVEMENTS
 
@@ -948,6 +945,20 @@ Release 2.7.0 - UNRELEASED
     HDFS-7704. DN heartbeat to Active NN may be blocked and expire if
     connection to Standby NN continues to time out (Rushabh Shah via kihwal)
 
+    BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
+
+      HDFS-7720. Quota by Storage Type API, tools and ClientNameNode
+      Protocol changes. (Xiaoyu Yao via Arpit Agarwal)
+
+      HDFS-7723. Quota By Storage Type namenode implemenation. (Xiaoyu Yao
+      via Arpit Agarwal)
+
+      HDFS-7751. Fix TestHDFSCLI for quota with storage type.  (Xiaoyu Yao
+      via szetszwo)
+
+      HDFS-7776. Adding additional unit tests for Quota By Storage Type.
+      (Xiaoyu Yao via Arpit Agarwal)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 183 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java

@@ -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);
+
+  }
 }