Browse Source

HDFS-4857. Snapshot.Root and AbstractINodeDiff#snapshotINode should not be put into INodeMap when loading FSImage. Contributed by Jing Zhao

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1487655 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 12 years ago
parent
commit
01f27f5c23

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

@@ -610,6 +610,9 @@ Trunk (Unreleased)
     HDFS-4846. Clean up snapshot CLI commands output stacktrace for invalid
     arguments. (Jing Zhao via brandonli)
 
+    HDFS-4857. Snapshot.Root and AbstractINodeDiff#snapshotINode should not be 
+    put into INodeMap when loading FSImage. (jing9)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -407,7 +407,7 @@ public class FSImageFormat {
       int numChildren = in.readInt();
       for (int i = 0; i < numChildren; i++) {
         // load single inode
-        INode newNode = loadINodeWithLocalName(false, in);
+        INode newNode = loadINodeWithLocalName(false, in, true);
         addToParent(parent, newNode);
       }
       return numChildren;
@@ -556,10 +556,11 @@ public class FSImageFormat {
     }
 
     public INode loadINodeWithLocalName(boolean isSnapshotINode,
-        DataInput in) throws IOException {
+        DataInput in, boolean updateINodeMap) throws IOException {
       final byte[] localName = FSImageSerialization.readLocalName(in);
       INode inode = loadINode(localName, isSnapshotINode, in);
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
+      if (updateINodeMap
+          && LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
         namesystem.dir.addToInodeMap(inode);
       }
       return inode;

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

@@ -129,7 +129,7 @@ public class Snapshot implements Comparable<byte[]> {
   static Snapshot read(DataInput in, FSImageFormat.Loader loader)
       throws IOException {
     final int snapshotId = in.readInt();
-    final INode root = loader.loadINodeWithLocalName(false, in);
+    final INode root = loader.loadINodeWithLocalName(false, in, false);
     return new Snapshot(snapshotId, root.asDirectory(), null);
   }
 

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

@@ -127,7 +127,7 @@ public class SnapshotFSImageFormat {
     
     // 3. Load snapshotINode 
     final INodeFile snapshotINode = in.readBoolean()?
-        loader.loadINodeWithLocalName(true, in).asFile(): null;
+        loader.loadINodeWithLocalName(true, in, false).asFile(): null;
     
     return new FileDiff(snapshot, snapshotINode, posterior, fileSize);
   }
@@ -194,7 +194,7 @@ public class SnapshotFSImageFormat {
     int deletedSize = in.readInt();
     List<INode> deletedList = new ArrayList<INode>(deletedSize);
     for (int i = 0; i < deletedSize; i++) {
-      final INode deleted = loader.loadINodeWithLocalName(true, in);
+      final INode deleted = loader.loadINodeWithLocalName(true, in, true);
       deletedList.add(deleted);
       // set parent: the parent field of an INode in the deleted list is not 
       // useful, but set the parent here to be consistent with the original 
@@ -246,8 +246,8 @@ public class SnapshotFSImageFormat {
   }
   
   /**
-   * Load the snapshotINode field of {@link SnapshotDiff}.
-   * @param snapshot The Snapshot associated with the {@link SnapshotDiff}.
+   * Load the snapshotINode field of {@link AbstractINodeDiff}.
+   * @param snapshot The Snapshot associated with the {@link AbstractINodeDiff}.
    * @param in The {@link DataInput} to read.
    * @param loader The {@link Loader} instance that this loading procedure is 
    *               using.
@@ -263,7 +263,7 @@ public class SnapshotFSImageFormat {
     } else {
       // another boolean is used to indicate whether snapshotINode is non-null
       return in.readBoolean()?
-          loader.loadINodeWithLocalName(true, in).asDirectory(): null;
+          loader.loadINodeWithLocalName(true, in, false).asDirectory(): null;
     }
   }
    
@@ -348,7 +348,8 @@ public class SnapshotFSImageFormat {
 
       final INodeReference.WithCount withCount;
       if (firstReferred) {
-        final INode referred = loader.loadINodeWithLocalName(isSnapshotINode, in);
+        final INode referred = loader.loadINodeWithLocalName(isSnapshotINode,
+            in, true);
         withCount = new INodeReference.WithCount(null, referred);
         referenceMap.put(withCount.getId(), withCount);
       } else {

+ 40 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -40,6 +41,8 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.log4j.Level;
@@ -162,6 +165,43 @@ public class TestFSImageWithSnapshot {
     }
   }
   
+  /**
+   * Test when there is snapshot taken on root
+   */
+  @Test
+  public void testSnapshotOnRoot() throws Exception {
+    final Path root = new Path("/");
+    hdfs.allowSnapshot(root);
+    hdfs.createSnapshot(root, "s1");
+    
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+        .numDataNodes(REPLICATION).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+    
+    INodeDirectorySnapshottable rootNode = 
+        (INodeDirectorySnapshottable) fsn.dir.getINode4Write(root.toString());
+    assertTrue("The children list of root should be empty", 
+        rootNode.getChildrenList(null).isEmpty());
+    // one snapshot on root: s1
+    List<DirectoryDiff> diffList = rootNode.getDiffs().asList();
+    assertEquals(1, diffList.size());
+    assertEquals("s1", diffList.get(0).getSnapshot().getRoot().getLocalName());
+  }
+
   /**
    * Testing steps:
    * <pre>