瀏覽代碼

HDFS-6266. Merge r1589920 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1589921 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 年之前
父節點
當前提交
e91f6f809b

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

@@ -72,6 +72,8 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6282. Re-add testIncludeByRegistrationName. (cmccabe)
 
+    HDFS-6266. Identify full path for a given INode. (jing9)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

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

@@ -2911,7 +2911,9 @@ public class FSDirectory implements Closeable {
   /**
    * Given an INode get all the path complents leading to it from the root.
    * If an Inode corresponding to C is given in /A/B/C, the returned
-   * patch components will be {root, A, B, C}
+   * patch components will be {root, A, B, C}.
+   * Note that this method cannot handle scenarios where the inode is in a
+   * snapshot.
    */
   static byte[][] getPathComponents(INode inode) {
     List<byte[]> components = new ArrayList<byte[]>();
@@ -2922,7 +2924,54 @@ public class FSDirectory implements Closeable {
     }
     return components.toArray(new byte[components.size()][]);
   }
-  
+
+  /**
+   * The same functionality with {@link #getPathComponents(INode)}, but can
+   * handle snapshots.
+   */
+  public static byte[][] getPathComponentsWithSnapshot(INode inode) {
+    List<byte[]> components = new ArrayList<byte[]>();
+    boolean inSnapshot = false;
+    int snapshotId = Snapshot.CURRENT_STATE_ID;
+    do {
+      if (inode instanceof INodeReference.WithCount) {
+        // identify the corresponding WithName or DstReference node
+        inode = ((WithCount) inode).getParentRef(snapshotId);
+      } else { // normal INode and WithName/DstReference
+        if (inode instanceof INodeDirectory
+            && inode.asDirectory().isSnapshottable() && inSnapshot
+            && snapshotId != Snapshot.CURRENT_STATE_ID) {
+          INodeDirectorySnapshottable sdir = (INodeDirectorySnapshottable) inode
+              .asDirectory();
+          Snapshot snapshot = sdir.getSnapshotById(snapshotId);
+          if (snapshot != null) {
+            components.add(0, snapshot.getRoot().getLocalNameBytes());
+            components.add(0, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES);
+            // the snapshot has been found, thus no need to check snapshottable
+            // directory afterwards
+            inSnapshot = false;
+          }
+        }
+        INode parent = inode.getParentReference() != null ? inode
+            .getParentReference() : inode.getParent();
+        if (parent != null && parent instanceof INodeDirectory) {
+          int sid = parent.asDirectory().searchChild(inode);
+          Preconditions.checkState(sid != Snapshot.NO_SNAPSHOT_ID);
+          if (sid != Snapshot.CURRENT_STATE_ID
+              && snapshotId == Snapshot.CURRENT_STATE_ID) {
+            snapshotId = sid;
+            inSnapshot = true;
+          }
+          components.add(0, inode.getLocalNameBytes());
+        } else if (parent == null) { // root
+          components.add(0, inode.getLocalNameBytes());
+        }
+        inode = parent;
+      }
+    } while (inode != null);
+    return components.toArray(new byte[components.size()][]);
+  }
+
   /**
    * @return path components for reserved path, else null.
    */

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -656,7 +655,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * @return array of byte arrays each of which represents
    * a single path component.
    */
-  static byte[][] getPathComponents(String path) {
+  @VisibleForTesting
+  public static byte[][] getPathComponents(String path) {
     return getPathComponents(getPathNames(path));
   }
 

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -356,6 +356,29 @@ public class INodeDirectory extends INodeWithAdditionalFields
     
     return sf.getChild(this, name, snapshotId);
   }
+
+  /**
+   * Search for the given INode in the children list and the deleted lists of
+   * snapshots.
+   * @return {@link Snapshot#CURRENT_STATE_ID} if the inode is in the children
+   * list; {@link Snapshot#NO_SNAPSHOT_ID} if the inode is neither in the
+   * children list nor in any snapshot; otherwise the snapshot id of the
+   * corresponding snapshot diff list.
+   */
+  int searchChild(INode inode) {
+    INode child = getChild(inode.getLocalNameBytes(), Snapshot.CURRENT_STATE_ID);
+    if (child != inode) {
+      // inode is not in parent's children list, thus inode must be in
+      // snapshot. identify the snapshot id and later add it into the path
+      DirectoryDiffList diffs = getDiffs();
+      if (diffs == null) {
+        return Snapshot.NO_SNAPSHOT_ID;
+      }
+      return diffs.findSnapshotDeleted(inode);
+    } else {
+      return Snapshot.CURRENT_STATE_ID;
+    }
+  }
   
   /**
    * @param snapshotId

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java

@@ -418,6 +418,23 @@ public abstract class INodeReference extends INode {
         return withNameList.get(-i - 2);
       }
     }
+
+    public INodeReference getParentRef(int snapshotId) {
+      // when the given snapshotId is CURRENT_STATE_ID, it is possible that we
+      // do not know where the corresponding inode belongs, thus we simply
+      // return the last reference node
+      if (snapshotId == Snapshot.CURRENT_STATE_ID) {
+        return this.getParentReference() != null ? this.getParentReference()
+            : this.getLastWithName();
+      }
+      // otherwise we search the withNameList
+      for (int i = 0; i < withNameList.size(); i++) {
+        if (snapshotId <= withNameList.get(i).lastSnapshotId) {
+          return withNameList.get(i);
+        }
+      }
+      return this.getParentReference();
+    }
   }
   
   /** A reference with a fixed name. */

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

@@ -364,6 +364,25 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       }
       return false;
     }
