Browse Source

HDFS-4848. copyFromLocal and renaming a file to .snapshot should output that .snapshot is a reserved name. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1487942 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 12 năm trước cách đây
mục cha
commit
bf99961d67

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

@@ -616,6 +616,9 @@ Trunk (Unreleased)
     HDFS-4863. The root directory should be added to the snapshottable 
     directory list while loading fsimage. (jing9)
 
+    HDFS-4848. copyFromLocal and renaming a file to ".snapshot" should output 
+    that ".snapshot" is a reserved name. (Jing Zhao via brandonli)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -97,15 +97,4 @@ 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 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -48,7 +48,6 @@ 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.IllegalNameException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -696,11 +695,13 @@ public class FSDirectory implements Closeable {
           // its reference number here
           final INode originalChild = withCount.getReferredINode();
           srcChild = originalChild;
+          srcChild.setLocalName(srcChildName);
         } else {
           withCount.removeReference(oldSrcChild.asReference());
           final INodeReference originalRef = new INodeReference.DstReference(
               srcParent, withCount, srcRefDstSnapshot);
           srcChild = originalRef;
+          withCount.getReferredINode().setLocalName(srcChildName);
         }
         
         if (isSrcInSnapshot) {
@@ -980,11 +981,13 @@ public class FSDirectory implements Closeable {
           // its reference number here
           final INode originalChild = withCount.getReferredINode();
           srcChild = originalChild;
+          srcChild.setLocalName(srcChildName);
         } else {
           withCount.removeReference(oldSrcChild.asReference());
           final INodeReference originalRef = new INodeReference.DstReference(
               srcParent, withCount, srcRefDstSnapshot);
           srcChild = originalRef;
+          withCount.getReferredINode().setLocalName(srcChildName);
         }
         
         if (srcParent instanceof INodeDirectoryWithSnapshot) {
@@ -2070,20 +2073,20 @@ public class FSDirectory implements Closeable {
 
   /** Verify if the snapshot name is legal. */
   void verifySnapshotName(String snapshotName, String path)
-      throws PathComponentTooLongException, IllegalNameException {
+      throws PathComponentTooLongException {
     final byte[] bytes = DFSUtil.string2Bytes(snapshotName);
     verifyINodeName(bytes);
     verifyMaxComponentLength(bytes, path, 0);
   }
   
   /** Verify if the inode name is legal. */
-  void verifyINodeName(byte[] childName) throws IllegalNameException {
+  void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException {
     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);
+      throw new HadoopIllegalArgumentException(s);
     }
   }
 
@@ -2208,7 +2211,7 @@ public class FSDirectory implements Closeable {
     try {
       return addLastINode(inodesInPath, i, false);
     } catch (QuotaExceededException e) {
-      NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e); 
+      NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e);
     }
     return false;
   }

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java

@@ -27,17 +27,16 @@ import static org.mockito.Mockito.when;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 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.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;
 
@@ -106,7 +105,8 @@ public class TestFsLimits {
     addChildWithName("333", null);
     addChildWithName("4444", null);
     addChildWithName("55555", null);
-    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR,
+        HadoopIllegalArgumentException.class);
   }
 
   @Test
@@ -146,7 +146,8 @@ public class TestFsLimits {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY, 2);
     fsIsReady = false;
     
-    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR, IllegalNameException.class);
+    addChildWithName(HdfsConstants.DOT_SNAPSHOT_DIR,
+        HadoopIllegalArgumentException.class);
     addChildWithName("1", null);
     addChildWithName("22", null);
     addChildWithName("333", null);
@@ -168,7 +169,7 @@ public class TestFsLimits {
       fs.verifyINodeName(child.getLocalNameBytes());
 
       rootInode.addChild(child);
-    } catch (QuotaExceededException e) {
+    } catch (Throwable e) {
       generated = e.getClass();
     }
     assertEquals(expected, generated);

