Browse Source

HDFS-4098. Add FileWithLink, INodeFileUnderConstructionWithLink and INodeFileUnderConstructionSnapshot in order to support append to snapshotted files.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1432788 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
25116c26fd

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

@@ -100,3 +100,7 @@ Branch-2802 Snapshot (Unreleased)
 
 
   HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
   HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
   (Jing Zhao via szetszwo)
   (Jing Zhao via szetszwo)
+
+  HDFS-4098. Add FileWithLink, INodeFileUnderConstructionWithLink and
+  INodeFileUnderConstructionSnapshot in order to support append to snapshotted
+  files.  (szetszwo)

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

@@ -323,7 +323,7 @@ public class FSEditLogLoader {
       if (oldFile.isUnderConstruction()) {
       if (oldFile.isUnderConstruction()) {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
-        INodeFile newFile = ucFile.convertToInodeFile(ucFile.getModificationTime());
+        INodeFile newFile = ucFile.toINodeFile(ucFile.getModificationTime());
         fsDir.unprotectedReplaceINodeFile(addCloseOp.path, ucFile, newFile,
         fsDir.unprotectedReplaceINodeFile(addCloseOp.path, ucFile, newFile,
             iip.getLatestSnapshot());
             iip.getLatestSnapshot());
       }
       }

+ 12 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1324,10 +1324,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           doAccessTime = false;
           doAccessTime = false;
         }
         }
 
 
-        long now = now();
-        final INodesInPath iip = dir.getMutableINodesInPath(src);
+        final INodesInPath iip = dir.getINodesInPath(src);
         final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
         final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
-        if (doAccessTime && isAccessTimeSupported()) {
+        if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
+            && doAccessTime && isAccessTimeSupported()) {
+          final long now = now();
           if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
           if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
             // if we have to set access time but we only have the readlock, then
             // if we have to set access time but we only have the readlock, then
             // restart this entire operation with the writeLock.
             // restart this entire operation with the writeLock.
@@ -1948,17 +1949,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
       boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
-    //TODO SNAPSHOT: INodeFileUnderConstruction with link
-    INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                                    file.getLocalNameBytes(),
-                                    file.getFileReplication(),
-                                    file.getModificationTime(),
-                                    file.getPreferredBlockSize(),
-                                    file.getBlocks(),
-                                    file.getPermissionStatus(),
-                                    leaseHolder,
-                                    clientMachine,
-                                    clientNode);
+    if (latestSnapshot != null) {
+      file = (INodeFile)file.recordModification(latestSnapshot).left;
+    }
+    final INodeFileUnderConstruction cons
+        = INodeFileUnderConstruction.toINodeFileUnderConstruction(
+            file, leaseHolder, clientMachine, clientNode);
+
     dir.replaceINodeFile(src, file, cons, latestSnapshot);
     dir.replaceINodeFile(src, file, cons, latestSnapshot);
     leaseManager.addLease(cons.getClientName(), src);
     leaseManager.addLease(cons.getClientName(), src);
     
     
@@ -3271,7 +3268,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
     // The file is no longer pending.
     // The file is no longer pending.
     // Create permanent INode, update blocks
     // Create permanent INode, update blocks
-    INodeFile newFile = pendingFile.convertToInodeFile(now());
+    INodeFile newFile = pendingFile.toINodeFile(now());
     dir.replaceINodeFile(src, pendingFile, newFile, latestSnapshot);
     dir.replaceINodeFile(src, pendingFile, newFile, latestSnapshot);
 
 
     // close file and persist block allocations for this file
     // close file and persist block allocations for this file

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

@@ -497,7 +497,7 @@ public abstract class INode implements Comparable<byte[]> {
   /**
   /**
    * Set last access time of inode.
    * Set last access time of inode.
    */
    */
-  INode setAccessTime(long atime, Snapshot latest) {
+  public INode setAccessTime(long atime, Snapshot latest) {
     Pair<? extends INode, ? extends INode> pair = recordModification(latest);
     Pair<? extends INode, ? extends INode> pair = recordModification(latest);
     INode nodeToUpdate = pair != null ? pair.left : this;    
     INode nodeToUpdate = pair != null ? pair.left : this;    
     nodeToUpdate.accessTime = atime;
     nodeToUpdate.accessTime = atime;

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -28,8 +28,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileSnapshot;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithLink;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 
 /** I-node for closed file. */
 /** I-node for closed file. */
@@ -96,7 +94,7 @@ public class INodeFile extends INode implements BlockCollection {
         preferredBlockSize);
         preferredBlockSize);
   }
   }
 
 
