浏览代码

HDFS-4684. Use INode id for image serialization when writing INodeReference.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1466718 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 年之前
父节点
当前提交
4c00514ede

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

@@ -228,3 +228,6 @@ Branch-2802 Snapshot (Unreleased)
 
   HDFS-4647. Rename should call setLocalName after an inode is removed from
   snapshots.  (Arpit Agarwal via szetszwo)
+
+  HDFS-4684. Use INode id for image serialization when writing INodeReference.
+  (szetszwo)

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

@@ -270,10 +270,6 @@ public class FSImageFormat {
         } else {
           imgTxId = 0;
         }
-        
-        if (supportSnapshot) {
-          snapshotMap = namesystem.getSnapshotManager().read(in, this);
-        }
 
         // read the last allocated inode id in the fsimage
         if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) {
@@ -289,6 +285,10 @@ public class FSImageFormat {
           }
         }
         
+        if (supportSnapshot) {
+          snapshotMap = namesystem.getSnapshotManager().read(in, this);
+        }
+
         // read compression related info
         FSImageCompression compression;
         if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
@@ -843,7 +843,6 @@ public class FSImageFormat {
         context.checkCancelled();
         fout.getChannel().force(true);
       } finally {
-        referenceMap.removeAllINodeReferenceWithId();
         out.close();
       }
 

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

@@ -41,8 +41,6 @@ import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
-import com.google.common.base.Preconditions;
-
 /**
  * A helper class defining static methods for reading/writing snapshot related
  * information from/to FSImage.
@@ -307,46 +305,23 @@ public class SnapshotFSImageFormat {
   }
   
 
-  /** A reference with a fixed id for fsimage serialization. */
-  private static class INodeReferenceWithId extends INodeReference {
-    final long id;
-
-    private INodeReferenceWithId(WithCount parent, INode referred, long id) {
-      super(parent, referred);
-      this.id = id;
-    }
-    
-    /** @return the reference id. */
-    private long getReferenceId() {
-      return id;
-    }
-  }
-
   /** A reference map for fsimage serialization. */
   public static class ReferenceMap {
     private final Map<Long, INodeReference.WithCount> referenceMap
         = new HashMap<Long, INodeReference.WithCount>();
-    private long referenceId = 0;
 
     public void writeINodeReferenceWithCount(INodeReference.WithCount withCount,
         DataOutput out, boolean writeUnderConstruction) throws IOException {
       final INode referred = withCount.getReferredINode();
-      final boolean firstReferred = !(referred instanceof INodeReferenceWithId);
+      final long id = withCount.getId();
+      final boolean firstReferred = !referenceMap.containsKey(id);
       out.writeBoolean(firstReferred);
 
       if (firstReferred) {
         FSImageSerialization.saveINode2Image(referred, out,
             writeUnderConstruction, this);
-        final long id = ++referenceId;
         referenceMap.put(id, withCount);
-
-        final INodeReferenceWithId withId = new INodeReferenceWithId(
-            withCount, referred, id);
-        withCount.setReferredINode(withId);
-        referred.setParentReference(withId);
       } else {
-        final long id = ((INodeReferenceWithId)referred).getReferenceId();
-        Preconditions.checkState(referenceMap.containsKey(id));
         out.writeLong(id);
       }
     }
@@ -360,7 +335,7 @@ public class SnapshotFSImageFormat {
       if (firstReferred) {
         final INode referred = loader.loadINodeWithLocalName(isSnapshotINode, in);
         withCount = new INodeReference.WithCount(null, referred);
-        referenceMap.put(++referenceId, withCount);
+        referenceMap.put(withCount.getId(), withCount);
       } else {
         final long id = in.readLong();
         withCount = referenceMap.get(id);
@@ -368,16 +343,5 @@ public class SnapshotFSImageFormat {
       }
       return withCount;
     }
-    
-    public void removeAllINodeReferenceWithId() {
-      for(INodeReference.WithCount withCount : referenceMap.values()) {
-        final INodeReference ref = withCount.getReferredINode().asReference();
-        final INode referred = ref.getReferredINode();
-        withCount.setReferredINode(referred);
-        referred.setParentReference(withCount);
-        ref.clear();
-      }
-      referenceMap.clear();
-    }
   }
 }