+
+    /**
+     * Find the corresponding snapshot whose deleted list contains the given
+     * inode.
+     * @return the id of the snapshot. {@link Snapshot#NO_SNAPSHOT_ID} if the
+     * given inode is not in any of the snapshot.
+     */
+    public int findSnapshotDeleted(final INode child) {
+      final List<DirectoryDiff> diffList = asList();
+      for(int i = diffList.size() - 1; i >= 0; i--) {
+        final ChildrenDiff diff = diffList.get(i).diff;
+        final int d = diff.searchIndex(ListType.DELETED,
+            child.getLocalNameBytes());
+        if (d >= 0 && diff.getList(ListType.DELETED).get(d) == child) {
+          return diffList.get(i).getSnapshotId();
+        }
+      }
+      return Snapshot.NO_SNAPSHOT_ID;
+    }
   }
   
   private static Map<INode, INode> cloneDiffList(List<INode> diffList) {

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

@@ -206,7 +206,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
     return i < 0? null: snapshotsByNames.get(i);
   }
   
-  Snapshot getSnapshotById(int sid) {
+  public Snapshot getSnapshotById(int sid) {
     for (Snapshot s : snapshotsByNames) {
       if (s.getId() == sid) {
         return s;

+ 15 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -82,6 +82,7 @@ import java.util.concurrent.TimeoutException;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -1212,7 +1213,20 @@ public class DFSTestUtil {
     long c = (val + factor - 1) / factor;
     return c * factor;
   }
-  
+
+  public static void checkComponentsEquals(byte[][] expected, byte[][] actual) {
+    assertEquals("expected: " + DFSUtil.byteArray2PathString(expected)
+        + ", actual: " + DFSUtil.byteArray2PathString(actual), expected.length,
+        actual.length);
+    int i = 0;
+    for (byte[] e : expected) {
+      byte[] actualComponent = actual[i++];
+      assertTrue("expected: " + DFSUtil.bytes2String(e) + ", actual: "
+          + DFSUtil.bytes2String(actualComponent),
+          Arrays.equals(e, actualComponent));
+    }
+  }
+
   /**
    * A short-circuit test context which makes it easier to get a short-circuit
    * configuration and set everything up.

+ 1 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -26,7 +26,6 @@ import static org.junit.Assert.fail;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.junit.Assert;
@@ -792,14 +791,6 @@ public class TestINodeFile {
     return dir; // Last Inode in the chain
   }
   
-  private static void checkEquals(byte[][] expected, byte[][] actual) {
-    assertEquals(expected.length, actual.length);
-    int i = 0;
-    for (byte[] e : expected) {
-      assertTrue(Arrays.equals(e, actual[i++]));
-    }
-  }
-  
   /**
    * Test for {@link FSDirectory#getPathComponents(INode)}
    */
@@ -809,7 +800,7 @@ public class TestINodeFile {
     INode inode = createTreeOfInodes(path);
     byte[][] expected = INode.getPathComponents(path);
     byte[][] actual = FSDirectory.getPathComponents(inode);
-    checkEquals(expected, actual);
+    DFSTestUtil.checkComponentsEquals(expected, actual);
   }
   
   /**

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

@@ -0,0 +1,295 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.snapshot;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeReference;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test getting the full path name of a given inode. The INode may be in
+ * snapshot.
+ */
+public class TestFullPathNameWithSnapshot {
+  private static final int BLOCKSIZE = 1024;
+  private static final short REPLICATION = 1;
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private FSDirectory fsdir;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+
+    fsdir = cluster.getNamesystem().getFSDirectory();
+    dfs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Normal case without snapshot involved
+   */
+  @Test
+  public void testNormalINode() throws Exception {
+    final Path bar = new Path("/foo/bar");
+    dfs.mkdirs(bar);
+    final Path file = new Path(bar, "file");
+    DFSTestUtil.createFile(dfs, file, BLOCKSIZE, REPLICATION, 0L);
+    INode fileNode = fsdir.getINode4Write(file.toString());
+    byte[][] pathComponents = FSDirectory
+        .getPathComponentsWithSnapshot(fileNode);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(file.toString()),
+        pathComponents);
+  }
+
+  /**
+   * INode in deleted list
+   */
+  @Test
+  public void testDeletedINode() throws Exception {
+    final Path foo = new Path("/foo");
+    final Path f1 = new Path(foo, "f1");
+    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
+    final Path bar = new Path(foo, "bar");
+    dfs.mkdirs(bar);
+    final Path f2 = new Path(bar, "f2");
+    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
+
+    INode f1Node = fsdir.getINode4Write(f1.toString());
+    INode f2Node = fsdir.getINode4Write(f2.toString());
+
+    SnapshotTestHelper.createSnapshot(dfs, foo, "s1");
+    dfs.delete(bar, true);
+    SnapshotTestHelper.createSnapshot(dfs, foo, "s2");
+    dfs.delete(f1, true);
+
+    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
+    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
+    // expected: /foo/.snapshot/s2/f1
+    String f1Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s2",
+        f1.getName()).toString();
+    // expected: /foo/.snapshot/s1/bar/f2
+    String f2Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/f2")
+        .toString();
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f1Snapshot),
+        f1Components);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f2Snapshot),
+        f2Components);
+
+    // delete snapshot s2
+    dfs.deleteSnapshot(foo, "s2");
+    // expected: /foo/.snapshot/s1/f1
+    f1Snapshot = SnapshotTestHelper.getSnapshotPath(foo, "s1", f1.getName())
+        .toString();
+    f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(f1Snapshot),
+        f1Components);
+  }
+
+  /**
+   * INode after renaming
+   */
+  @Test
+  public void testRenamedINode() throws Exception {
+    final Path foo = new Path("/foo");
+    final Path bar = new Path(foo, "bar");
+    final Path f1 = new Path(bar, "f1");
+    final Path f2 = new Path(bar, "f2");
+    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
+    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
+
+    // create snapshot s1
+    SnapshotTestHelper.createSnapshot(dfs, foo, "s1");
+    INode f2Node = fsdir.getINode4Write(f2.toString());
+    // delete /foo/bar/f2
+    dfs.delete(f2, true);
+    // rename bar to bar2
+    final Path bar2 = new Path(foo, "bar2");
+    dfs.rename(bar, bar2);
+    // create snapshot s2
+    SnapshotTestHelper.createSnapshot(dfs, foo, "s2");
+
+    // /foo/.snapshot/s1/bar
+    Path barPath = SnapshotTestHelper.getSnapshotPath(foo, "s1", bar.getName());
+    INode barNode = fsdir.getINode(barPath.toString());
+    Assert.assertTrue(barNode instanceof INodeReference.WithName);
+    INode bar2Node = fsdir.getINode(bar2.toString());
+    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
+    byte[][] barComponents = FSDirectory.getPathComponentsWithSnapshot(barNode);
+    byte[][] bar2Components = FSDirectory
+        .getPathComponentsWithSnapshot(bar2Node);
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(barPath.toString()), barComponents);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
+        bar2Components);
+
+    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
+    // expected: /foo/.snapshot/s1/bar/f2
+    Path deletedf2 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/f2");
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(deletedf2.toString()), f2Components);
+
+    final Path newf1 = new Path(bar2, f1.getName());
+    INode f1Node = fsdir.getINode(newf1.toString());
+    Assert.assertTrue(dfs.delete(newf1, true));
+    Path deletedf1 = SnapshotTestHelper.getSnapshotPath(foo, "s2", "bar2/f1");
+    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(deletedf1.toString()), f1Components);
+  }
+
+  /**
+   * Similar with testRenamedINode but the rename is across two snapshottable
+   * directory.
+   */
+  @Test
+  public void testRenamedINode2() throws Exception {
+    final Path foo1 = new Path("/foo1");
+    final Path foo2 = new Path("/foo2");
+    final Path bar = new Path(foo1, "bar");
+    final Path f1 = new Path(bar, "f1");
+    final Path f2 = new Path(bar, "f2");
+    dfs.mkdirs(foo2);
+    DFSTestUtil.createFile(dfs, f1, BLOCKSIZE, REPLICATION, 0L);
+    DFSTestUtil.createFile(dfs, f2, BLOCKSIZE, REPLICATION, 0L);
+
+    // create snapshots on foo1 and foo2
+    SnapshotTestHelper.createSnapshot(dfs, foo1, "s1");
+    SnapshotTestHelper.createSnapshot(dfs, foo2, "s2");
+    INode f2Node = fsdir.getINode4Write(f2.toString());
+    // delete /foo1/bar/f2
+    dfs.delete(f2, true);
+    // rename bar to bar2
+    final Path bar2 = new Path(foo2, "bar2");
+    dfs.rename(bar, bar2);
+    // create snapshot s3 and s4 on foo1 and foo2
+    SnapshotTestHelper.createSnapshot(dfs, foo1, "s3");
+    SnapshotTestHelper.createSnapshot(dfs, foo2, "s4");
+
+    // /foo1/.snapshot/s1/bar
+    Path barPath = SnapshotTestHelper
+        .getSnapshotPath(foo1, "s1", bar.getName());
+    INode barNode = fsdir.getINode(barPath.toString());
+    Assert.assertTrue(barNode instanceof INodeReference.WithName);
+    INode bar2Node = fsdir.getINode(bar2.toString());
+    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
+    byte[][] barComponents = FSDirectory.getPathComponentsWithSnapshot(barNode);
+    byte[][] bar2Components = FSDirectory
+        .getPathComponentsWithSnapshot(bar2Node);
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(barPath.toString()), barComponents);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
+        bar2Components);
+
+    byte[][] f2Components = FSDirectory.getPathComponentsWithSnapshot(f2Node);
+    // expected: /foo1/.snapshot/s1/bar/f2
+    Path deletedf2 = SnapshotTestHelper.getSnapshotPath(foo1, "s1", "bar/f2");
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(deletedf2.toString()), f2Components);
+
+    final Path newf1 = new Path(bar2, f1.getName());
+    INode f1Node = fsdir.getINode(newf1.toString());
+    Assert.assertTrue(dfs.delete(newf1, true));
+    // /foo2/.snapshot/s4/bar2/f1
+    Path deletedf1 = SnapshotTestHelper.getSnapshotPath(foo2, "s4", "bar2/f1");
+    byte[][] f1Components = FSDirectory.getPathComponentsWithSnapshot(f1Node);
+    DFSTestUtil.checkComponentsEquals(
+        INode.getPathComponents(deletedf1.toString()), f1Components);
+  }
+
+  /**
+   * Rename a directory to its prior descendant
+   */
+  @Test
+  public void testNestedRename() throws Exception {
+    final Path sdir1 = new Path("/dir1");
+    final Path sdir2 = new Path("/dir2");
+    final Path foo = new Path(sdir1, "foo");
+    final Path bar = new Path(foo, "bar");
+    dfs.mkdirs(bar);
+    dfs.mkdirs(sdir2);
+
+    SnapshotTestHelper.createSnapshot(dfs, sdir1, "s1");
+
+    // /dir1/foo/bar -> /dir2/bar
+    final Path bar2 = new Path(sdir2, "bar");
+    dfs.rename(bar, bar2);
+
+    // /dir1/foo -> /dir2/bar/foo
+    final Path foo2 = new Path(bar2, "foo");
+    dfs.rename(foo, foo2);
+
+    // /dir2/bar
+    INode bar2Node = fsdir.getINode(bar2.toString());
+    Assert.assertTrue(bar2Node instanceof INodeReference.DstReference);
+    byte[][] bar2Components = FSDirectory
+        .getPathComponentsWithSnapshot(bar2Node);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(bar2.toString()),
+        bar2Components);
+
+    // /dir1/.snapshot/s1/foo/bar
+    String oldbar = SnapshotTestHelper.getSnapshotPath(sdir1, "s1", "foo/bar")
+        .toString();
+    INode oldbarNode = fsdir.getINode(oldbar);
+    Assert.assertTrue(oldbarNode instanceof INodeReference.WithName);
+    byte[][] oldbarComponents = FSDirectory
+        .getPathComponentsWithSnapshot(oldbarNode);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(oldbar),
+        oldbarComponents);
+
+    // /dir2/bar/foo
+    INode foo2Node = fsdir.getINode(foo2.toString());
+    Assert.assertTrue(foo2Node instanceof INodeReference.DstReference);
+    byte[][] foo2Components = FSDirectory
+        .getPathComponentsWithSnapshot(foo2Node);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(foo2.toString()),
+        foo2Components);
+
+    // /dir1/.snapshot/s1/foo
+    String oldfoo = SnapshotTestHelper.getSnapshotPath(sdir1, "s1",
+        foo.getName()).toString();
+    INode oldfooNode = fsdir.getINode(oldfoo);
+    Assert.assertTrue(oldfooNode instanceof INodeReference.WithName);
+    byte[][] oldfooComponents = FSDirectory
+        .getPathComponentsWithSnapshot(oldfooNode);
+    DFSTestUtil.checkComponentsEquals(INode.getPathComponents(oldfoo),
+        oldfooComponents);
+  }
+}

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

@@ -1829,7 +1829,7 @@ public class TestRenameWithSnapshots {
   }
   
   /**
-   * move a directory to its prior descedant
+   * move a directory to its prior descendant
    */
   @Test
   public void testRename2PreDescendant_2() throws Exception {