Browse Source

HDFS-13257. Code cleanup: INode never throws QuotaExceededException. Contributed by Tsz Wo Nicholas Sze.

Arpit Agarwal 7 years ago
parent
commit
4c57fb0cd9

+ 0 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Content.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import org.apache.hadoop.hdfs.util.EnumCounters;
-
 /**
  * The content types such as file, directory and symlink to be computed.
  */
@@ -39,34 +37,4 @@ public enum Content {
   SNAPSHOT,
   /** The number of snapshottable directories. */
   SNAPSHOTTABLE_DIRECTORY;
-
-  /** Content counts. */
-  public static class Counts extends EnumCounters<Content> {
-    public static Counts newInstance() {
-      return new Counts();
-    }
-
-    private Counts() {
-      super(Content.class);
-    }
-  }
-
-  private static final EnumCounters.Factory<Content, Counts> FACTORY
-      = new EnumCounters.Factory<Content, Counts>() {
-    @Override
-    public Counts newInstance() {
-      return Counts.newInstance();
-    }
-  };
-
-  /** A map of counters for the current state and the snapshots. */
-  public static class CountsMap
-      extends EnumCounters.Map<CountsMap.Key, Content, Counts> {
-    /** The key type of the map. */
-    public enum Key { CURRENT, SNAPSHOT }
-
-    CountsMap() {
-      super(FACTORY);
-    }
-  }
 }

+ 0 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DirectoryWithQuotaFeature.java

@@ -146,25 +146,6 @@ public final class DirectoryWithQuotaFeature implements INode.Feature {
     }
   }
 
-  void addSpaceConsumed(final INodeDirectory dir, final QuotaCounts counts,
-      boolean verify) throws QuotaExceededException {
-    if (dir.isQuotaSet()) {
-      // The following steps are important:
-      // check quotas in this inode and all ancestors before changing counts
-      // so that no change is made if there is any quota violation.
-      // (1) verify quota in this inode
-      if (verify) {
-        verifyQuota(counts);
-      }
-      // (2) verify quota and then add count in ancestors
-      dir.addSpaceConsumed2Parent(counts, verify);
-      // (3) add count in this inode
-      addSpaceConsumed2Cache(counts);
-    } else {
-      dir.addSpaceConsumed2Parent(counts, verify);
-    }
-  }
-  
   /** Update the space/namespace/type usage of the tree
    * 
    * @param delta the change of the namespace/space/type usage

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -689,13 +689,13 @@ class FSDirRenameOp {
       return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst);
     }
 
-    void updateMtimeAndLease(long timestamp) throws QuotaExceededException {
+    void updateMtimeAndLease(long timestamp) {
       srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshotId());
       final INode dstParent = dstParentIIP.getLastINode();
       dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshotId());
     }
 
-    void restoreSource() throws QuotaExceededException {
+    void restoreSource() {
       // Rename failed - restore src
       final INode oldSrcChild = srcChild;
       // put it back
@@ -722,7 +722,7 @@ class FSDirRenameOp {
       }
     }
 
-    void restoreDst(BlockStoragePolicySuite bsps) throws QuotaExceededException {
+    void restoreDst(BlockStoragePolicySuite bsps) {
       Preconditions.checkState(oldDstChild != null);
       final INodeDirectory dstParent = dstParentIIP.getLastINode().asDirectory();
       if (dstParent.isWithSnapshot()) {
@@ -738,8 +738,8 @@ class FSDirRenameOp {
       }
     }
 
-    boolean cleanDst(BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks)
-        throws QuotaExceededException {
+    boolean cleanDst(
+        BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks) {
       Preconditions.checkState(oldDstChild != null);
       List<INode> removedINodes = new ChunkedArrayList<>();
       List<Long> removedUCFiles = new ChunkedArrayList<>();
@@ -762,13 +762,13 @@ class FSDirRenameOp {
       return filesDeleted;
     }
 
-    void updateQuotasInSourceTree(BlockStoragePolicySuite bsps) throws QuotaExceededException {
+    void updateQuotasInSourceTree(BlockStoragePolicySuite bsps) {
       // update the quota usage in src tree
       if (isSrcInSnapshot) {
         // get the counts after rename
         QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps, false);
         newSrcCounts.subtract(oldSrcCounts);
-        srcParent.addSpaceConsumed(newSrcCounts, false);
+        srcParent.addSpaceConsumed(newSrcCounts);
       }
     }
   }

+ 2 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1293,13 +1293,8 @@ public class FSDirectory implements Closeable {
     updateCount(existing, pos, counts, checkQuota);
 
     boolean isRename = (inode.getParent() != null);
-    boolean added;
-    try {
-      added = parent.addChild(inode, true, existing.getLatestSnapshotId());
-    } catch (QuotaExceededException e) {
-      updateCountNoQuotaCheck(existing, pos, counts.negation());
-      throw e;
-    }
+    final boolean added = parent.addChild(inode, true,
+        existing.getLatestSnapshotId());
     if (!added) {
       updateCountNoQuotaCheck(existing, pos, counts.negation());
       return null;

+ 2 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -471,21 +470,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
   /**
    * Check and add namespace/storagespace/storagetype consumed to itself and the ancestors.
-   * @throws QuotaExceededException if quote is violated.
    */
