Explorar o código

HDFS-4758. Disallow nested snapshottable directories and unwrap RemoteException.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1477867 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze %!s(int64=12) %!d(string=hai) anos
pai
achega
98c0421fbf
Modificáronse 14 ficheiros con 162 adicións e 55 borrados
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 40 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  5. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  6. 38 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  7. 1 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  8. 52 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java
  9. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  10. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  11. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java
  12. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotMetrics.java
  13. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java
  14. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshottableDirListing.java

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

@@ -326,3 +326,6 @@ Branch-2802 Snapshot (Unreleased)
   (Jing Zhao via szetszwo)
 
   HDFS-4760. Update inodeMap after node replacement.  (Jing Zhao via szetszwo)
+
+  HDFS-4758. Disallow nested snapshottable directories and unwrap
+  RemoteException.  (szetszwo)

+ 40 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -41,18 +41,16 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPAC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
 
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -113,13 +111,13 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
@@ -2061,7 +2059,11 @@ public class DFSClient implements java.io.Closeable {
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
     checkOpen();
-    return namenode.createSnapshot(snapshotRoot, snapshotName);
+    try {
+      return namenode.createSnapshot(snapshotRoot, snapshotName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2075,7 +2077,11 @@ public class DFSClient implements java.io.Closeable {
    */
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    namenode.deleteSnapshot(snapshotRoot, snapshotName);
+    try {
+      namenode.deleteSnapshot(snapshotRoot, snapshotName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2089,7 +2095,11 @@ public class DFSClient implements java.io.Closeable {
   public void renameSnapshot(String snapshotDir, String snapshotOldName,
       String snapshotNewName) throws IOException {
     checkOpen();
-    namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+    try {
+      namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2101,7 +2111,11 @@ public class DFSClient implements java.io.Closeable {
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
     checkOpen();
-    return namenode.getSnapshottableDirListing();
+    try {
+      return namenode.getSnapshottableDirListing();
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
 
   /**
@@ -2111,7 +2125,11 @@ public class DFSClient implements java.io.Closeable {
    */
   public void allowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    namenode.allowSnapshot(snapshotRoot);
+    try {
+      namenode.allowSnapshot(snapshotRoot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2121,7 +2139,11 @@ public class DFSClient implements java.io.Closeable {
    */
   public void disallowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    namenode.disallowSnapshot(snapshotRoot);
+    try {
+      namenode.disallowSnapshot(snapshotRoot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2132,8 +2154,12 @@ public class DFSClient implements java.io.Closeable {
   public SnapshotDiffReport getSnapshotDiffReport(Path snapshotDir,
       String fromSnapshot, String toSnapshot) throws IOException {
     checkOpen();
-    return namenode.getSnapshotDiffReport(snapshotDir.toString(), fromSnapshot,
-        toSnapshot);
+    try {
+      return namenode.getSnapshotDiffReport(snapshotDir.toString(),
+          fromSnapshot, toSnapshot);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**

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

@@ -542,7 +542,7 @@ public class FSEditLogLoader {
     case OP_ALLOW_SNAPSHOT: {
       AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
       fsNamesys.getSnapshotManager().setSnapshottable(
-          allowSnapshotOp.snapshotRoot);
+          allowSnapshotOp.snapshotRoot, false);
       break;
     }
     case OP_DISALLOW_SNAPSHOT: {

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

@@ -5809,7 +5809,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         .shouldAvoidStaleDataNodesForWrite();
   }
   
-  SnapshotManager getSnapshotManager() {
+  public SnapshotManager getSnapshotManager() {
     return snapshotManager;
   }
   
@@ -5826,7 +5826,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       dir.writeLock();
       try {
-        snapshotManager.setSnapshottable(path);
+        snapshotManager.setSnapshottable(path, true);
       } finally {
         dir.writeUnlock();
       }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -185,6 +185,16 @@ public abstract class INode implements Diff.Element<byte[]> {
     return this == child.asReference().getReferredINode();
   }
   
+  /** @return true if the given inode is an ancestor directory of this inode. */
+  public final boolean isAncestorDirectory(final INodeDirectory dir) {
+    for(INodeDirectory p = getParent(); p != null; p = p.getParent()) {
+      if (p == dir) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   /**
    * When {@link #recordModification} is called on a referred node,
    * this method tells which snapshot the modification should be

+ 38 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -53,6 +53,7 @@ import com.google.common.base.Preconditions;
  * if necessary.
  */
 public class SnapshotManager implements SnapshotStats {
+  private boolean allowNestedSnapshots = false;
   private final FSDirectory fsdir;
   private static final int SNAPSHOT_ID_BIT_WIDTH = 24;
 
@@ -68,13 +69,45 @@ public class SnapshotManager implements SnapshotStats {
     this.fsdir = fsdir;
   }
 
+  /** Used in tests only */
+  void setAllowNestedSnapshots(boolean allowNestedSnapshots) {
+    this.allowNestedSnapshots = allowNestedSnapshots;
+  }
+
+  private void checkNestedSnapshottable(INodeDirectory dir, String path)
+      throws SnapshotException {
+    if (allowNestedSnapshots) {
+      return;
+    }
+
+    for(INodeDirectorySnapshottable s : snapshottables.values()) {
+      if (s.isAncestorDirectory(dir)) {
+        throw new SnapshotException(
+            "Nested snapshottable directories not allowed: path=" + path
+            + ", the ancestor " + s.getFullPathName()
+            + " is already a snapshottable directory.");
+      }
+      if (dir.isAncestorDirectory(s)) {
+        throw new SnapshotException(
+            "Nested snapshottable directories not allowed: path=" + path
+            + ", the subdirectory " + s.getFullPathName()
+            + " is already a snapshottable directory.");
+      }
+    }
+  }
+
   /**
    * Set the given directory as a snapshottable directory.
    * If the path is already a snapshottable directory, update the quota.
    */
-  public void setSnapshottable(final String path) throws IOException {
-    final INodesInPath iip = fsdir.getLastINodeInPath(path);
-    final INodeDirectory d = INodeDirectory.valueOf(iip.getINode(0), path);
+  public void setSnapshottable(final String path, boolean checkNestedSnapshottable)
+      throws IOException {
+    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
+    final INodeDirectory d = INodeDirectory.valueOf(iip.getLastINode(), path);
+    if (checkNestedSnapshottable) {
+      checkNestedSnapshottable(d, path);
+    }
+
 
     final INodeDirectorySnapshottable s;
     if (d.isSnapshottable()) {
@@ -114,9 +147,9 @@ public class SnapshotManager implements SnapshotStats {
    * @throws SnapshotException if there are snapshots in the directory.
    */
   public void resetSnapshottable(final String path) throws IOException {
-    final INodesInPath iip = fsdir.getLastINodeInPath(path);
+    final INodesInPath iip = fsdir.getINodesInPath4Write(path);
     final INodeDirectorySnapshottable s = INodeDirectorySnapshottable.valueOf(
-        iip.getINode(0), path);
+        iip.getLastINode(), path);
     if (s.getNumSnapshots() > 0) {
       throw new SnapshotException("The directory " + path + " has snapshot(s). "
           + "Please redo the operation after removing all the snapshots.");

+ 1 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -45,21 +45,16 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
@@ -95,13 +90,10 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenRenewer;
@@ -337,18 +329,7 @@ public class WebHdfsFileSystem extends FileSystem
       return ioe;
     }
 
-    final RemoteException re = (RemoteException)ioe;
-    return re.unwrapRemoteException(AccessControlException.class,
-        InvalidToken.class,
-        AuthenticationException.class,
-        AuthorizationException.class,
-        FileAlreadyExistsException.class,
-        FileNotFoundException.class,
-        ParentNotDirectoryException.class,
-        UnresolvedPathException.class,
-        SafeModeException.class,
-        DSQuotaExceededException.class,
-        NSQuotaExceededException.class);
+    return ((RemoteException)ioe).unwrapRemoteException();
   }
 
   /**

+ 52 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -83,6 +83,8 @@ public class TestNestedSnapshots {
    */
   @Test (timeout=300000)
   public void testNestedSnapshots() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     final Path foo = new Path("/testNestedSnapshots/foo");
     final Path bar = new Path(foo, "bar");
     final Path file1 = new Path(bar, "file1");
@@ -110,6 +112,7 @@ public class TestNestedSnapshots {
     assertFile(s1path, s2path, file1, true, true, true);
     assertFile(s1path, s2path, file2, true, false, false);
 
+    //test root
     final String rootStr = "/";
     final Path rootPath = new Path(rootStr);
     hdfs.allowSnapshot(rootPath);
@@ -120,6 +123,47 @@ public class TestNestedSnapshots {
     print("delete snapshot " + rootSnapshot);
     hdfs.disallowSnapshot(rootPath);
     print("disallow snapshot " + rootStr);
+    
+    //change foo to non-snapshottable
+    hdfs.deleteSnapshot(foo, s1name);
+    hdfs.disallowSnapshot(foo);
+    
+    //test disallow nested snapshots
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(false);
+    try {
+      hdfs.allowSnapshot(rootPath);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "ancestor");
+    }
+    try {
+      hdfs.allowSnapshot(foo);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "ancestor");
+    }
+
+    final Path sub1Bar = new Path(bar, "sub1");
+    final Path sub2Bar = new Path(sub1Bar, "sub2");
+    hdfs.mkdirs(sub2Bar);
+    try {
+      hdfs.allowSnapshot(sub1Bar);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "subdirectory");
+    }
+    try {
+      hdfs.allowSnapshot(sub2Bar);
+      Assert.fail();
+    } catch(SnapshotException se) {
+      assertNestedSnapshotException(se, "subdirectory");
+    }
+  }
+  
+  static void assertNestedSnapshotException(SnapshotException se, String substring) {
+    Assert.assertTrue(se.getMessage().startsWith(
+        "Nested snapshottable directories not allowed"));
+    Assert.assertTrue(se.getMessage().contains(substring));
   }
 
   private static void print(String message) throws UnresolvedLinkException {
@@ -226,22 +270,18 @@ public class TestNestedSnapshots {
       // createSnapshot should fail with quota
       hdfs.createSnapshot(dir);
       Assert.fail();
-    } catch(RemoteException re) {
-      final IOException ioe = re.unwrapRemoteException();
-      if (ioe instanceof NSQuotaExceededException) {
-        SnapshotTestHelper.LOG.info("The exception is expected.", ioe);
-      }
+    } catch(NSQuotaExceededException e) {
+      SnapshotTestHelper.LOG.info("The exception is expected.", e);
     }
 
     try {
       // setPermission f1 should fail with quote since it cannot add diff.
       hdfs.setPermission(f1, new FsPermission((short)0));
       Assert.fail();
-    } catch(RemoteException re) {
-      final IOException ioe = re.unwrapRemoteException();
-      if (ioe instanceof NSQuotaExceededException) {
-        SnapshotTestHelper.LOG.info("The exception is expected.", ioe);
-      }
+    } catch(RemoteException e) {
+      Assert.assertSame(NSQuotaExceededException.class,
+          e.unwrapRemoteException().getClass());
+      SnapshotTestHelper.LOG.info("The exception is expected.", e);
     }
 
     // setPermission f2 since it was created after the snapshot
@@ -294,6 +334,8 @@ public class TestNestedSnapshots {
    */
   @Test
   public void testDisallowNestedSnapshottableDir() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     final Path dir = new Path("/dir");
     final Path sub = new Path(dir, "sub");
     hdfs.mkdirs(sub);

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -231,6 +231,7 @@ public class TestSnapshot {
   private void runTestSnapshot() throws Exception {
     for (int i = 0; i < SNAPSHOT_ITERATION_NUMBER; i++) {
       // create snapshot and check the creation
+      cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
       TestDirectoryTree.Node[] ssNodes = createSnapshots();
       
       // prepare the modifications for the snapshotted dirs

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -784,6 +784,8 @@ public class TestSnapshotDeletion {
    */
   @Test (timeout=300000)
   public void testRenameSnapshotDiff() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     final Path subFile0 = new Path(sub, "file0");
     final Path subsubFile0 = new Path(subsub, "file0");
     DFSTestUtil.createFile(hdfs, subFile0, BLOCKSIZE, REPLICATION, seed);

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

@@ -171,6 +171,8 @@ public class TestSnapshotDiffReport {
   /** Test the computation and representation of diff between snapshots */
   @Test (timeout=60000)
   public void testDiffReport() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     Path subsub1 = new Path(sub1, "subsub1");
     Path subsubsub1 = new Path(subsub1, "subsubsub1");
     hdfs.mkdirs(subsubsub1);

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotMetrics.java

@@ -78,6 +78,8 @@ public class TestSnapshotMetrics {
    */
   @Test
   public void testSnapshottableDirs() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     assertGauge("SnapshottableDirectories", 0, getMetrics(NS_METRICS));
     assertCounter("AllowSnapshotOps", 0L, getMetrics(NN_METRICS));
     assertCounter("DisallowSnapshotOps", 0L, getMetrics(NN_METRICS));
@@ -130,6 +132,8 @@ public class TestSnapshotMetrics {
    */
   @Test
   public void testSnapshots() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     assertGauge("Snapshots", 0, getMetrics(NS_METRICS));
     assertCounter("CreateSnapshotOps", 0L, getMetrics(NN_METRICS));
     

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotRename.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
-import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -167,7 +166,7 @@ public class TestSnapshotRename {
     // Create snapshot for sub1
     SnapshotTestHelper.createSnapshot(hdfs, sub1, "s1");
     
-    exception.expect(RemoteException.class);
+    exception.expect(SnapshotException.class);
     String error = "The snapshot wrongName does not exist for directory "
         + sub1.toString();
     exception.expectMessage(error);
@@ -184,7 +183,7 @@ public class TestSnapshotRename {
     SnapshotTestHelper.createSnapshot(hdfs, sub1, "s1");
     SnapshotTestHelper.createSnapshot(hdfs, sub1, "s2");
     
-    exception.expect(RemoteException.class);
+    exception.expect(SnapshotException.class);
     String error = "The snapshot s2 already exists for directory "
         + sub1.toString();
     exception.expectMessage(error);

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshottableDirListing.java

@@ -75,6 +75,8 @@ public class TestSnapshottableDirListing {
    */
   @Test (timeout=60000)
   public void testListSnapshottableDir() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     // Initially there is no snapshottable directories in the system
     SnapshottableDirectoryStatus[] dirs = hdfs.getSnapshottableDirListing();
     assertNull(dirs);
@@ -158,6 +160,8 @@ public class TestSnapshottableDirListing {
    */
   @Test (timeout=60000)
   public void testListWithDifferentUser() throws Exception {
+    cluster.getNamesystem().getSnapshotManager().setAllowNestedSnapshots(true);
+
     // first make dir1 and dir2 snapshottable
     hdfs.allowSnapshot(dir1);
     hdfs.allowSnapshot(dir2);