You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2013/01/15 05:33:14 UTC

svn commit: r1433284 - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/ src/test/java/org/apache/hadoop/hdfs/se...

Author: szetszwo
Date: Tue Jan 15 04:33:14 2013
New Revision: 1433284

URL: http://svn.apache.org/viewvc?rev=1433284&view=rev
Log:
svn merge -c -1432788 for reverting HDFS-4098. Add FileWithLink, INodeFileUnderConstructionWithLink and INodeFileUnderConstructionSnapshot in order to support append to snapshotted files.

Removed:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithLink.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionSnapshot.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileUnderConstructionWithLink.java
Modified:
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithLink.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
    hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-2802.txt Tue Jan 15 04:33:14 2013
@@ -101,9 +101,5 @@ Branch-2802 Snapshot (Unreleased)
   HDFS-4245. Include snapshot related operations in TestOfflineEditsViewer.
   (Jing Zhao via szetszwo)
 
-  HDFS-4098. Add FileWithLink, INodeFileUnderConstructionWithLink and
-  INodeFileUnderConstructionSnapshot in order to support append to snapshotted
-  files.  (szetszwo)
-
   HDFS-4395. In INodeDirectorySnapshottable's constructor, the passed-in dir
   could be an INodeDirectoryWithSnapshot.  (Jing Zhao via szetszwo)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Tue Jan 15 04:33:14 2013
@@ -324,7 +324,7 @@ public class FSEditLogLoader {
       if (oldFile.isUnderConstruction()) {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
         fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
-        INodeFile newFile = ucFile.toINodeFile(ucFile.getModificationTime());
+        INodeFile newFile = ucFile.convertToInodeFile(ucFile.getModificationTime());
         fsDir.unprotectedReplaceINodeFile(addCloseOp.path, ucFile, newFile,
             iip.getLatestSnapshot());
       }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue Jan 15 04:33:14 2013
@@ -1357,11 +1357,10 @@ public class FSNamesystem implements Nam
           doAccessTime = false;
         }
 
-        final INodesInPath iip = dir.getINodesInPath(src);
+        long now = now();
+        final INodesInPath iip = dir.getMutableINodesInPath(src);
         final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
-        if (!iip.isSnapshot() //snapshots are readonly, so don't update atime.
-            && doAccessTime && isAccessTimeSupported()) {
-          final long now = now();
+        if (doAccessTime && isAccessTimeSupported()) {
           if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
             // if we have to set access time but we only have the readlock, then
             // restart this entire operation with the writeLock.
@@ -1982,13 +1981,18 @@ public class FSNamesystem implements Nam
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
       boolean writeToEditLog, Snapshot latestSnapshot) throws IOException {
-    if (latestSnapshot != null) {
-      file = (INodeFile)file.recordModification(latestSnapshot).left;
-    }
-    final INodeFileUnderConstruction cons
-        = INodeFileUnderConstruction.toINodeFileUnderConstruction(
-            file, leaseHolder, clientMachine, clientNode);
-
+    //TODO SNAPSHOT: INodeFileUnderConstruction with link
+    INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                                    file.getId(),
+                                    file.getLocalNameBytes(),
+                                    file.getFileReplication(),
+                                    file.getModificationTime(),
+                                    file.getPreferredBlockSize(),
+                                    file.getBlocks(),
+                                    file.getPermissionStatus(),
+                                    leaseHolder,
+                                    clientMachine,
+                                    clientNode);
     dir.replaceINodeFile(src, file, cons, latestSnapshot);
     leaseManager.addLease(cons.getClientName(), src);
     
@@ -3297,7 +3301,7 @@ public class FSNamesystem implements Nam
 
     // The file is no longer pending.
     // Create permanent INode, update blocks
-    INodeFile newFile = pendingFile.toINodeFile(now());
+    INodeFile newFile = pendingFile.convertToInodeFile(now());
     dir.replaceINodeFile(src, pendingFile, newFile, latestSnapshot);
 
     // close file and persist block allocations for this file

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java Tue Jan 15 04:33:14 2013
@@ -509,7 +509,7 @@ public abstract class INode implements C
   /**
    * Set last access time of inode.
    */
-  public INode setAccessTime(long atime, Snapshot latest) {
+  INode setAccessTime(long atime, Snapshot latest) {
     Pair<? extends INode, ? extends INode> pair = recordModification(latest);
     INode nodeToUpdate = pair != null ? pair.left : this;    
     nodeToUpdate.accessTime = atime;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java Tue Jan 15 04:33:14 2013
@@ -28,6 +28,8 @@ import org.apache.hadoop.hdfs.protocol.B
 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.namenode.snapshot.INodeFileSnapshot;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithLink;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 /** I-node for closed file. */
@@ -94,7 +96,7 @@ public class INodeFile extends INode imp
         preferredBlockSize);
   }
 
-  protected INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime,
+  INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime,
       BlockInfo[] blklist, short replication, long preferredBlockSize) {
     super(id, name, permissions, null, mtime, atime);
     header = HeaderFormat.combineReplication(header, replication);
@@ -109,7 +111,7 @@ public class INodeFile extends INode imp
   }
 
   @Override
-  public Pair<? extends INodeFile, ? extends INodeFile> createSnapshotCopy() {
+  public Pair<INodeFileWithLink, INodeFileSnapshot> createSnapshotCopy() {
     return parent.replaceINodeFile(this).createSnapshotCopy();
   }
 
@@ -139,7 +141,7 @@ public class INodeFile extends INode imp
     return getFileReplication();
   }
 
