Browse Source

HDFS-4480. Eliminate the file snapshot circular linked list.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1444280 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
afe77ce53d
20 changed files with 153 additions and 532 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  3. 1 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  4. 1 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  5. 13 47
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  7. 28 71
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  8. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  9. 0 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  10. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  11. 46 128
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java
  12. 0 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectoryWithSnapshot.java
  13. 7 64
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithSnapshot.java
  14. 7 87
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithSnapshot.java
  15. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
  17. 0 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  18. 14 54
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
  19. 25 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  20. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

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

@@ -151,3 +151,5 @@ Branch-2802 Snapshot (Unreleased)
   (Jing Zhao via suresh)
 
   HDFS-4446. Support file snapshots with diff lists.  (szetszwo)
+
+  HDFS-4480. Eliminate the file snapshot circular linked list.  (szetszwo)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -97,7 +97,7 @@ public class LayoutVersion {
         "Serialize block lists with delta-encoded variable length ints, " +
         "add OP_UPDATE_BLOCKS"),
     RESERVED_REL1_2_0(-41, -32, "Reserved for release 1.2.0", true, CONCAT),
-    SNAPSHOT(-41, -39, "Support for snapshot feature", false);
+    SNAPSHOT(-42, -40, "Support for snapshot feature", false);
     
     final int lv;
     final int ancestorLV;

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

@@ -60,7 +60,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
@@ -1207,22 +1206,7 @@ public class FSDirectory implements Closeable {
       final INodeFile newnode, final Snapshot latest) {
     Preconditions.checkState(hasWriteLock());
 
-    INodeDirectory parent = oldnode.getParent();
-    final INode removed = parent.removeChild(oldnode, latest);
-    Preconditions.checkState(removed == oldnode,
-        "removed != oldnode=%s, removed=%s", oldnode, removed);
-
-    //cleanup the removed object
-    parent = removed.getParent(); //parent could be replaced.
-    removed.clearReferences();
-    if (removed instanceof FileWithSnapshot) {
-      final FileWithSnapshot withSnapshot = (FileWithSnapshot)removed;
-      if (withSnapshot.isEverythingDeleted()) {
-        withSnapshot.removeSelf();
-      }
-    }
-
-    parent.addChild(newnode, false, latest);
+    oldnode.getParent().replaceChild(newnode);
 
     /* Currently oldnode and newnode are assumed to contain the same
      * blocks. Otherwise, blocks need to be removed from the blocksMap.

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

@@ -51,7 +51,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -550,7 +549,6 @@ public class FSImageFormat {
     long computeFileSize = -1;
     boolean snapshottable = false;
     boolean withSnapshot = false;
-    boolean withLink = false;
     
     int imgVersion = getLayoutVersion();
     long inodeId = namesystem.allocateNewInodeId();
@@ -582,7 +580,6 @@ public class FSImageFormat {
       if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
         computeFileSize = in.readLong();
         if (computeFileSize < 0) {
-          withLink = in.readBoolean();
         } else {
           underConstruction = in.readBoolean();
           if (underConstruction) {
@@ -621,7 +618,7 @@ public class FSImageFormat {
 
       return INode.newINode(inodeId, permissions, blocks, symlink, replication,
           modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
-          withLink, computeFileSize, snapshottable, withSnapshot,
+          computeFileSize, snapshottable, withSnapshot,
           underConstruction, clientName, clientMachine);
   }
 
@@ -641,10 +638,6 @@ public class FSImageFormat {
         final INodesInPath iip = fsDir.getLastINodeInPath(path);
         INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
         cons.setLocalName(oldnode.getLocalNameBytes());
-        if (oldnode instanceof FileWithSnapshot
-            && cons instanceof FileWithSnapshot) {
-          ((FileWithSnapshot) oldnode).insertBefore((FileWithSnapshot) cons);
-        }
         fsDir.unprotectedReplaceINodeFile(path, oldnode, cons,
             iip.getLatestSnapshot());
         namesystem.leaseManager.addLease(cons.getClientName(), path); 

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

@@ -138,22 +138,21 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 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.SnapshotDiffReport.DiffReportEntry;
 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.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -168,7 +167,6 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Util;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapINodeUpdateEntry;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@@ -2828,48 +2826,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    Iterator<Map.Entry<Block, BlocksMapINodeUpdateEntry>> iter = blocks
-        .iterator();
-    while (iter.hasNext()) {
+    int start = 0;
+    int end = 0;
+    List<Block> toDeleteList = blocks.getToDeleteList();
+    while (start < toDeleteList.size()) {
+      end = BLOCK_DELETION_INCREMENT + start;
+      end = end > toDeleteList.size() ? toDeleteList.size() : end;
       writeLock();
       try {
-        for (int numberToHandle = BLOCK_DELETION_INCREMENT; iter.hasNext()
-            && numberToHandle > 0; numberToHandle--) {
-          Map.Entry<Block, BlocksMapINodeUpdateEntry> entry = iter.next();
-          updateBlocksMap(entry);
+        for (int i = start; i < end; i++) {
+          blockManager.removeBlock(toDeleteList.get(i));
         }
       } finally {
         writeUnlock();
       }
-    }
-  }
-  
-  /**
-   * Update the blocksMap for a given block.
-   * 
-   * @param entry
-   *          The update entry containing both the block and its new INode. The
-   *          block should be removed from the blocksMap if the INode is null,
-   *          otherwise the INode for the block will be updated in the
-   *          blocksMap.
-   */
-  private void updateBlocksMap(
-      Map.Entry<Block, BlocksMapINodeUpdateEntry> entry) {
-    Block block = entry.getKey();
-    BlocksMapINodeUpdateEntry value = entry.getValue();
-    if (value == null) {
-      blockManager.removeBlock(block);
-    } else {
-      BlockCollection toDelete = value.getToDelete();
-      BlockInfo originalBlockInfo = blockManager.getStoredBlock(block);
-      // The FSDirectory tree and the blocksMap share the same INode reference.
-      // Thus we use "==" to check if the INode for the block belongs to the
-      // current file (instead of the INode from a snapshot file).
-      if (originalBlockInfo != null
-          && toDelete == originalBlockInfo.getBlockCollection()) {
-        blockManager.addBlockCollection(originalBlockInfo,
-            value.getToReplace());
-      }
+      start = end;
     }
   }
   
@@ -2891,11 +2862,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     boolean trackBlockCounts = isSafeModeTrackingBlocks();
     int numRemovedComplete = 0, numRemovedSafe = 0;
 
-    Iterator<Map.Entry<Block, BlocksMapINodeUpdateEntry>> blockIter = 
-        blocks.iterator();
-    while (blockIter.hasNext()) {
-      Map.Entry<Block, BlocksMapINodeUpdateEntry> entry = blockIter.next();
-      Block b = entry.getKey();
+    for (Block b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
         BlockInfo bi = blockManager.getStoredBlock(b);
         if (bi.isComplete()) {
@@ -2905,9 +2872,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           }
         }
       }
-      updateBlocksMap(entry);
+      blockManager.removeBlock(b);
     }
