Browse Source

HDFS-4666. Define ".snapshot" as a reserved inode name so that users cannot create a file/directory with ".snapshot" as the name. If ".snapshot" is used in a previous version of HDFS, it must be renamed before upgrade; otherwise, upgrade will fail.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1468238 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
d13f6ebe20
14 changed files with 316 additions and 305 deletions
  1. 5 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java
  3. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  5. 39 25
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  6. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  7. 7 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  8. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  9. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  11. 11 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java
  12. 8 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
  13. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  14. 204 246
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt

@@ -236,3 +236,8 @@ Branch-2802 Snapshot (Unreleased)
   szetszwo)
 
   HDFS-4692. Use timestamp as default snapshot names.  (szetszwo)
+
+  HDFS-4666. Define ".snapshot" as a reserved inode name so that users cannot
+  create a file/directory with ".snapshot" as the name.  If ".snapshot" is used
+  in a previous version of HDFS, it must be renamed before upgrade; otherwise,
+  upgrade will fail.  (szetszwo)

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FSLimitException.java

@@ -97,4 +97,15 @@ public abstract class FSLimitException extends QuotaExceededException {
       " is exceeded: limit=" + quota + " items=" + count; 
     }
   }
+
+  /** The given name is illegal. */
+  public static final class IllegalNameException extends FSLimitException {
+    public static final long serialVersionUID = 1L;
+    
+    public IllegalNameException() {}
+
+    public IllegalNameException(String msg) {
+      super(msg);
+    }
+  }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -18,7 +18,9 @@
 package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 
 /************************************
@@ -107,4 +109,10 @@ public class HdfsConstants {
    * A special path component contained in the path for a snapshot file/dir
    */
   public static final String DOT_SNAPSHOT_DIR = ".snapshot";
+
+  public static final byte[] DOT_SNAPSHOT_DIR_BYTES
+      = DFSUtil.string2Bytes(DOT_SNAPSHOT_DIR);
+  
+  public static final String SEPARATOR_DOT_SNAPSHOT_DIR
+      = Path.SEPARATOR + DOT_SNAPSHOT_DIR; 
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -98,7 +98,7 @@ public class LayoutVersion {
         "add OP_UPDATE_BLOCKS"),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
     ADD_INODE_ID(-42, -40, "Assign a unique inode id for each inode", false),
-    SNAPSHOT(-43, -42, "Support for snapshot feature", false);
+    SNAPSHOT(-43, "Support for snapshot feature");
     
     final int lv;
     final int ancestorLV;