-  public void setFileReplication(short replication, Snapshot latest) {
+  protected void setFileReplication(short replication, Snapshot latest) {
     if (latest != null) {
       final Pair<? extends INode, ? extends INode> p = recordModification(latest);
       if (p != null) {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java Tue Jan 15 04:33:14 2013
@@ -29,10 +29,6 @@ import org.apache.hadoop.hdfs.server.blo
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 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.
@@ -49,29 +45,6 @@ public class INodeFileUnderConstruction 
     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.getId(),
-        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 final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -104,13 +77,6 @@ public class INodeFileUnderConstruction 
     this.clientMachine = clientMachine;
     this.clientNode = clientNode;
   }
- 
-  protected INodeFileUnderConstruction(INodeFileUnderConstruction that) {
-    super(that);
-    this.clientName = that.clientName;
-    this.clientMachine = that.clientMachine;
-    this.clientNode = that.clientNode;
-  }
 
   String getClientName() {
     return clientName;
@@ -136,26 +102,30 @@ public class INodeFileUnderConstruction 
     return true;
   }
 
-  /**
-   * 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();
+  //
+  // 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
     return new INodeFile(getId(), getLocalNameBytes(), getPermissionStatus(),
         mtime, getModificationTime(),
         getBlocks(), getFileReplication(), getPreferredBlockSize());
   }
   
-  /** 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 if all of the blocks in this file are marked as completed.
+   */
+  private boolean allBlocksComplete() {
+    for (BlockInfo b : getBlocks()) {
+      if (!b.isComplete()) {
+        return false;
+      }
     }
+    return true;
   }
 
   /**

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithLink.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithLink.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeFileWithLink.java Tue Jan 15 04:33:14 2013
@@ -18,19 +18,23 @@
 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;
 
 /**
- * Represent an {@link INodeFile} that is snapshotted.
- * Note that snapshot files are represented by {@link INodeFileSnapshot}.
+ * 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.
  */
 @InterfaceAudience.Private
-public class INodeFileWithLink extends INodeFile implements FileWithLink {
-  private FileWithLink next;
+public class INodeFileWithLink extends INodeFile {
+  private INodeFileWithLink next;
 
   public INodeFileWithLink(INodeFile f) {
     super(f);
-    setNext(this);
+    next = this;
   }
 
   @Override
@@ -39,35 +43,124 @@ public class INodeFileWithLink extends I
         new INodeFileSnapshot(this));
   }
 
-  @SuppressWarnings("unchecked")
-  @Override
-  public <N extends INodeFile & FileWithLink> N getNext() {
-    return (N)next;
-  }
-
-  @Override
-  public <N extends INodeFile & FileWithLink> void setNext(N next) {
+  void setNext(INodeFileWithLink next) {
     this.next = next;
   }
 
-  @Override
-  public <N extends INodeFile & FileWithLink> void insert(N inode) {
+  INodeFileWithLink getNext() {
+    return next;
+  }
+  
+  /** Insert inode to the circular linked list. */
+  void insert(INodeFileWithLink inode) {
     inode.setNext(this.getNext());
     this.setNext(inode);
   }
 
+  /**
+   * @return the max file replication of the elements
+   *         in the circular linked list.
+   */
   @Override
   public short getBlockReplication() {
-    return Util.getBlockReplication(this);
+    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;
   }
 
+  /**
+   * {@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
   public int collectSubtreeBlocksAndClear(BlocksMapUpdateInfo info) {
-    if (next == null || next == this) {
+    if (next == this) {
       // this is the only remaining inode.
-      return super.collectSubtreeBlocksAndClear(info);
+      super.collectSubtreeBlocksAndClear(info);
     } else {
-      return Util.collectSubtreeBlocksAndClear(this, info);
+      // 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);
     }
   }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java Tue Jan 15 04:33:14 2013
@@ -364,7 +364,8 @@ public class TestSnapshotPathINodes {
    * Test {@link INodeDirectory#getExistingPathINodes(byte[][], int, boolean)} 
    * for snapshot file while modifying file after snapshot.
    */
-  @Test
+//  TODO: disable it temporarily since it uses append.
+//  @Test
   public void testSnapshotPathINodesAfterModification() throws Exception {
     //file1 was deleted, create it again.
     DFSTestUtil.createFile(hdfs, file1, 1024, REPLICATION, seed);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java?rev=1433284&r1=1433283&r2=1433284&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java Tue Jan 15 04:33:14 2013
@@ -270,9 +270,10 @@ public class TestSnapshot {
       Modification delete = new FileDeletion(
           node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
           hdfs);
-      Modification append = new FileAppend(
-          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
-          hdfs, (int) BLOCKSIZE);
+//      TODO: fix append for snapshots
+//      Modification append = new FileAppend(
+//          node.fileList.get((node.nullFileIndex + 2) % node.fileList.size()),
+//          hdfs, (int) BLOCKSIZE);
       Modification chmod = new FileChangePermission(
           node.fileList.get((node.nullFileIndex + 3) % node.fileList.size()),
           hdfs, genRandomPermission());
@@ -289,7 +290,8 @@ public class TestSnapshot {
       
       mList.add(create);
       mList.add(delete);
-      mList.add(append);
+      // TODO: fix append for snapshots
+//      mList.add(append);
       mList.add(chmod);
       mList.add(chown);
       mList.add(replication);