-
     if (trackBlockCounts) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Adjusting safe-mode totals for deletion of " + src + ":" +

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

@@ -208,7 +208,7 @@ class FSPermissionChecker {
       if (mode.getOtherAction().implies(access)) { return; }
     }
     throw new AccessControlException("Permission denied: user=" + user
-        + ", access=" + access + ", inode=" + inode);
+        + ", access=" + access + ", inode=" + inode.getFullPathName());
   }
 
   private void checkStickyBit(INode parent, INode inode, Snapshot snapshot

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

@@ -19,10 +19,9 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,7 +32,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
@@ -593,11 +591,10 @@ public abstract class INode implements Diff.Element<byte[]> {
    * @param dsQuota disk quota
    * @param preferredBlockSize block size
    * @param numBlocks number of blocks
-   * @param withLink whether the node is INodeWithLink
    * @param computeFileSize non-negative computeFileSize means the node is 
    *                        INodeFileSnapshot
    * @param snapshottable whether the node is {@link INodeDirectorySnapshottable}
-   * @param withSnapshot whether the node is {@link INodeDirectoryWithSnapshot}
+   * @param withSnapshot whether the node has snapshots
    * @param underConstruction whether the node is 
    *                          {@link INodeFileUnderConstructionSnapshot}
    * @param clientName clientName of {@link INodeFileUnderConstructionSnapshot}
@@ -608,7 +605,7 @@ public abstract class INode implements Diff.Element<byte[]> {
   static INode newINode(long id, PermissionStatus permissions,
       BlockInfo[] blocks, String symlink, short replication,
       long modificationTime, long atime, long nsQuota, long dsQuota,
-      long preferredBlockSize, int numBlocks, boolean withLink,
+      long preferredBlockSize, int numBlocks,
       long computeFileSize, boolean snapshottable, boolean withSnapshot, 
       boolean underConstruction, String clientName, String clientMachine) {
     if (symlink.length() != 0) { // check if symbolic link
@@ -636,7 +633,7 @@ public abstract class INode implements Diff.Element<byte[]> {
 //          fileNode, computeFileSize, clientName, clientMachine)
 //          : new INodeFileWithSnapshot(fileNode, computeFileSize); 
 //    } else {
-      return withLink ? new INodeFileWithSnapshot(fileNode, null) : fileNode;
+      return withSnapshot ? new INodeFileWithSnapshot(fileNode) : fileNode;
 //    }
   }
 
@@ -673,84 +670,44 @@ public abstract class INode implements Diff.Element<byte[]> {
   /**
    * Information used for updating the blocksMap when deleting files.
    */
-  public static class BlocksMapUpdateInfo implements
-      Iterable<Map.Entry<Block, BlocksMapINodeUpdateEntry>> {
-    private final Map<Block, BlocksMapINodeUpdateEntry> updateMap;
+  public static class BlocksMapUpdateInfo {
+    /**
+     * The list of blocks that need to be removed from blocksMap
+     */
+    private List<Block> toDeleteList;
+    
+    public BlocksMapUpdateInfo(List<Block> toDeleteList) {
+      this.toDeleteList = toDeleteList == null ? new ArrayList<Block>()
+          : toDeleteList;
+    }
     
     public BlocksMapUpdateInfo() {
-      updateMap = new HashMap<Block, BlocksMapINodeUpdateEntry>();
+      toDeleteList = new ArrayList<Block>();
+    }
+    
+    /**
+     * @return The list of blocks that need to be removed from blocksMap
+     */
+    public List<Block> getToDeleteList() {
+      return toDeleteList;
     }
     
     /**
-     * Add a to-be-deleted block. This block should belongs to a file without
-     * snapshots. We thus only need to put a block-null pair into the updateMap.
-     * 
+     * Add a to-be-deleted block into the
+     * {@link BlocksMapUpdateInfo#toDeleteList}
      * @param toDelete the to-be-deleted block
      */
     public void addDeleteBlock(Block toDelete) {
       if (toDelete != null) {
-        updateMap.put(toDelete, null);
+        toDeleteList.add(toDelete);
       }
     }
     
     /**
-     * Add a given block, as well as its old and new BlockCollection
-     * information, into the updateMap.
-     * 
-     * @param toUpdateBlock
-     *          The given block
-     * @param entry
-     *          The BlocksMapINodeUpdateEntry instance containing both the
-     *          original BlockCollection of the given block and the new
-     *          BlockCollection of the given block for updating the blocksMap.
-     *          The new BlockCollection should be the INode of one of the
-     *          corresponding file's snapshot.
-     */
-    public void addUpdateBlock(Block toUpdateBlock,
-        BlocksMapINodeUpdateEntry entry) {
-      updateMap.put(toUpdateBlock, entry);
-    }
-
-    /**
-     * Clear {@link BlocksMapUpdateInfo#updateMap}
+     * Clear {@link BlocksMapUpdateInfo#toDeleteList}
      */
     public void clear() {
-      updateMap.clear();
-    }
-
-    @Override
-    public Iterator<Map.Entry<Block, BlocksMapINodeUpdateEntry>> iterator() {
-      return updateMap.entrySet().iterator();
-    }
-  }
-  
-  /**
-   * When deleting a file with snapshot, we cannot directly remove its record
-   * from blocksMap. Instead, we should consider replacing the original record
-   * in blocksMap with INode of snapshot.
-   */
-  public static class BlocksMapINodeUpdateEntry {
-    /**
-     * The BlockCollection of the file to be deleted
-     */
-    private final BlockCollection toDelete;
-    /**
-     * The BlockCollection of the to-be-deleted file's snapshot
-     */
-    private final BlockCollection toReplace;
-
-    public BlocksMapINodeUpdateEntry(BlockCollection toDelete,
-        BlockCollection toReplace) {
-      this.toDelete = toDelete;
-      this.toReplace = toReplace;
-    }
-
-    public BlockCollection getToDelete() {
-      return toDelete;
-    }
-
-    public BlockCollection getToReplace() {
-      return toReplace;
+      toDeleteList.clear();
     }
   }
 }

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

@@ -199,7 +199,7 @@ public class INodeDirectory extends INode {
     return parent.replaceChild(newDir);
   }
 
-  private final <N extends INode> N replaceChild(final N newChild) {
+  final <N extends INode> N replaceChild(final N newChild) {
     assertChildrenNonNull();
     final int i = searchChildrenForExistingINode(newChild);
     final INode oldChild = children.set(i, newChild);
@@ -212,7 +212,7 @@ public class INodeDirectory extends INode {
       final INodeFile child) {
     Preconditions.checkArgument(!(child instanceof INodeFileWithSnapshot),
         "Child file is already an INodeFileWithSnapshot, child=" + child);
-    return replaceChild(new INodeFileWithSnapshot(child, null));
+    return replaceChild(new INodeFileWithSnapshot(child));
   }
 
   /** Replace a child {@link INodeFile} with an {@link INodeFileUnderConstructionWithSnapshot}. */

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

@@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -353,15 +352,6 @@ public class INodeFile extends INode implements BlockCollection {
     out.print(", fileSize=" + computeFileSize(true, snapshot));
     // only compare the first block
     out.print(", blocks=" + (blocks == null? null: blocks[0]));
-    if (this instanceof FileWithSnapshot) {
-      final FileWithSnapshot withSnapshot = (FileWithSnapshot) this;
-      final FileWithSnapshot next = withSnapshot.getNext();
-      // next link pointing to itself is equivalent to no link 
-      if (withSnapshot.getNext() != this) {
-        out.print(", next="
-            + (next != null ? next.asINodeFile().getObjectString() : "null"));
-      }
-    }
     out.println();
   }
 }

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

@@ -123,9 +123,11 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
   protected INodeFile toINodeFile(long mtime) {
     assertAllBlocksComplete();
 
-    return new INodeFile(getId(), getLocalNameBytes(), getPermissionStatus(),
-        mtime, getModificationTime(),
+    final INodeFile f = new INodeFile(getId(), getLocalNameBytes(),
+        getPermissionStatus(), mtime, getModificationTime(),
         getBlocks(), getFileReplication(), getPreferredBlockSize());
+    f.setParent(getParent());
+    return f;
   }
   
   @Override

+ 46 - 128
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshot.java

@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapINodeUpdateEntry;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 
@@ -62,66 +63,47 @@ public interface FileWithSnapshot {
     }
   }
 
+  /**
+   * A list of {@link FileDiff}.
+   */
+  static class FileDiffList extends AbstractINodeDiffList<INodeFile, FileDiff> {
+    final INodeFile currentINode;
+
+    FileDiffList(INodeFile currentINode, List<FileDiff> diffs) {
+      super(diffs);
+      this.currentINode = currentINode;
+    }
+
+    @Override
+    INodeFile getCurrentINode() {
+      return currentINode;
+    }
+
+    @Override
+    FileDiff addSnapshotDiff(Snapshot snapshot) {
+      return addLast(new FileDiff(snapshot, getCurrentINode()));
+    }
+  }
+
   /** @return the {@link INodeFile} view of this object. */
   public INodeFile asINodeFile();
-  
-  /** @return the next element. */
-  public FileWithSnapshot getNext();
 
-  /** Set the next element. */
-  public void setNext(FileWithSnapshot next);
-  
-  /** Insert inode to the circular linked list, after the current node. */
-  public void insertAfter(FileWithSnapshot inode);
-  
-  /** Insert inode to the circular linked list, before the current node. */
-  public void insertBefore(FileWithSnapshot inode);
-  
-  /** Remove self from the circular list */
-  public void removeSelf();
+  /** @return the file diff list. */
+  public FileDiffList getFileDiffList();
 
   /** Is the current file deleted? */
   public boolean isCurrentFileDeleted();
 
-  /** Are the current file and all snapshot copies deleted? */
-  public boolean isEverythingDeleted();
-
-  /** @return the max file replication in the inode and its snapshot copies. */
-  public short getMaxFileReplication();
-  
-  /** @return the max file size in the inode and its snapshot copies. */
-  public long computeMaxFileSize();
-
   /** Utility methods for the classes which implement the interface. */
-  public static class Util {
-    /** @return The previous node in the circular linked list */
-    static FileWithSnapshot getPrevious(FileWithSnapshot file) {
-      FileWithSnapshot previous = file.getNext();
-      while (previous.getNext() != file) {
-        previous = previous.getNext();
-      }
-      return previous;
-    }
-    
-    /** Replace the old file with the new file in the circular linked list. */
-    static void replace(FileWithSnapshot oldFile, FileWithSnapshot newFile) {
-      final FileWithSnapshot oldNext = oldFile.getNext();
-      if (oldNext == null) {
-        newFile.setNext(null);
-      } else {
-        if (oldNext != oldFile) {
-          newFile.setNext(oldNext);
-          getPrevious(oldFile).setNext(newFile);
-        }
-        oldFile.setNext(null);
-      }
-    }
-
-    /** @return the max file replication of the file in the diff list. */
-    static <N extends INodeFile, D extends AbstractINodeDiff<N, D>>
-        short getMaxFileReplication(short max,
-              final AbstractINodeDiffList<N, D> diffs) {
-      for(AbstractINodeDiff<N, D> d : diffs) {
+  static class Util {
+    /** 
+     * @return block replication, which is the max file replication among
+     *         the file and the diff list.
+     */
+    static short getBlockReplication(final FileWithSnapshot file) {
+      short max = file.isCurrentFileDeleted()? 0
+          : file.asINodeFile().getFileReplication();
+      for(FileDiff d : file.getFileDiffList().asList()) {
         if (d.snapshotINode != null) {
           final short replication = d.snapshotINode.getFileReplication();
           if (replication > max) {
@@ -132,67 +114,26 @@ public interface FileWithSnapshot {
       return max;
     }
 
-    /**
-     * @return the max file replication of the elements
-     *         in the circular linked list.
-     */
-    static short getBlockReplication(final FileWithSnapshot file) {
-      short max = file.getMaxFileReplication();
-      // i may be null since next will be set to null when the INode is deleted
-      for(FileWithSnapshot i = file.getNext();
-          i != file && i != null;
-          i = i.getNext()) {
-        final short replication = i.getMaxFileReplication();
-        if (replication > max) {
-          max = replication;
-        }
-      }
-      return max;
-    }
-
     /**
      * If some blocks at the end of the block list no longer belongs to
      * any inode, collect them and update the block list.
      */
     static void collectBlocksAndClear(final FileWithSnapshot file,
         final BlocksMapUpdateInfo info) {
-      final FileWithSnapshot next = file.getNext();
-
-      // find max file size, max replication and the last inode.
-      long maxFileSize = file.computeMaxFileSize();
-      short maxReplication = file.getMaxFileReplication();
-      FileWithSnapshot last = null;
-      if (next != null && next != file) {
-        for(FileWithSnapshot i = next; i != file; i = i.getNext()) {
-          final long size = i.computeMaxFileSize();
-          if (size > maxFileSize) {
-            maxFileSize = size;
-          }
-          final short rep = i.getMaxFileReplication();
-          if (rep > maxReplication) {
-            maxReplication = rep;
-          }
-          last = i;
-        }
+      // find max file size.
+      final long max;
+      if (file.isCurrentFileDeleted()) {
+        final FileDiff last = file.getFileDiffList().getLast();
+        max = last == null? 0: last.fileSize;
+      } else { 
+        max = file.asINodeFile().computeFileSize(true, null);
       }
 
-      collectBlocksBeyondMax(file, maxFileSize, info);
-
-      if (file.isEverythingDeleted()) {
-        // Set the replication of the current INode to the max of all the other
-        // linked INodes, so that in case the current INode is retrieved from the
-        // blocksMap before it is removed or updated, the correct replication
-        // number can be retrieved.
-        if (maxReplication > 0) {
-          file.asINodeFile().setFileReplication(maxReplication, null);
-        }
-
-        // remove the file from the circular linked list.
-        if (last != null) {
-          last.setNext(next);
-        }
-        file.setNext(null);
+      collectBlocksBeyondMax(file, max, info);
 
+      // if everything is deleted, set blocks to null.
+      if (file.isCurrentFileDeleted()
+          && file.getFileDiffList().asList().isEmpty()) {
         file.asINodeFile().setBlocks(null);
       }
     }
@@ -206,16 +147,6 @@ public interface FileWithSnapshot {
         for(long size = 0; n < oldBlocks.length && max > size; n++) {
           size += oldBlocks[n].getNumBytes();
         }
-
-        // collect update blocks
-        final FileWithSnapshot next = file.getNext();
-        if (next != null && next != file && file.isEverythingDeleted() && collectedBlocks != null) {
-          final BlocksMapINodeUpdateEntry entry = new BlocksMapINodeUpdateEntry(
-              file.asINodeFile(), next.asINodeFile());
-          for (int i = 0; i < n; i++) {
-            collectedBlocks.addUpdateBlock(oldBlocks[i], entry);
-          }
-        }
         
         // starting from block n, the data is beyond max.
         if (n < oldBlocks.length) {
@@ -230,9 +161,6 @@ public interface FileWithSnapshot {
           
           // set new blocks
           file.asINodeFile().setBlocks(newBlocks);
-          for(FileWithSnapshot i = next; i != null && i != file; i = i.getNext()) {
-            i.asINodeFile().setBlocks(newBlocks);
-          }
 
           // collect the blocks beyond max.  
           if (collectedBlocks != null) {
@@ -243,15 +171,5 @@ public interface FileWithSnapshot {
         }
       }
     }
-    
-    static String circularListString(final FileWithSnapshot file) {
-      final StringBuilder b = new StringBuilder("* -> ")
-          .append(file.asINodeFile().getObjectString());
-      FileWithSnapshot n = file.getNext();
-      for(; n != null && n != file; n = n.getNext()) {
-        b.append(" -> ").append(n.asINodeFile().getObjectString());
-      }
-      return b.append(n == null? " -> null": " -> *").toString();
-    }
   }
 }

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

@@ -497,12 +497,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
       if (removed == null) {
         //remove failed, undo
         diff.undoDelete(child, undoInfo);
-      } else {
-        //clean up the previously created file, if there is any.
-        final INode trashed = undoInfo.getTrashedElement();
-        if (trashed != null && trashed instanceof FileWithSnapshot) {
-          ((FileWithSnapshot)trashed).removeSelf();
-        }
       }
     }
     return removed;

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

@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot.FileDiffList;
 
 /**
  * Represent an {@link INodeFileUnderConstruction} that is snapshotted.
@@ -67,16 +66,14 @@ public class INodeFileUnderConstructionWithSnapshot
   }
 
   private final FileUcDiffList diffs;
-  private FileWithSnapshot next;
 
   INodeFileUnderConstructionWithSnapshot(final INodeFile f,
       final String clientName,
       final String clientMachine,
-      final DatanodeDescriptor clientNode,
-      final FileDiffList diffs) {
+      final DatanodeDescriptor clientNode) {
     super(f, clientName, clientMachine, clientNode);
-    this.diffs = new FileUcDiffList(this, diffs == null? null: diffs.asList());
-    setNext(this);
+    this.diffs = new FileUcDiffList(this, f instanceof FileWithSnapshot?
+        ((FileWithSnapshot)f).getFileDiffList().asList(): null);
   }
 
   /**
@@ -86,18 +83,16 @@ public class INodeFileUnderConstructionWithSnapshot
    * @param f The given {@link INodeFileUnderConstruction} instance
    */
   public INodeFileUnderConstructionWithSnapshot(INodeFileUnderConstruction f) {
-    this(f, f.getClientName(), f.getClientMachine(), f.getClientNode(), null);
+    this(f, f.getClientName(), f.getClientMachine(), f.getClientNode());
   }
   
   @Override
   protected INodeFileWithSnapshot toINodeFile(final long mtime) {
     assertAllBlocksComplete();
     final long atime = getModificationTime();
-    final INodeFileWithSnapshot f = new INodeFileWithSnapshot(this, diffs);
+    final INodeFileWithSnapshot f = new INodeFileWithSnapshot(this);
     f.setModificationTime(mtime, null);
     f.setAccessTime(atime, null);
-    // link f with this
-    this.insertBefore(f);
     return f;
   }
 
@@ -106,11 +101,6 @@ public class INodeFileUnderConstructionWithSnapshot
     return getParent() == null;
   }
 
-  @Override
-  public boolean isEverythingDeleted() {
-    return isCurrentFileDeleted() && diffs.asList().isEmpty();
-  }
-
   @Override
   public INodeFileUnderConstructionWithSnapshot recordModification(
       final Snapshot latest) {
@@ -128,39 +118,8 @@ public class INodeFileUnderConstructionWithSnapshot
   }
 
   @Override
-  public FileWithSnapshot getNext() {
-    return next;
-  }
-
-  @Override
-  public void setNext(FileWithSnapshot next) {
-    this.next = next;
-  }
-
-  @Override
-  public void insertAfter(FileWithSnapshot inode) {
-    inode.setNext(this.getNext());
-    this.setNext(inode);
-  }
-  
-  @Override
-  public void insertBefore(FileWithSnapshot inode) {
-    inode.setNext(this);
-    if (this.next == null || this.next == this) {
-      this.next = inode;
-      return;
-    }
-    FileWithSnapshot previous = Util.getPrevious(this);
-    previous.setNext(inode);
-  }
-
-  @Override
-  public void removeSelf() {
-    if (this.next != null && this.next != this) {
-      FileWithSnapshot previous = Util.getPrevious(this);
-      previous.setNext(next);
-    }
-    this.next = null;
+  public FileDiffList getFileDiffList() {
+    return diffs;
   }
 
   @Override
@@ -170,12 +129,6 @@ public class INodeFileUnderConstructionWithSnapshot
         : super.getFileReplication(null);
   }
 
-  @Override
-  public short getMaxFileReplication() {
-    final short max = isCurrentFileDeleted()? 0: getFileReplication();
-    return Util.getMaxFileReplication(max, diffs);
-  }
-
   @Override
   public short getBlockReplication() {
     return Util.getBlockReplication(this);
@@ -189,16 +142,6 @@ public class INodeFileUnderConstructionWithSnapshot
         : super.computeFileSize(includesBlockInfoUnderConstruction, null);
   }
 
-  @Override
-  public long computeMaxFileSize() {
-    if (isCurrentFileDeleted()) {
-      final FileDiff last = diffs.getLast();
-      return last == null? 0: last.fileSize;
-    } else { 
-      return super.computeFileSize(true, null);
-    }
-  }
-
   @Override
   public int destroySubtreeAndCollectBlocks(final Snapshot snapshot,
       final BlocksMapUpdateInfo collectedBlocks) {

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

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import java.util.List;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -31,35 +29,12 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 @InterfaceAudience.Private
 public class INodeFileWithSnapshot extends INodeFile
     implements FileWithSnapshot {
-  /**
-   * A list of file diffs.
-   */
-  static class FileDiffList extends AbstractINodeDiffList<INodeFile, FileDiff> {
-    final INodeFile currentINode;
-
-    FileDiffList(INodeFile currentINode, List<FileDiff> diffs) {
-      super(diffs);
-      this.currentINode = currentINode;
-    }
-
-    @Override
-    INodeFile getCurrentINode() {
-      return currentINode;
-    }
-
-    @Override
-    FileDiff addSnapshotDiff(Snapshot snapshot) {
-      return addLast(new FileDiff(snapshot, getCurrentINode()));
-    }
-  }
-
   private final FileDiffList diffs;
-  private FileWithSnapshot next;
 
-  public INodeFileWithSnapshot(INodeFile f, FileDiffList diffs) {
+  public INodeFileWithSnapshot(INodeFile f) {
     super(f);
-    this.diffs = new FileDiffList(this, diffs == null? null: diffs.asList());
-    setNext(this);
+    this.diffs = new FileDiffList(this, f instanceof FileWithSnapshot?
+        ((FileWithSnapshot)f).getFileDiffList().asList(): null);
   }
 
   @Override
@@ -67,11 +42,8 @@ public class INodeFileWithSnapshot extends INodeFile
       final String clientName,
       final String clientMachine,
       final DatanodeDescriptor clientNode) {
-    final INodeFileUnderConstructionWithSnapshot f
-        = new INodeFileUnderConstructionWithSnapshot(this,
-            clientName, clientMachine, clientNode, diffs);
-    this.insertBefore(f);
-    return f;
+    return new INodeFileUnderConstructionWithSnapshot(this,
+        clientName, clientMachine, clientNode);
   }
 
   @Override
@@ -79,11 +51,6 @@ public class INodeFileWithSnapshot extends INodeFile
     return getParent() == null;
   }
 
-  @Override
-  public boolean isEverythingDeleted() {
-    return isCurrentFileDeleted() && diffs.asList().isEmpty();
-  }
-
   @Override
   public INodeFileWithSnapshot recordModification(final Snapshot latest) {
     // if this object is NOT the latest snapshot copy, this object is created
@@ -100,39 +67,8 @@ public class INodeFileWithSnapshot extends INodeFile
   }
 
   @Override
-  public FileWithSnapshot getNext() {
-    return next;
-  }
-
-  @Override
-  public void setNext(FileWithSnapshot next) {
-    this.next = next;
-  }
-
-  @Override
-  public void insertAfter(FileWithSnapshot inode) {
-    inode.setNext(this.getNext());
-    this.setNext(inode);
-  }
-  
-  @Override
-  public void insertBefore(FileWithSnapshot inode) {
-    inode.setNext(this);
-    if (this.next == null || this.next == this) {
-      this.next = inode;
-      return;
-    }
-    FileWithSnapshot previous = Util.getPrevious(this);
-    previous.setNext(inode);
-  }
-
-  @Override
-  public void removeSelf() {
-    if (this.next != null && this.next != this) {
-      FileWithSnapshot previous = Util.getPrevious(this);
-      previous.setNext(next);
-    }
-    this.next = null;
+  public FileDiffList getFileDiffList() {
+    return diffs;
   }
 
   @Override
@@ -142,12 +78,6 @@ public class INodeFileWithSnapshot extends INodeFile
         : super.getFileReplication(null);
   }
 
-  @Override
-  public short getMaxFileReplication() {
-    final short max = isCurrentFileDeleted()? 0: getFileReplication();
-    return Util.getMaxFileReplication(max, diffs);
-  }
-
   @Override
   public short getBlockReplication() {
     return Util.getBlockReplication(this);
@@ -161,16 +91,6 @@ public class INodeFileWithSnapshot extends INodeFile
         : super.computeFileSize(includesBlockInfoUnderConstruction, null);
   }
 
-  @Override
-  public long computeMaxFileSize() {
-    if (isCurrentFileDeleted()) {
-      final FileDiff last = diffs.getLast();
-      return last == null? 0: last.fileSize;
-    } else { 
-      return super.computeFileSize(true, null);
-    }
-  }
-
   @Override
   public int destroySubtreeAndCollectBlocks(final Snapshot snapshot,
       final BlocksMapUpdateInfo collectedBlocks) {

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

@@ -201,12 +201,8 @@ public class SnapshotFSImageFormat {
         }
         // deleted must be an FileWithSnapshot (INodeFileSnapshot or 
         // INodeFileUnderConstructionSnapshot)
-        FileWithSnapshot deletedWithLink = (FileWithSnapshot) deleted;
         INodeFile cNode = (INodeFile) createdList.get(c);
-        INodeFileWithSnapshot cNodeWithLink = (INodeFileWithSnapshot) cNode;
         ((INodeFile) deleted).setBlocks(cNode.getBlocks());
-        // insert deleted into the circular list
-        cNodeWithLink.insertBefore(deletedWithLink);
       }
     }
     return deletedList;

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

@@ -411,7 +411,7 @@ public class TestSnapshotPathINodes {
     // The modification time of the snapshot INode should be the same with the
     // original INode before modification
     assertEquals(inodes[inodes.length - 1].getModificationTime(),
-        ssInodes[ssInodes.length - 1].getModificationTime());
+        snapshotFileNode.getModificationTime(ssNodesInPath.getPathSnapshot()));
 
     // Check the INode for /TestSnapshot/sub1/file1 again
     names = INode.getPathNames(file1.toString());

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

@@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.Util;
 import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
@@ -244,26 +243,6 @@ public class SnapshotTestHelper {
     }
     return null;
   }
-  
-  /**
-   * Check if the given nodes can form a circular list
-   */
-  static void checkCircularList(INodeFile... nodes) {
-    for (int i = 0; i < nodes.length; i++) {
-      FileWithSnapshot next = ((FileWithSnapshot)nodes[i]).getNext();
-      INodeFile expectedNext = nodes[(i + 1) % nodes.length];
-      if (next != expectedNext) {
-        final StringBuilder b = new StringBuilder("nodes = [")
-            .append(nodes[0].getObjectString());
-        for(int j = 1; j < nodes.length; j++) {
-          b.append(", ").append(nodes[i].getObjectString());
-        }
-        b.append("]\nbut the circular list of nodes[").append(i).append("] is ")
-         .append(Util.circularListString((FileWithSnapshot)nodes[i]));
-        throw new AssertionError(b.toString());
-      }
-    }
-  }
 
   /**
    * A class creating directories trees for snapshot testing. For simplicity,

+ 14 - 54
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java

@@ -18,13 +18,13 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.Random;
 
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -36,9 +36,10 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.ChildrenDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiff;
+import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -47,6 +48,10 @@ import org.junit.Test;
  * Test snapshot functionalities while file appending.
  */
 public class TestINodeFileUnderConstructionWithSnapshot {
+  {
+    ((Log4JLogger)INode.LOG).getLogger().setLevel(Level.ALL);
+    SnapshotTestHelper.disableLogs();
+  }
 
   static final long seed = 0;
   static final short REPLICATION = 3;
@@ -91,55 +96,26 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
     DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
     
-    // check the circular list and corresponding inodes: there should only be a
-    // reference of the current node in the created list
     INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString());
-    final byte[] filename = fileNode.getLocalNameBytes(); 
-    INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir
-        .getINode(dir.toString());
-    ChildrenDiff diff = dirNode.getDiffs().getLast().getChildrenDiff();
-    INodeFile nodeInCreated = (INodeFile)diff.searchCreated(filename);
-    assertTrue(fileNode == nodeInCreated);
-    INodeFile nodeInDeleted = (INodeFile)diff.searchDeleted(filename);
-    assertNull(nodeInDeleted);
     
     // 2. create snapshot --> modify the file --> append
     hdfs.createSnapshot(dir, "s1");
     hdfs.setReplication(file, (short) (REPLICATION - 1));
     DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
     
-    // check the circular list and corresponding inodes
-    DirectoryDiff last = dirNode.getDiffs().getLast();
-    Snapshot snapshot = last.snapshot;
-    diff = last.getChildrenDiff();
+    // check corresponding inodes
     fileNode = (INodeFile) fsdir.getINode(file.toString());
-    nodeInCreated = (INodeFile)diff.searchCreated(filename);
-    assertTrue(fileNode == nodeInCreated);
     assertEquals(REPLICATION - 1, fileNode.getFileReplication());
     assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(true));
-    nodeInDeleted = (INodeFile)diff.searchDeleted(filename);
-    assertEquals(REPLICATION, nodeInDeleted.getFileReplication(snapshot));
-    assertEquals(BLOCKSIZE * 2, nodeInDeleted.computeFileSize(true, snapshot));
-    SnapshotTestHelper.checkCircularList(fileNode, nodeInDeleted);
 
     // 3. create snapshot --> append
     hdfs.createSnapshot(dir, "s2");
     DFSTestUtil.appendFile(hdfs, file, BLOCKSIZE);
     
-    // check the circular list and corresponding inodes
-    last = dirNode.getDiffs().getLast();
-    snapshot = last.snapshot;
-    diff = last.getChildrenDiff();
+    // check corresponding inodes
     fileNode = (INodeFile) fsdir.getINode(file.toString());
-    nodeInCreated = (INodeFile)diff.searchCreated(filename);
-    assertTrue(fileNode == nodeInCreated);
-    assertEquals(REPLICATION - 1,  nodeInCreated.getFileReplication());
+    assertEquals(REPLICATION - 1,  fileNode.getFileReplication());
     assertEquals(BLOCKSIZE * 4, fileNode.computeFileSize(true));
-    INodeFile nodeInDeleted2 = (INodeFile)diff.searchDeleted(filename);
-    assertEquals(REPLICATION - 1, nodeInDeleted2.getFileReplication());
-    assertEquals(BLOCKSIZE * 3, nodeInDeleted2.computeFileSize(true, snapshot));
-    SnapshotTestHelper.checkCircularList(fileNode, nodeInDeleted2, nodeInDeleted);
-
   }
   
   private HdfsDataOutputStream appendFileWithoutClosing(Path file, int length)
@@ -170,16 +146,11 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     // check: an INodeFileUnderConstructionWithSnapshot should be stored into s0's
     // deleted list, with size BLOCKSIZE*2
     INodeFile fileNode = (INodeFile) fsdir.getINode(file.toString());
-    final byte[] filename = fileNode.getLocalNameBytes(); 
-    assertEquals(BLOCKSIZE * 2, ((INodeFile) fileNode).computeFileSize(true));
+    assertEquals(BLOCKSIZE * 2, fileNode.computeFileSize(true));
     INodeDirectorySnapshottable dirNode = (INodeDirectorySnapshottable) fsdir
         .getINode(dir.toString());
     DirectoryDiff last = dirNode.getDiffs().getLast();
     Snapshot s0 = last.snapshot;
-    ChildrenDiff diff = last.getChildrenDiff();
-    INodeFileUnderConstructionWithSnapshot nodeInDeleted_S0
-        = (INodeFileUnderConstructionWithSnapshot)diff.searchDeleted(filename);
-    assertEquals(BLOCKSIZE * 2, nodeInDeleted_S0.computeFileSize(true, s0));
     
     // 2. append without closing stream
     out = appendFileWithoutClosing(file, BLOCKSIZE);
@@ -187,10 +158,7 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     
     // re-check nodeInDeleted_S0
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
-    diff = dirNode.getDiffs().getLast().getChildrenDiff();
-    nodeInDeleted_S0
-        = (INodeFileUnderConstructionWithSnapshot)diff.searchDeleted(filename);
-    assertEquals(BLOCKSIZE * 2, nodeInDeleted_S0.computeFileSize(true, s0));
+    assertEquals(BLOCKSIZE * 2, fileNode.computeFileSize(true, s0));
     
     // 3. take snapshot --> close stream
     hdfs.createSnapshot(dir, "s1");
@@ -202,16 +170,8 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     dirNode = (INodeDirectorySnapshottable) fsdir.getINode(dir.toString());
     last = dirNode.getDiffs().getLast();
     Snapshot s1 = last.snapshot;
-    diff = last.getChildrenDiff();
-    INodeFile nodeInCreated_S1 = (INodeFile)diff.searchCreated(filename);
-    assertTrue(fileNode == nodeInCreated_S1);
     assertTrue(fileNode instanceof INodeFileWithSnapshot);
-    INodeFile nodeInDeleted_S1 = (INodeFile)diff.searchDeleted(filename);
-    assertTrue(nodeInDeleted_S1 instanceof INodeFileUnderConstructionWithSnapshot);
-    assertEquals(BLOCKSIZE * 3, nodeInDeleted_S1.computeFileSize(true, s1));
-    // also check the circular linked list
-    SnapshotTestHelper.checkCircularList(
-        fileNode, nodeInDeleted_S1, nodeInDeleted_S0);
+    assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(true, s1));
     
     // 4. modify file --> append without closing stream --> take snapshot -->
     // close stream
@@ -221,6 +181,6 @@ public class TestINodeFileUnderConstructionWithSnapshot {
     out.close();
     
     // re-check the size of nodeInDeleted_S1
-    assertEquals(BLOCKSIZE * 3, nodeInDeleted_S1.computeFileSize(true, s1));
+    assertEquals(BLOCKSIZE * 3, fileNode.computeFileSize(true, s1));
   }  
 }

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

@@ -25,6 +25,7 @@ import java.io.FileWriter;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Random;
 
@@ -129,7 +130,7 @@ public class TestSnapshot {
   private void modifyCurrentDirAndCheckSnapshots(Modification[] modifications)
       throws Exception {
     for (Modification modification : modifications) {
-      System.out.println(++modificationCount + ") modification = " + modification);
+      System.out.println(++modificationCount + ") " + modification);
       modification.loadSnapshots();
       modification.modify();
       modification.checkSnapshots();
@@ -611,17 +612,30 @@ public class TestSnapshot {
         }
         assertEquals(s, originalSnapshotFileLen, currentSnapshotFileLen);
         // Read the snapshot file out of the boundary
-        if (currentSnapshotFileLen != -1L) {
+        if (currentSnapshotFileLen != -1L
+            && !(this instanceof FileAppendNotClose)) {
           FSDataInputStream input = fs.open(snapshotFile);
           int readLen = input.read(currentSnapshotFileLen, buffer, 0, 1);
-          assertEquals(readLen, -1);
+          if (readLen != -1) {
+            s = "FAILED: " + getClass().getSimpleName()
+                + ": file="  + file + ", snapshotFile" + snapshotFile
+                + "\n\n currentSnapshotFileLen = " + currentSnapshotFileLen
+                +   "\n                readLen = " + readLen
+                + "\n\nfile        : " + fsdir.getINode(file.toString()).toDetailString()
+                + "\n\nsnapshotFile: " + fsdir.getINode(snapshotFile.toString()).toDetailString();
+            
+            System.out.println(s);
+            SnapshotTestHelper.dumpTreeRecursively(fsdir.getINode("/"));
+          }
+          assertEquals(s, -1, readLen);
+          input.close();
         }
       }
     }
   }
 
   /**
-   * Appending a specified length to an existing file
+   * Appending a specified length to an existing file but not close the file
    */
   static class FileAppendNotClose extends FileAppend {
     HdfsDataOutputStream out;
@@ -638,7 +652,7 @@ public class TestSnapshot {
 
       out = (HdfsDataOutputStream)fs.append(file);
       out.write(toAppend);
-      out.hflush();
+      out.hsync(EnumSet.of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
     }
   }
 
@@ -648,7 +662,8 @@ public class TestSnapshot {
   static class FileAppendClose extends FileAppend {
     final FileAppendNotClose fileAppendNotClose;
 
-    FileAppendClose(Path file, FileSystem fs, int len, FileAppendNotClose fileAppendNotClose) {
+    FileAppendClose(Path file, FileSystem fs, int len,
+        FileAppendNotClose fileAppendNotClose) {
       super(file, fs, len);
       this.fileAppendNotClose = fileAppendNotClose;
     }
@@ -656,6 +671,10 @@ public class TestSnapshot {
     @Override
     void modify() throws Exception {
       assertTrue(fs.exists(file));
+      byte[] toAppend = new byte[appendLen];
+      random.nextBytes(toAppend);
+
+      fileAppendNotClose.out.write(toAppend);
       fileAppendNotClose.out.close();
     }
   }

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

@@ -303,11 +303,9 @@ public class TestSnapshotDeletion {
     
     // call getBlockReplication, check circular list after snapshot deletion
     INodeFile nodeFile13 = (INodeFile)fsdir.getINode(file13.toString());
-    SnapshotTestHelper.checkCircularList(nodeFile13);
     assertEquals(REP_1, nodeFile13.getBlockReplication());
 
     INodeFile nodeFile12 = (INodeFile)fsdir.getINode(file12_s1.toString());
-    SnapshotTestHelper.checkCircularList(nodeFile12);
     assertEquals(REP_1, nodeFile12.getBlockReplication());
   }