+ 39 - 25
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -23,6 +23,7 @@ import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
@@ -41,11 +42,12 @@ import org.apache.hadoop.fs.permission.FsAction;
 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.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.FSLimitException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.IllegalNameException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -1355,7 +1357,7 @@ public class FSDirectory implements Closeable {
 
     readLock();
     try {
-      if (srcs.endsWith(Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR)) {
+      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getSnapshotsListing(srcs, startAfter);
       }
       final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, true);
@@ -1393,10 +1395,10 @@ public class FSDirectory implements Closeable {
    */
   private DirectoryListing getSnapshotsListing(String src, byte[] startAfter)
       throws UnresolvedLinkException, IOException {
-    assert hasReadLock();
-    final String dotSnapshot = Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR;
-    Preconditions.checkArgument(src.endsWith(dotSnapshot), 
-        src + " does not end with " + dotSnapshot);
+    Preconditions.checkState(hasReadLock());
+    Preconditions.checkArgument(
+        src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR), 
+        "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
     
     final String dirPath = normalizePath(src.substring(0,
         src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
@@ -1428,7 +1430,7 @@ public class FSDirectory implements Closeable {
     String srcs = normalizePath(src);
     readLock();
     try {
-      if (srcs.endsWith(Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR)) {
+      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
         return getFileInfo4DotSnapshot(srcs);
       }
       final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, resolveLink);
@@ -1442,9 +1444,9 @@ public class FSDirectory implements Closeable {
   
   private HdfsFileStatus getFileInfo4DotSnapshot(String src)
       throws UnresolvedLinkException {
-    final String dotSnapshot = Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR;
-    Preconditions.checkArgument(src.endsWith(dotSnapshot), 
-        src + " does not end with " + dotSnapshot);
+    Preconditions.checkArgument(
+        src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR), 
+        "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
     
     final String dirPath = normalizePath(src.substring(0,
         src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
@@ -1927,38 +1929,47 @@ public class FSDirectory implements Closeable {
     verifyQuota(dst, dstIndex, delta.get(Quota.NAMESPACE),
         delta.get(Quota.DISKSPACE), src[i - 1]);
   }
+
+  /** Verify if the snapshot name is legal. */
+  void verifySnapshotName(String snapshotName, String path)
+      throws PathComponentTooLongException, IllegalNameException {
+    final byte[] bytes = DFSUtil.string2Bytes(snapshotName);
+    verifyINodeName(bytes);
+    verifyMaxComponentLength(bytes, path, 0);
+  }
   
-  /**
-   * Verify that filesystem limit constraints are not violated
-   */
-  void verifyFsLimits(INode[] pathComponents, int pos, INode child)
-      throws FSLimitException {
-    verifyMaxComponentLength(child.getLocalName(), pathComponents, pos);
-    verifyMaxDirItems(pathComponents, pos);
+  /** Verify if the inode name is legal. */
+  void verifyINodeName(byte[] childName) throws IllegalNameException {
+    if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
+      String s = "\"" + HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name.";
+      if (!ready) {
+        s += "  Please rename it before upgrade.";
+      }
+      throw new IllegalNameException(s);
+    }
   }
 
   /**
    * Verify child's name for fs limit.
    * @throws PathComponentTooLongException child's name is too long.
    */
-  public void verifyMaxComponentLength(String childName,
-      Object parentPath, int pos) throws PathComponentTooLongException {
+  void verifyMaxComponentLength(byte[] childName, Object parentPath, int pos)
+      throws PathComponentTooLongException {
     if (maxComponentLength == 0) {
       return;
     }
 
-    final int length = childName.length();
+    final int length = childName.length;
     if (length > maxComponentLength) {
       final String p = parentPath instanceof INode[]?
           getFullPathName((INode[])parentPath, pos - 1): (String)parentPath;
       final PathComponentTooLongException e = new PathComponentTooLongException(
-          maxComponentLength, length, p, childName);
+          maxComponentLength, length, p, DFSUtil.bytes2String(childName));
       if (ready) {
         throw e;
       } else {
         // Do not throw if edits log is still being processed
-        NameNode.LOG.error("FSDirectory.verifyMaxComponentLength: "
-            + e.getLocalizedMessage());
+        NameNode.LOG.error("ERROR in FSDirectory.verifyINodeName", e);
       }
     }
   }
@@ -1967,7 +1978,7 @@ public class FSDirectory implements Closeable {
    * Verify children size for fs limit.
    * @throws MaxDirectoryItemsExceededException too many children.
    */
-  private void verifyMaxDirItems(INode[] pathComponents, int pos)
+  void verifyMaxDirItems(INode[] pathComponents, int pos)
       throws MaxDirectoryItemsExceededException {
     if (maxDirItems == 0) {
       return;
@@ -2015,8 +2026,11 @@ public class FSDirectory implements Closeable {
     // original location becase a quota violation would cause the the item
     // to go "poof".  The fs limits must be bypassed for the same reason.
     if (checkQuota) {
-      verifyFsLimits(inodes, pos, child);
+      verifyMaxComponentLength(child.getLocalNameBytes(), inodes, pos);
+      verifyMaxDirItems(inodes, pos);
     }
+    // always verify inode name
+    verifyINodeName(child.getLocalNameBytes());
     
     final Quota.Counts counts = child.computeQuotaUsage();
     updateCount(iip, pos,

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

@@ -565,6 +565,10 @@ public class FSImageFormat {
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in) throws IOException {
     final int imgVersion = getLayoutVersion();
+    if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+      namesystem.getFSDirectory().verifyINodeName(localName);
+    }
+
     long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
            in.readLong() : namesystem.allocateNewInodeId();
     
@@ -903,7 +907,7 @@ public class FSImageFormat {
      *                 actually leads to.
      * @return The snapshot path.                
      */
-    private String computeSnapshotPath(String nonSnapshotPath, 
+    private static String computeSnapshotPath(String nonSnapshotPath, 
         Snapshot snapshot) {
       String snapshotParentFullPath = snapshot.getRoot().getParent()
           .getFullPathName();
@@ -911,10 +915,8 @@ public class FSImageFormat {
       String relativePath = nonSnapshotPath.equals(snapshotParentFullPath) ? 
           Path.SEPARATOR : nonSnapshotPath.substring(
                snapshotParentFullPath.length());
-      String snapshotFullPath = snapshotParentFullPath + Path.SEPARATOR
-          + HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + snapshotName
-          + relativePath;
-      return snapshotFullPath;
+      return Snapshot.getSnapshotPath(snapshotParentFullPath,
+          snapshotName + relativePath);
     }
     
     /**

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

@@ -5806,7 +5806,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (snapshotName == null || snapshotName.isEmpty()) {
         snapshotName = Snapshot.generateDefaultSnapshotName();
       }
-      dir.verifyMaxComponentLength(snapshotName, snapshotRoot, 0);
+      dir.verifySnapshotName(snapshotName, snapshotRoot);
       dir.writeLock();
       try {
         snapshotPath = snapshotManager.createSnapshot(snapshotRoot, snapshotName);
@@ -5844,7 +5844,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             safeMode);
       }
       checkOwner(pc, path);
-      dir.verifyMaxComponentLength(snapshotNewName, path, 0);
+      dir.verifySnapshotName(snapshotNewName, path);
       
       snapshotManager.renameSnapshot(path, snapshotOldName, snapshotNewName);
       getEditLog().logRenameSnapshot(path, snapshotOldName, snapshotNewName);
@@ -5854,12 +5854,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
-      Path oldSnapshotRoot = new Path(path, HdfsConstants.DOT_SNAPSHOT_DIR
-          + "/" + snapshotOldName);
-      Path newSnapshotRoot = new Path(path, HdfsConstants.DOT_SNAPSHOT_DIR
-          + "/" + snapshotNewName);
-      logAuditEvent(true, "renameSnapshot", oldSnapshotRoot.toString(),
-          newSnapshotRoot.toString(), null);
+      String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName);
+      String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName);
+      logAuditEvent(true, "renameSnapshot", oldSnapshotRoot, newSnapshotRoot, null);
     }
   }
   
@@ -5959,9 +5956,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
     
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
-      Path rootPath = new Path(snapshotRoot, HdfsConstants.DOT_SNAPSHOT_DIR
-          + Path.SEPARATOR + snapshotName);
-      logAuditEvent(true, "deleteSnapshot", rootPath.toString(), null, null);
+      String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
+      logAuditEvent(true, "deleteSnapshot", rootPath, null, null);
     }
   }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.FileNotFoundException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -511,8 +512,8 @@ public class INodeDirectory extends INodeWithAdditionalFields {
    * @return true if path component is {@link HdfsConstants#DOT_SNAPSHOT_DIR}
    */
   private static boolean isDotSnapshotDir(byte[] pathComponent) {
-    return pathComponent == null ? false : HdfsConstants.DOT_SNAPSHOT_DIR
-        .equalsIgnoreCase(DFSUtil.bytes2String(pathComponent));
+    return pathComponent == null ? false
+        : Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
 
   /**

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -49,9 +49,16 @@ public class Snapshot implements Comparable<byte[]> {
     return new SimpleDateFormat(DEFAULT_SNAPSHOT_NAME_PATTERN).format(new Date());
   }
 
-  static String getSnapshotPath(String snapshottableDir, String snapshotName) {
-    return new Path(snapshottableDir, HdfsConstants.DOT_SNAPSHOT_DIR
-        + Path.SEPARATOR + snapshotName).toString();
+  public static String getSnapshotPath(String snapshottableDir,
+      String snapshotRelativePath) {
+    final StringBuilder b = new StringBuilder(snapshottableDir);
+    if (b.charAt(b.length() - 1) != Path.SEPARATOR_CHAR) {
+      b.append(Path.SEPARATOR);
+    }
+    return b.append(HdfsConstants.DOT_SNAPSHOT_DIR)
+        .append(Path.SEPARATOR)
+        .append(snapshotRelativePath)
+        .toString();
   }
   
   /** 
@@ -123,9 +130,7 @@ public class Snapshot implements Comparable<byte[]> {
     
     @Override
     public String getFullPathName() {
-      return getParent().getFullPathName() + Path.SEPARATOR
-          + HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR
-          + this.getLocalName();
+      return getSnapshotPath(getParent().getFullPathName(), getLocalName());
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -123,7 +123,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42};
+      -40, -41, -42, -43};
   private int imageVersion = 0;
 
   /* (non-Javadoc)

+ 11 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java

@@ -43,14 +43,18 @@ public class SnapshotDiff {
     if (Path.CUR_DIR.equals(name)) { // current directory
       return "";
     }
-    if (name.startsWith(HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR)
-        || name.startsWith(Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR
-            + Path.SEPARATOR)) {
-      // get the snapshot name
-      int i = name.indexOf(HdfsConstants.DOT_SNAPSHOT_DIR);
-      return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
+    final int i;
+    if (name.startsWith(HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR)) {
+      i = 0;
+    } else if (name.startsWith(
+        HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR + Path.SEPARATOR)) {
+      i = 1;
+    } else {
+      return name;
     }
-    return name;
+
+    // get the snapshot name
+    return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
   }
   
   public static void main(String[] argv) throws IOException {

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java

@@ -33,8 +33,10 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.IllegalNameException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.junit.Before;
 import org.junit.Test;
@@ -104,6 +106,7 @@ public class TestFsLimits {
     addChildWithName("333", null);
     addChildWithName("4444", null);
     addChildWithName("55555", null);
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
   }
 
   @Test
@@ -143,6 +146,7 @@ public class TestFsLimits {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
     fsIsReady = false;
     
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
     addChildWithName("1", null);
     addChildWithName("22", null);
     addChildWithName("333", null);
@@ -159,7 +163,10 @@ public class TestFsLimits {
     
     Class<?> generated = null;
     try {
-      fs.verifyFsLimits(inodes, 1, child);
+      fs.verifyMaxComponentLength(child.getLocalNameBytes(), inodes, 1);
+      fs.verifyMaxDirItems(inodes, 1);
+      fs.verifyINodeName(child.getLocalNameBytes());
+
       rootInode.addChild(child);
     } catch (QuotaExceededException e) {
       generated = e.getClass();

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


File diff suppressed because it is too large
+ 204 - 246
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml


Some files were not shown because too many files changed in this diff