+ 68 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1389,8 +1390,8 @@ public class TestRenameWithSnapshots {
     INodeDirectory root = fsdir.getINode4Write("/").asDirectory();
     root.replaceChild(dir3, mockDir3, fsdir.getINodeMap());
     
-    final Path foo_dir2 = new Path(sdir2, "foo");
-    final Path foo_dir3 = new Path(sdir3, "foo");
+    final Path foo_dir2 = new Path(sdir2, "foo2");
+    final Path foo_dir3 = new Path(sdir3, "foo3");
     hdfs.rename(foo, foo_dir2);
     boolean result = hdfs.rename(foo_dir2, foo_dir3);
     assertFalse(result);
@@ -1406,7 +1407,7 @@ public class TestRenameWithSnapshots {
     ChildrenDiff childrenDiff = dir2Diffs.get(0).getChildrenDiff();
     assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
     assertEquals(1, childrenDiff.getList(ListType.CREATED).size());
-    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo");
+    final Path foo_s2 = SnapshotTestHelper.getSnapshotPath(sdir2, "s2", "foo2");
     assertFalse(hdfs.exists(foo_s2));
     
     INode fooNode = fsdir.getINode4Write(foo_dir2.toString());
@@ -1442,7 +1443,7 @@ public class TestRenameWithSnapshots {
     assertEquals(0, childrenDiff.getList(ListType.DELETED).size());
     assertEquals(0, childrenDiff.getList(ListType.CREATED).size());
     
-    final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo");
+    final Path foo_s3 = SnapshotTestHelper.getSnapshotPath(sdir2, "s3", "foo2");
     assertFalse(hdfs.exists(foo_s2));
     assertTrue(hdfs.exists(foo_s3));
     
@@ -1470,14 +1471,14 @@ public class TestRenameWithSnapshots {
     final Path bar = new Path(foo, "bar");
     DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
     
-    final Path foo2 = new Path(sdir2, "foo");
+    final Path foo2 = new Path(sdir2, "foo2");
     hdfs.mkdirs(foo2);
     
     SnapshotTestHelper.createSnapshot(hdfs, sdir1, "s1");
     SnapshotTestHelper.createSnapshot(hdfs, sdir2, "s2");
     
     // rename foo2 to foo3, so that foo3 will be a reference node
-    final Path foo3 = new Path(sdir3, "foo");
+    final Path foo3 = new Path(sdir3, "foo3");
     hdfs.rename(foo2, foo3);
     
     INode foo3Node = fsdir.getINode4Write(foo3.toString());
@@ -1665,6 +1666,67 @@ public class TestRenameWithSnapshots {
     assertEquals(0, diffList.size());
   }
   
+  /**
+   * Test rename to an invalid name (xxx/.snapshot)
+   */
+  @Test
+  public void testRenameUndo_7() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    final Path bar = new Path(foo, "bar");
+    DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPL, SEED);
+    
+    // create a snapshot on root
+    SnapshotTestHelper.createSnapshot(hdfs, root, snap1);
+    
+    // rename bar to /foo/.snapshot which is invalid
+    final Path invalid = new Path(foo, HdfsConstants.DOT_SNAPSHOT_DIR);
+    try {
+      hdfs.rename(bar, invalid);
+      fail("expect exception since invalid name is used for rename");
+    } catch (Exception e) {
+      GenericTestUtils.assertExceptionContains("\"" +
+          HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name", e);
+    }
+    
+    // check
+    INodeDirectoryWithSnapshot fooNode = (INodeDirectoryWithSnapshot) fsdir
+        .getINode4Write(foo.toString());
+    ReadOnlyList<INode> children = fooNode.getChildrenList(null);
+    assertEquals(1, children.size());
+    List<DirectoryDiff> diffList = fooNode.getDiffs().asList();
+    assertEquals(1, diffList.size());
+    DirectoryDiff diff = diffList.get(0);
+    // this diff is generated while renaming
+    assertEquals(snap1, Snapshot.getSnapshotName(diff.snapshot));
+    // after undo, the diff should be empty
+    assertTrue(diff.getChildrenDiff().getList(ListType.DELETED).isEmpty());
+    assertTrue(diff.getChildrenDiff().getList(ListType.CREATED).isEmpty());
+    
+    // bar was converted to filewithsnapshot while renaming
+    INodeFileWithSnapshot barNode = (INodeFileWithSnapshot) fsdir
+        .getINode4Write(bar.toString());
+    assertSame(barNode, children.get(0));
+    assertSame(fooNode, barNode.getParent());
+    List<FileDiff> barDiffList = barNode.getDiffs().asList();
+    assertEquals(1, barDiffList.size());
+    FileDiff barDiff = barDiffList.get(0);
+    assertEquals(snap1, Snapshot.getSnapshotName(barDiff.snapshot));
+    
+    // restart cluster multiple times to make sure the fsimage and edits log are
+    // correct. Note that when loading fsimage, foo and bar will be converted 
+    // back to normal INodeDirectory and INodeFile since they do not store any 
+    // snapshot data
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPL).build();
+    cluster.waitActive();
+    restartClusterAndCheckImage();
+  }
+  
   /**
    * Test the rename undo when quota of dst tree is exceeded after rename.
    */