-  public void addSpaceConsumed(QuotaCounts counts, boolean verify)
-    throws QuotaExceededException {
-    addSpaceConsumed2Parent(counts, verify);
-  }
-
-  /**
-   * Check and add namespace/storagespace/storagetype consumed to itself and the ancestors.
-   * @throws QuotaExceededException if quote is violated.
-   */
-  void addSpaceConsumed2Parent(QuotaCounts counts, boolean verify)
-    throws QuotaExceededException {
+  public void addSpaceConsumed(QuotaCounts counts) {
     if (parent != null) {
-      parent.addSpaceConsumed(counts, verify);
+      parent.addSpaceConsumed(counts);
     }
   }
 

+ 10 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
@@ -171,13 +170,12 @@ public class INodeDirectory extends INodeWithAdditionalFields
   }
 
   @Override
-  public void addSpaceConsumed(QuotaCounts counts, boolean verify)
-    throws QuotaExceededException {
+  public void addSpaceConsumed(QuotaCounts counts) {
+    super.addSpaceConsumed(counts);
+
     final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
-    if (q != null) {
-      q.addSpaceConsumed(this, counts, verify);
-    } else {
-      addSpaceConsumed2Parent(counts, verify);
+    if (q != null && isQuotaSet()) {
+      q.addSpaceConsumed2Cache(counts);
     }
   }
 
@@ -281,7 +279,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
   public Snapshot addSnapshot(int id, String name,
       final LeaseManager leaseManager, final boolean captureOpenFiles,
       int maxSnapshotLimit)
-      throws SnapshotException, QuotaExceededException {
+      throws SnapshotException {
     return getDirectorySnapshottableFeature().addSnapshot(this, id, name,
         leaseManager, captureOpenFiles, maxSnapshotLimit);
   }
@@ -543,7 +541,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    *         otherwise, return true;
    */
   public boolean addChild(INode node, final boolean setModTime,
-      final int latestSnapshotId) throws QuotaExceededException {
+      final int latestSnapshotId) {
     final int low = searchChildren(node.getLocalNameBytes());
     if (low >= 0) {
       return false;
@@ -739,10 +737,9 @@ public class INodeDirectory extends INodeWithAdditionalFields
    *          The reference node to be removed/replaced
    * @param newChild
    *          The node to be added back
-   * @throws QuotaExceededException should not throw this exception
    */
   public void undoRename4ScrParent(final INodeReference oldChild,
-      final INode newChild) throws QuotaExceededException {
+      final INode newChild) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     assert sf != null : "Directory does not have snapshot feature";
     sf.getDiffs().removeDeletedChild(oldChild);
@@ -756,8 +753,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
    * and delete possible record in the deleted list.  
    */
   public void undoRename4DstParent(final BlockStoragePolicySuite bsps,
-      final INode deletedChild,
-      int latestSnapshotId) throws QuotaExceededException {
+      final INode deletedChild, int latestSnapshotId) {
     DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
     assert sf != null : "Directory does not have snapshot feature";
     boolean removeDeletedChild = sf.getDiffs().removeDeletedChild(deletedChild);
@@ -767,8 +763,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
     // been stored in deleted list before
     if (added && !removeDeletedChild) {
       final QuotaCounts counts = deletedChild.computeQuotaUsage(bsps);
-      addSpaceConsumed(counts, false);
-
+      addSpaceConsumed(counts);
     }
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -30,7 +30,6 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.Content;
@@ -171,7 +170,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
   public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name,
       final LeaseManager leaseManager, final boolean captureOpenFiles,
       int maxSnapshotLimit)
-      throws SnapshotException, QuotaExceededException {
+      throws SnapshotException {
     //check snapshot quota
     final int n = getNumSnapshots();
     if (n + 1 > snapshotQuota) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.*;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -524,7 +523,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
    * to make sure that parent is in the given snapshot "latest".
    */
   public boolean addChild(INodeDirectory parent, INode inode,
-      boolean setModTime, int latestSnapshotId) throws QuotaExceededException {
+      boolean setModTime, int latestSnapshotId) {
     ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latestSnapshotId,
         parent).diff;
     final int undoInfo = diff.create(inode);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -120,7 +120,7 @@ public class FSImageFormatPBSnapshot {
     }
 
     private INodeReference loadINodeReference(
-        INodeReferenceSection.INodeReference r) throws IOException {
+        INodeReferenceSection.INodeReference r) {
       long referredId = r.getReferredId();
       INode referred = fsDir.getInode(referredId);
       WithCount withCount = (WithCount) referred.getParentReference();

+ 3 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java

@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hdfs.util;
 
-import java.util.Arrays;
-import java.util.HashMap;
-
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.ArrayUtils;
 
+import java.util.Arrays;
+
 /**
  * Counters for an enum type.
  * 
@@ -31,7 +30,7 @@ import org.apache.commons.lang.ArrayUtils;
  * enum Fruit { APPLE, ORANGE, GRAPE }
  * </pre>
  * An {@link EnumCounters} object can be created for counting the numbers of
- * APPLE, ORANGLE and GRAPE.
+ * APPLE, ORANGE and GRAPE.
  *
  * @param <E> the enum type
  */
@@ -178,69 +177,4 @@ public class EnumCounters<E extends Enum<E>> {
     }
     return false;
   }
-
-  /**
-   * A factory for creating counters.
-   * 
-   * @param <E> the enum type
-   * @param <C> the counter type
-   */
-  public static interface Factory<E extends Enum<E>,
-                                  C extends EnumCounters<E>> {
-    /** Create a new counters instance. */
-    public C newInstance(); 
-  }
-
-  /**
-   * A key-value map which maps the keys to {@link EnumCounters}.
-   * Note that null key is supported.
-   *
-   * @param <K> the key type
-   * @param <E> the enum type
-   * @param <C> the counter type
-   */
-  public static class Map<K, E extends Enum<E>, C extends EnumCounters<E>> {
-    /** The factory for creating counters. */
-    private final Factory<E, C> factory;
-    /** Key-to-Counts map. */
-    private final java.util.Map<K, C> counts = new HashMap<K, C>();
-    
-    /** Construct a map. */
-    public Map(final Factory<E, C> factory) {
-      this.factory = factory;
-    }
-
-    /** @return the counters for the given key. */
-    public final C getCounts(final K key) {
-      C c = counts.get(key);
-      if (c == null) {
-        c = factory.newInstance();
-        counts.put(key, c); 
-      }
-      return c;
-    }
-    
-    /** @return the sum of the values of all the counters. */
-    public final C sum() {
-      final C sum = factory.newInstance();
-      for(C c : counts.values()) {
-        sum.add(c);
-      }
-      return sum;
-    }
-    
-    /** @return the sum of the values of all the counters for e. */
-    public final long sum(final E e) {
-      long sum = 0;
-      for(C c : counts.values()) {
-        sum += c.get(e);
-      }
-      return sum;
-    }
-
-    @Override
-    public String toString() {
-      return counts.toString();
-    }
-  }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java

@@ -132,7 +132,7 @@ public class TestQuota {
   }
 
   @AfterClass
-  public static void tearDownClass() throws Exception {
+  public static void tearDownClass() {
     try {
       System.out.flush();
       System.err.flush();
@@ -170,7 +170,7 @@ public class TestQuota {
    * @throws Exception
    */
   @Test
-  public void testDSQuotaExceededExceptionIsHumanReadable() throws Exception {
+  public void testDSQuotaExceededExceptionIsHumanReadable() {
     Integer bytes = 1024;
     try {
       throw new DSQuotaExceededException(bytes, bytes);