-  INodeFile(byte[] name, PermissionStatus permissions, long mtime, long atime,
+  protected INodeFile(byte[] name, PermissionStatus permissions, long mtime, long atime,
       BlockInfo[] blklist, short replication, long preferredBlockSize) {
       BlockInfo[] blklist, short replication, long preferredBlockSize) {
     super(name, permissions, null, mtime, atime);
     super(name, permissions, null, mtime, atime);
     header = HeaderFormat.combineReplication(header, replication);
     header = HeaderFormat.combineReplication(header, replication);
@@ -111,7 +109,7 @@ public class INodeFile extends INode implements BlockCollection {
   }
   }
 
 
   @Override
   @Override
-  public Pair<INodeFileWithLink, INodeFileSnapshot> createSnapshotCopy() {
+  public Pair<? extends INodeFile, ? extends INodeFile> createSnapshotCopy() {
     return parent.replaceINodeFile(this).createSnapshotCopy();
     return parent.replaceINodeFile(this).createSnapshotCopy();
   }
   }
 
 
@@ -141,7 +139,7 @@ public class INodeFile extends INode implements BlockCollection {
     return getFileReplication();
     return getFileReplication();
   }
   }
 
 
-  protected void setFileReplication(short replication, Snapshot latest) {
+  public void setFileReplication(short replication, Snapshot latest) {
     if (latest != null) {
     if (latest != null) {
       final Pair<? extends INode, ? extends INode> p = recordModification(latest);
       final Pair<? extends INode, ? extends INode> p = recordModification(latest);
       if (p != null) {
       if (p != null) {

+ 49 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -29,6 +29,10 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithLink;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithLink;
+
+import com.google.common.base.Preconditions;
 
 
 /**
 /**
  * I-node for file being written.
  * I-node for file being written.
@@ -45,6 +49,28 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
     return (INodeFileUnderConstruction)file;
     return (INodeFileUnderConstruction)file;
   }
   }
 
 
+  /** Convert the given file to an {@link INodeFileUnderConstruction}. */
+  public static INodeFileUnderConstruction toINodeFileUnderConstruction(
+      INodeFile file,
+      String clientName,
+      String clientMachine,
+      DatanodeDescriptor clientNode) {
+    Preconditions.checkArgument(!(file instanceof INodeFileUnderConstruction),
+        "file is already an INodeFileUnderConstruction");
+    final INodeFileUnderConstruction uc = new INodeFileUnderConstruction(
+        file.getLocalNameBytes(),
+        file.getFileReplication(),
+        file.getModificationTime(),
+        file.getPreferredBlockSize(),
+        file.getBlocks(),
+        file.getPermissionStatus(),
+        clientName,
+        clientMachine,
+        clientNode);
+    return file instanceof INodeFileWithLink?
+        new INodeFileUnderConstructionWithLink(uc): uc;
+  }
+
   private  String clientName;         // lease holder
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -56,11 +82,8 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
                              String clientName,
                              String clientName,
                              String clientMachine,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
                              DatanodeDescriptor clientNode) {
-    super(permissions.applyUMask(UMASK), BlockInfo.EMPTY_ARRAY, replication,
-        modTime, modTime, preferredBlockSize);
-    this.clientName = clientName;
-    this.clientMachine = clientMachine;
-    this.clientNode = clientNode;
+    this(null, replication, modTime, preferredBlockSize, BlockInfo.EMPTY_ARRAY,
+        permissions.applyUMask(UMASK), clientName, clientMachine, clientNode);
   }
   }
 
 
   INodeFileUnderConstruction(byte[] name,
   INodeFileUnderConstruction(byte[] name,
@@ -78,6 +101,13 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
     this.clientMachine = clientMachine;
     this.clientMachine = clientMachine;
     this.clientNode = clientNode;
     this.clientNode = clientNode;
   }
   }
+ 
+  protected INodeFileUnderConstruction(INodeFileUnderConstruction that) {
+    super(that);
+    this.clientName = that.clientName;
+    this.clientMachine = that.clientMachine;
+    this.clientNode = that.clientNode;
+  }
 
 
   String getClientName() {
   String getClientName() {
     return clientName;
     return clientName;
@@ -103,30 +133,26 @@ public class INodeFileUnderConstruction extends INodeFile implements MutableBloc
     return true;
     return true;
   }
   }
 
 
-  //
-  // converts a INodeFileUnderConstruction into a INodeFile
-  // use the modification time as the access time
-  //
-  INodeFile convertToInodeFile(long mtime) {
-    assert allBlocksComplete() : "Can't finalize inode " + this
-      + " since it contains non-complete blocks! Blocks are "
-      + Arrays.asList(getBlocks());
-    //TODO SNAPSHOT: may convert to INodeFileWithLink
+  /**
+   * Converts an INodeFileUnderConstruction to an INodeFile.
+   * The original modification time is used as the access time.
+   * The new modification is the specified mtime.
+   */
+  protected INodeFile toINodeFile(long mtime) {
+    assertAllBlocksComplete();
     return new INodeFile(getLocalNameBytes(), getPermissionStatus(),
     return new INodeFile(getLocalNameBytes(), getPermissionStatus(),
         mtime, getModificationTime(),
         mtime, getModificationTime(),
         getBlocks(), getFileReplication(), getPreferredBlockSize());
         getBlocks(), getFileReplication(), getPreferredBlockSize());
   }
   }
   
   
-  /**
-   * @return true if all of the blocks in this file are marked as completed.
-   */
-  private boolean allBlocksComplete() {
-    for (BlockInfo b : getBlocks()) {
-      if (!b.isComplete()) {
-        return false;
-      }
+  /** Assert all blocks are complete. */
+  protected void assertAllBlocksComplete() {
+    final BlockInfo[] blocks = getBlocks();
+    for (int i = 0; i < blocks.length; i++) {
+      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
+          + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
+          getClass().getSimpleName(), this, i, Arrays.asList(getBlocks()));
     }
     }
-    return true;
   }
   }
 
 
   /**
   /**

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

@@ -0,0 +1,154 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapINodeUpdateEntry;
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * {@link INodeFile} with a link to the next element.
+ * The link of all the snapshot files and the original file form a circular
+ * linked list so that all elements are accessible by any of the elements.
+ */
+@InterfaceAudience.Private
+public interface FileWithLink {
+  /** @return the next element. */
+  public <N extends INodeFile & FileWithLink> N getNext();
+
+  /** Set the next element. */
+  public <N extends INodeFile & FileWithLink> void setNext(N next);
+  
+  /** Insert inode to the circular linked list. */
+  public <N extends INodeFile & FileWithLink> void insert(N inode);
+  
+  /** Utility methods for the classes which implement the interface. */
+  static class Util {
+    /**
+     * @return the max file replication of the elements
+     *         in the circular linked list.
+     */
+    static <F extends INodeFile & FileWithLink,
+            N extends INodeFile & FileWithLink> short getBlockReplication(
+        final F file) {
+      short max = file.getFileReplication();
+      // i may be null since next will be set to null when the INode is deleted
+      for(N i = file.getNext(); i != file && i != null; i = i.getNext()) {
+        final short replication = i.getFileReplication();
+        if (replication > max) {
+          max = replication;
+        }
+      }
+      return max;
+    }
+
+    /**
+     * Remove the current inode from the circular linked list.
+     * If some blocks at the end of the block list no longer belongs to
+     * any other inode, collect them and update the block list.
+     */
+    static <F extends INodeFile & FileWithLink,
+            N extends INodeFile & FileWithLink>
+        int collectSubtreeBlocksAndClear(F file, BlocksMapUpdateInfo info) {
+      final N next = file.getNext();
+      Preconditions.checkState(next != file, "this is the only remaining inode.");
+
+      // There are other inode(s) using the blocks.
+      // Compute max file size excluding this and find the last inode.
+      long max = next.computeFileSize(true);
+      short maxReplication = next.getFileReplication();
+      FileWithLink last = next;
+      for(N i = next.getNext(); i != file; i = i.getNext()) {
+        final long size = i.computeFileSize(true);
+        if (size > max) {
+          max = size;
+        }
+        final short rep = i.getFileReplication();
+        if (rep > maxReplication) {
+          maxReplication = rep;
+        }
+        last = i;
+      }
+
+      collectBlocksBeyondMaxAndClear(file, max, info);
+      
+      // remove this from the circular linked list.
+      last.setNext(next);
+      // 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.
+      file.setFileReplication(maxReplication, null);
+      file.setNext(null);
+      // clear parent
+      file.setParent(null);
+      return 1;
+    }
+
+    static <F extends INodeFile & FileWithLink,
+            N extends INodeFile & FileWithLink>
+        void collectBlocksBeyondMaxAndClear(final F file,
+            final long max, final BlocksMapUpdateInfo info) {
+      final BlockInfo[] oldBlocks = file.getBlocks();
+      if (oldBlocks != null) {
+        //find the minimum n such that the size of the first n blocks > max
+        int n = 0;
+        for(long size = 0; n < oldBlocks.length && max > size; n++) {
+          size += oldBlocks[n].getNumBytes();
+        }
+
+        // Replace the INode for all the remaining blocks in blocksMap
+        final N next = file.getNext();
+        final BlocksMapINodeUpdateEntry entry = new BlocksMapINodeUpdateEntry(
+            file, next);
+        if (info != null) {
+          for (int i = 0; i < n; i++) {
+            info.addUpdateBlock(oldBlocks[i], entry);
+          }
+        }
+        
+        // starting from block n, the data is beyond max.
+        if (n < oldBlocks.length) {
+          // resize the array.  
+          final BlockInfo[] newBlocks;
+          if (n == 0) {
+            newBlocks = null;
+          } else {
+            newBlocks = new BlockInfo[n];
+            System.arraycopy(oldBlocks, 0, newBlocks, 0, n);
+          }
+          for(N i = next; i != file; i = i.getNext()) {
+            i.setBlocks(newBlocks);
+          }
+
+          // collect the blocks beyond max.  
+          if (info != null) {
+            for(; n < oldBlocks.length; n++) {
+              info.addDeleteBlock(oldBlocks[n]);
+            }
+          }
+        }
+        file.setBlocks(null);
+      }
+    }
+  }
+}

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

@@ -0,0 +1,44 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+
+/**
+ *  INode representing a snapshot of an {@link INodeFileUnderConstruction}.
+ */
+@InterfaceAudience.Private
+public class INodeFileUnderConstructionSnapshot
+    extends INodeFileUnderConstructionWithLink {
+  /** The file size at snapshot creation time. */
+  final long size;
+
+  INodeFileUnderConstructionSnapshot(INodeFileUnderConstructionWithLink f) {
+    super(f);
+    this.size = f.computeFileSize(true);
+    f.insert(this);
+  }
+
+  @Override
+  public long computeFileSize(boolean includesBlockInfoUnderConstruction) {
+    //ignore includesBlockInfoUnderConstruction 
+    //since files in a snapshot are considered as closed.
+    return size;
+  }
+}

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

@@ -0,0 +1,88 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+
+/**
+ * Represent an {@link INodeFileUnderConstruction} that is snapshotted.
+ * Note that snapshot files are represented by
+ * {@link INodeFileUnderConstructionSnapshot}.
+ */
+@InterfaceAudience.Private
+public class INodeFileUnderConstructionWithLink
+    extends INodeFileUnderConstruction implements FileWithLink {
+  private FileWithLink next;
+
+  public INodeFileUnderConstructionWithLink(INodeFileUnderConstruction f) {
+    super(f);
+    setNext(this);
+  }
+
+  @Override
+  protected INodeFileWithLink toINodeFile(final long mtime) {
+    assertAllBlocksComplete();
+    final long atime = getModificationTime();
+    final INodeFileWithLink f = new INodeFileWithLink(this);
+    f.setModificationTime(mtime, null);
+    f.setAccessTime(atime, null);
+    return f;
+  }
+
+  @Override
+  public Pair<? extends INodeFileUnderConstruction,
+      INodeFileUnderConstructionSnapshot> createSnapshotCopy() {
+    return new Pair<INodeFileUnderConstructionWithLink,
+        INodeFileUnderConstructionSnapshot>(
+            this, new INodeFileUnderConstructionSnapshot(this));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <N extends INodeFile & FileWithLink> N getNext() {
+    return (N)next;
+  }
+
+  @Override
+  public <N extends INodeFile & FileWithLink> void setNext(N next) {
+    this.next = next;
+  }
+  
+  @Override
+  public <N extends INodeFile & FileWithLink> void insert(N inode) {
+    inode.setNext(this.getNext());
+    this.setNext(inode);
+  }
+
+  @Override
+  public short getBlockReplication() {
+    return Util.getBlockReplication(this);
+  }
+
+  @Override
+  protected int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
+    if (next == null || next == this) {
+      // this is the only remaining inode.
+      return super.collectSubtreeBlocksAndClear(info);
+    } else {
+      return Util.collectSubtreeBlocksAndClear(this, info);
+    }
+  }
+}

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

@@ -18,23 +18,19 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 
 
 /**
 /**
- * INodeFile with a link to the next element.
- * This class is used to represent the original file that is snapshotted.
- * The snapshot files are represented by {@link INodeFileSnapshot}.
- * The link of all the snapshot files and the original file form a circular
- * linked list so that all elements are accessible by any of the elements.
+ * Represent an {@link INodeFile} that is snapshotted.
+ * Note that snapshot files are represented by {@link INodeFileSnapshot}.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class INodeFileWithLink extends INodeFile {
-  private INodeFileWithLink next;
+public class INodeFileWithLink extends INodeFile implements FileWithLink {
+  private FileWithLink next;
 
 
   public INodeFileWithLink(INodeFile f) {
   public INodeFileWithLink(INodeFile f) {
     super(f);
     super(f);
-    next = this;
+    setNext(this);
   }
   }
 
 
   @Override
   @Override
@@ -43,124 +39,35 @@ public class INodeFileWithLink extends INodeFile {
         new INodeFileSnapshot(this));
         new INodeFileSnapshot(this));
   }
   }
 
 
-  void setNext(INodeFileWithLink next) {
-    this.next = next;
+  @SuppressWarnings("unchecked")
+  @Override
+  public <N extends INodeFile & FileWithLink> N getNext() {
+    return (N)next;
   }
   }
 
 
-  INodeFileWithLink getNext() {
-    return next;
+  @Override
+  public <N extends INodeFile & FileWithLink> void setNext(N next) {
+    this.next = next;
   }
   }
-  
-  /** Insert inode to the circular linked list. */
-  void insert(INodeFileWithLink inode) {
+
+  @Override
+  public <N extends INodeFile & FileWithLink> void insert(N inode) {
     inode.setNext(this.getNext());
     inode.setNext(this.getNext());
     this.setNext(inode);
     this.setNext(inode);
   }
   }
 
 
-  /**
-   * @return the max file replication of the elements
-   *         in the circular linked list.
-   */
   @Override
   @Override
   public short getBlockReplication() {
   public short getBlockReplication() {
-    short max = getFileReplication();
-    // i may be null since next will be set to null when the INode is deleted
-    for(INodeFileWithLink i = next; i != this && i != null; i = i.getNext()) {
-      final short replication = i.getFileReplication();
-      if (replication > max) {
-        max = replication;
-      }
-    }
-    return max;
+    return Util.getBlockReplication(this);
   }
   }
 
 
-  /**
-   * {@inheritDoc}
-   * 
-   * Remove the current inode from the circular linked list.
-   * If some blocks at the end of the block list no longer belongs to
-   * any other inode, collect them and update the block list.
-   */
   @Override
   @Override
   public int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
   public int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
-    if (next == this) {
+    if (next == null || next == this) {
       // this is the only remaining inode.
       // this is the only remaining inode.
-      super.collectSubtreeBlocksAndClear(info);
+      return super.collectSubtreeBlocksAndClear(info);
     } else {
     } else {
-      // There are other inode(s) using the blocks.
-      // Compute max file size excluding this and find the last inode. 
-      long max = next.computeFileSize(true);
-      short maxReplication = next.getFileReplication();
-      INodeFileWithLink last = next;
-      for(INodeFileWithLink i = next.getNext(); i != this; i = i.getNext()) {
-        final long size = i.computeFileSize(true);
-        if (size > max) {
-          max = size;
-        }
-        final short rep = i.getFileReplication();
-        if (rep > maxReplication) {
-          maxReplication = rep;
-        }
-        last = i;
-      }
-
-      collectBlocksBeyondMaxAndClear(max, info);
-      
-      // remove this from the circular linked list.
-      last.next = this.next;
-      // 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.
-      this.setFileReplication(maxReplication, null);
-      this.next = null;
-      // clear parent
-      setParent(null);
-    }
-    return 1;
-  }
-
-  private void collectBlocksBeyondMaxAndClear(final long max,
-      final BlocksMapUpdateInfo info) {
-    final BlockInfo[] oldBlocks = getBlocks();
-    if (oldBlocks != null) {
-      //find the minimum n such that the size of the first n blocks > max
-      int n = 0;
-      for(long size = 0; n < oldBlocks.length && max > size; n++) {
-        size += oldBlocks[n].getNumBytes();
-      }
-
-      // Replace the INode for all the remaining blocks in blocksMap
-      BlocksMapINodeUpdateEntry entry = new BlocksMapINodeUpdateEntry(this,
-          next);
-      if (info != null) {
-        for (int i = 0; i < n; i++) {
-          info.addUpdateBlock(oldBlocks[i], entry);
-        }
-      }
-      
-      // starting from block n, the data is beyond max.
-      if (n < oldBlocks.length) {
-        // resize the array.  
-        final BlockInfo[] newBlocks;
-        if (n == 0) {
-          newBlocks = null;
-        } else {
-          newBlocks = new BlockInfo[n];
-          System.arraycopy(oldBlocks, 0, newBlocks, 0, n);
-        }
-        for(INodeFileWithLink i = next; i != this; i = i.getNext()) {
-          i.setBlocks(newBlocks);
-        }
-
-        // collect the blocks beyond max.  
-        if (info != null) {
-          for(; n < oldBlocks.length; n++) {
-            info.addDeleteBlock(oldBlocks[n]);
-          }
-        }
-      }
-      setBlocks(null);
+      return Util.collectSubtreeBlocksAndClear(this, info);
     }
     }
   }
   }
 }
 }

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

@@ -364,8 +364,7 @@ public class TestSnapshotPathINodes {
    * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for snapshot file while modifying file after snapshot.
    * for snapshot file while modifying file after snapshot.
    */
    */
-//  TODO: disable it temporarily since it uses append.
-//  @Test
+  @Test
   public void testSnapshotPathINodesAfterModification() throws Exception {
   public void testSnapshotPathINodesAfterModification() throws Exception {
     //file1 was deleted, create it again.
     //file1 was deleted, create it again.
     DFSTestUtil.createFile(hdfs, file1, 1024, REPLICATION, seed);
     DFSTestUtil.createFile(hdfs, file1, 1024, REPLICATION, seed);

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

@@ -270,10 +270,9 @@ public class TestSnapshot {
       Modification delete = new FileDeletion(
       Modification delete = new FileDeletion(
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           hdfs);
           hdfs);
-//      TODO: fix append for snapshots
-//      Modification append = new FileAppend(
-//          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
-//          hdfs, (int) BLOCKSIZE);
+      Modification append = new FileAppend(
+          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
+          hdfs, (int) BLOCKSIZE);
       Modification chmod = new FileChangePermission(
       Modification chmod = new FileChangePermission(
           node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
           node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
           hdfs, genRandomPermission());
           hdfs, genRandomPermission());
@@ -290,8 +289,7 @@ public class TestSnapshot {
       
       
       mList.add(create);
       mList.add(create);
       mList.add(delete);
       mList.add(delete);
-      // TODO: fix append for snapshots
-//      mList.add(append);
+      mList.add(append);
       mList.add(chmod);
       mList.add(chmod);
       mList.add(chown);
       mList.add(chown);
       mList.add(replication);
       mList.add(replication);