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 su...@apache.org on 2009/10/15 09:28:05 UTC

svn commit: r825416 - in /hadoop/hdfs/trunk: ./ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/fs/ src/test/hdfs/org/apache/hadoop/hdfs/

Author: suresh
Date: Thu Oct 15 07:28:04 2009
New Revision: 825416

URL: http://svn.apache.org/viewvc?rev=825416&view=rev
Log:
HDFS-677. Rename failure when both source and destination quota exceeds results in deletion of source. Contributed by Suresh Srinivas.

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=825416&r1=825415&r2=825416&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Thu Oct 15 07:28:04 2009
@@ -6,9 +6,9 @@
 
   NEW FEATURES
 
-  HDFS-503. This patch implements an optional layer over HDFS that 
-  implements offline erasure-coding.  It can be used to reduce the 
-  total storage requirements of HDFS.  (dhruba)
+    HDFS-503. This patch implements an optional layer over HDFS that 
+    implements offline erasure-coding.  It can be used to reduce the 
+    total storage requirements of HDFS.  (dhruba)
 
   IMPROVEMENTS
     
@@ -19,8 +19,8 @@
 
   BUG FIXES
   
-  HDFS-646. Fix test-patch failure by adding test-contrib ant target.
-  (gkesavan)
+    HDFS-646. Fix test-patch failure by adding test-contrib ant target.
+    (gkesavan)
 
 Release 0.21.0 - Unreleased
 
@@ -429,3 +429,6 @@
 
     HDFS-525. The SimpleDateFormat object in ListPathsServlet is not thread
     safe. (Suresh Srinivas and cdouglas)
+
+    HDFS-677. Rename failure when both source and destination quota exceeds
+    results in deletion of source. (suresh)

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=825416&r1=825415&r2=825416&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Thu Oct 15 07:28:04 2009
@@ -289,8 +289,8 @@
         (INodeFileUnderConstruction)inodes[inodes.length-1];
 
       // check quota limits and updated space consumed
-      updateCount(inodes, inodes.length-1, 0, 
-                  fileINode.getPreferredBlockSize()*fileINode.getReplication());
+      updateCount(inodes, inodes.length-1, 0,
+          fileINode.getPreferredBlockSize()*fileINode.getReplication(), true);
 
       // associate new last block for the file
       BlockInfoUnderConstruction blockInfo =
@@ -396,9 +396,11 @@
       // check the validation of the source
       if (srcInodes[srcInodes.length-1] == null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-                                     +"failed to rename "+src+" to "+dst+ " because source does not exist");
+            + "failed to rename " + src + " to " + dst
+            + " because source does not exist");
         return false;
-      } else if (srcInodes.length == 1) {
+      } 
+      if (srcInodes.length == 1) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
             +"failed to rename "+src+" to "+dst+ " because source is the root");
         return false;
@@ -407,71 +409,76 @@
         dst += Path.SEPARATOR + new Path(src).getName();
       }
       
-      // remove source
-      INode srcChild = null;
-      try {
-        srcChild = removeChild(srcInodes, srcInodes.length-1);
-      } catch (IOException e) {
-        // srcChild == null; go to next if statement
+      // check the validity of the destination
+      if (dst.equals(src)) {
+        return true;
       }
-      if (srcChild == null) {
+      if (dst.startsWith(src)) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            +"failed to rename "+src+" to "+dst+ " because the source can not be removed");
+            + "failed to rename " + src + " to " + dst
+            + " because destination starts with src");
         return false;
       }
-
-      String srcChildName = srcChild.getLocalName();
       
-      // check the validity of the destination
-      INode dstChild = null;
-      QuotaExceededException failureByQuota = null;
-
       byte[][] dstComponents = INode.getPathComponents(dst);
       INode[] dstInodes = new INode[dstComponents.length];
       rootDir.getExistingPathINodes(dstComponents, dstInodes);
-      if (dstInodes[dstInodes.length-1] != null) { //check if destination exists
+      if (dstInodes[dstInodes.length-1] != null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                                      +"failed to rename "+src+" to "+dst+ 
                                      " because destination exists");
-      } else if (dstInodes[dstInodes.length-2] == null) { // check if its parent exists
+        return false;
+      }
+      if (dstInodes[dstInodes.length-2] == null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
             +"failed to rename "+src+" to "+dst+ 
-            " because destination's parent does not exists");
+            " because destination's parent does not exist");
+        return false;
       }
-      else {
-        // add to the destination
-        srcChild.setLocalName(dstComponents[dstInodes.length-1]);
-        try {
-          // add it to the namespace
-          dstChild = addChild(dstInodes, dstInodes.length-1, srcChild, false);
-        } catch (QuotaExceededException qe) {
-          failureByQuota = qe;
+      
+      // Ensure dst has quota to accommodate rename
+      verifyQuotaForRename(srcInodes,dstInodes);
+      
+      INode dstChild = null;
+      INode srcChild = null;
+      String srcChildName = null;
+      try {
+        // remove src
+        srcChild = removeChild(srcInodes, srcInodes.length-1);
+        if (srcChild == null) {
+          NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+              + "failed to rename " + src + " to " + dst
+              + " because the source can not be removed");
+          return false;
         }
-      }
-      if (dstChild != null) {
-        if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
-            +src+" is renamed to "+dst);
+        srcChildName = srcChild.getLocalName();
+        srcChild.setLocalName(dstComponents[dstInodes.length-1]);
+        
+        // add src to the destination
+        dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
+            srcChild, -1, false);
+        if (dstChild != null) {
+          srcChild = null;
+          if (NameNode.stateChangeLog.isDebugEnabled()) {
+            NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: " + src
+                    + " is renamed to " + dst);
+          }
+          // update modification time of dst and the parent of src
+          srcInodes[srcInodes.length-2].setModificationTime(timestamp);
+          dstInodes[dstInodes.length-2].setModificationTime(timestamp);
+          return true;
         }
-
-        // update modification time of dst and the parent of src
-        srcInodes[srcInodes.length-2].setModificationTime(timestamp);
-        dstInodes[dstInodes.length-2].setModificationTime(timestamp);
-        return true;
-      } else {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            +"failed to rename "+src+" to "+dst);
-        try {
+      } finally {
+        if (dstChild == null && srcChild != null) {
           // put it back
           srcChild.setLocalName(srcChildName);
-          addChild(srcInodes, srcInodes.length-1, srcChild, false);
-        } catch (IOException ignored) {}
-        if (failureByQuota != null) {
-          throw failureByQuota;
-        } else {
-          return false;
+          addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, srcChild, -1,
+              false);
         }
       }
+      NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
+          +"failed to rename "+src+" to "+dst);
+      return false;
     }
   }
 
@@ -516,7 +523,7 @@
       // check disk quota
       long dsDelta = (replication - oldReplication[0]) *
            (fileNode.diskspaceConsumed()/oldReplication[0]);
-      updateCount(inodes, inodes.length-1, 0, dsDelta);
+      updateCount(inodes, inodes.length-1, 0, dsDelta, true);
 
       fileNode.setReplication(replication);
       fileBlocks = fileNode.getBlocks();
@@ -680,14 +687,8 @@
         return null;
       }
       int pos = inodes.length - 1;
-      try {
-        // Remove the node from the namespace
-        targetNode = removeChild(inodes, pos);
-      } catch(QuotaExceededException e) {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
-            "failed to remove " + src + " because " + e.getMessage());
-        return null;
-      }
+      // Remove the node from the namespace
+      targetNode = removeChild(inodes, pos);
       if (targetNode == null) {
         return null;
       }
@@ -893,7 +894,7 @@
         throw new FileNotFoundException(path + 
                                         " does not exist under rootDir.");
       }
-      updateCount(inodes, len-1, nsDelta, dsDelta);
+      updateCount(inodes, len-1, nsDelta, dsDelta, true);
     }
   }
   
@@ -903,10 +904,11 @@
    * @param numOfINodes the number of inodes to update starting from index 0
    * @param nsDelta the delta change of namespace
    * @param dsDelta the delta change of diskspace
+   * @param checkQuota if true then check if quota is exceeded
    * @throws QuotaExceededException if the new count violates any quota limit
    */
   private void updateCount(INode[] inodes, int numOfINodes, 
-                           long nsDelta, long dsDelta)
+                           long nsDelta, long dsDelta, boolean checkQuota)
                            throws QuotaExceededException {
     if (!ready) {
       //still initializing. do not check or update quotas.
@@ -915,28 +917,27 @@
     if (numOfINodes>inodes.length) {
       numOfINodes = inodes.length;
     }
-    // check existing components in the path  
-    int i=0;
-    try {
-      for(; i < numOfINodes; i++) {
-        if (inodes[i].isQuotaSet()) { // a directory with quota
-          INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
-          node.updateNumItemsInTree(nsDelta, dsDelta);
-        }
+    if (checkQuota) {
+      verifyQuota(inodes, numOfINodes, nsDelta, dsDelta, null);
+    }
+    for(int i = 0; i < numOfINodes; i++) {
+      if (inodes[i].isQuotaSet()) { // a directory with quota
+        INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
+        node.updateNumItemsInTree(nsDelta, dsDelta);
       }
+    }
+  }
+  
+  /** 
+   * update quota of each inode and check to see if quota is exceeded. 
+   * See {@link #updateCount(INode[], int, long, long, boolean)}
+   */ 
+  private void updateCountNoQuotaCheck(INode[] inodes, int numOfINodes, 
+                           long nsDelta, long dsDelta) {
+    try {
+      updateCount(inodes, numOfINodes, nsDelta, dsDelta, false);
     } catch (QuotaExceededException e) {
-      e.setPathName(getFullPathName(inodes, i));
-      // undo updates
-      for( ; i-- > 0; ) {
-        try {
-          if (inodes[i].isQuotaSet()) { // a directory with quota
-            INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
-            node.updateNumItemsInTree(-nsDelta, -dsDelta);
-          }
-        } catch (IOException ingored) {
-        }
-      }
-      throw e;
+      NameNode.LOG.warn("FSDirectory.updateCountNoQuotaCheck - unexpected ", e);
     }
   }
   
@@ -1047,7 +1048,7 @@
       long timestamp) throws QuotaExceededException {
     inodes[pos] = addChild(inodes, pos, 
         new INodeDirectory(name, permission, timestamp),
-        inheritPermission );
+        -1, inheritPermission );
   }
   
   /** Add a node child to the namespace. The full path name of the node is src.
@@ -1065,48 +1066,119 @@
                       inheritPermission);
     }
   }
+
+  /**
+   * Verify quota for adding or moving a new INode with required 
+   * namespace and diskspace to a given position.
+   *  
+   * @param inodes INodes corresponding to a path
+   * @param pos position where a new INode will be added
+   * @param nsDelta needed namespace
+   * @param dsDelta needed diskspace
+   * @param commonAncestor Last node in inodes array that is a common ancestor
+   *          for a INode that is being moved from one location to the other.
+   *          Pass null if a node is not being moved.
+   * @throws QuotaExceededException if quota limit is exceeded.
+   */
+  private void verifyQuota(INode[] inodes, int pos, long nsDelta, long dsDelta,
+      INode commonAncestor) throws QuotaExceededException {
+    if (pos>inodes.length) {
+      pos = inodes.length;
+    }
+    int i = pos - 1;
+    try {
+      // check existing components in the path  
+      for(; i >= 0; i--) {
+        if (commonAncestor == inodes[i]) {
+          // Moving an existing node. Stop checking for quota when common
+          // ancestor is reached
+          return;
+        }
+        if (inodes[i].isQuotaSet()) { // a directory with quota
+          INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i]; 
+          node.verifyQuota(nsDelta, dsDelta);
+        }
+      }
+    } catch (QuotaExceededException e) {
+      e.setPathName(getFullPathName(inodes, i));
+      throw e;
+    }
+  }
   
-  /** Add a node child to the inodes at index pos. 
-   * Its ancestors are stored at [0, pos-1]. 
-   * QuotaExceededException is thrown if it violates quota limit */
-  private <T extends INode> T addChild(INode[] pathComponents, int pos, T child,
-      boolean inheritPermission) throws QuotaExceededException {
-    return addChild(pathComponents, pos, child, -1, inheritPermission);
+  /**
+   * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
+   * dstInodes[dstInodes.length-1]
+   * 
+   * @param srcInodes directory from where node is being moved.
+   * @param dstInodes directory to where node is moved to.
+   * @throws QuotaExceededException if quota limit is exceeded.
+   */
+  private void verifyQuotaForRename(INode[] srcInodes, INode[]dstInodes)
+      throws QuotaExceededException {
+    INode srcInode = srcInodes[srcInodes.length - 1];
+    INode commonAncestor = null;
+    for(int i =0;srcInodes[i] == dstInodes[i]; i++) {
+      commonAncestor = srcInodes[i];
+    }
+    INode.DirCounts counts = new INode.DirCounts();
+    srcInode.spaceConsumedInTree(counts);
+    verifyQuota(dstInodes, dstInodes.length - 1, counts.getNsCount(),
+            counts.getDsCount(), commonAncestor);
   }
   
   /** Add a node child to the inodes at index pos. 
    * Its ancestors are stored at [0, pos-1]. 
    * QuotaExceededException is thrown if it violates quota limit */
-  private <T extends INode> T addChild(INode[] pathComponents, int pos, T child,
-       long childDiskspace, boolean inheritPermission) throws QuotaExceededException {
+  private <T extends INode> T addChild(INode[] pathComponents, int pos,
+      T child, long childDiskspace, boolean inheritPermission,
+      boolean checkQuota) throws QuotaExceededException {
     INode.DirCounts counts = new INode.DirCounts();
     child.spaceConsumedInTree(counts);
     if (childDiskspace < 0) {
       childDiskspace = counts.getDsCount();
     }
-    updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace);
+    updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace,
+        checkQuota);
     T addedNode = ((INodeDirectory)pathComponents[pos-1]).addChild(
         child, inheritPermission);
     if (addedNode == null) {
-      updateCount(pathComponents, pos, 
-                  -counts.getNsCount(), -childDiskspace);
+      updateCount(pathComponents, pos, -counts.getNsCount(), 
+          -childDiskspace, true);
     }
     return addedNode;
   }
+
+  private <T extends INode> T addChild(INode[] pathComponents, int pos,
+      T child, long childDiskspace, boolean inheritPermission)
+      throws QuotaExceededException {
+    return addChild(pathComponents, pos, child, childDiskspace,
+        inheritPermission, true);
+  }
+  
+  private <T extends INode> T addChildNoQuotaCheck(INode[] pathComponents,
+      int pos, T child, long childDiskspace, boolean inheritPermission) {
+    T inode = null;
+    try {
+      inode = addChild(pathComponents, pos, child, childDiskspace,
+          inheritPermission, false);
+    } catch (QuotaExceededException e) {
+      NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e); 
+    }
+    return inode;
+  }
   
   /** Remove an inode at index pos from the namespace.
    * Its ancestors are stored at [0, pos-1].
    * Count of each ancestor with quota is also updated.
    * Return the removed node; null if the removal fails.
    */
-  private INode removeChild(INode[] pathComponents, int pos)
-  throws QuotaExceededException {
+  private INode removeChild(INode[] pathComponents, int pos) {
     INode removedNode = 
       ((INodeDirectory)pathComponents[pos-1]).removeChild(pathComponents[pos]);
     if (removedNode != null) {
       INode.DirCounts counts = new INode.DirCounts();
       removedNode.spaceConsumedInTree(counts);
-      updateCount(pathComponents, pos,
+      updateCountNoQuotaCheck(pathComponents, pos,
                   -counts.getNsCount(), -counts.getDsCount());
     }
     return removedNode;

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java?rev=825416&r1=825415&r2=825416&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java Thu Oct 15 07:28:04 2009
@@ -116,18 +116,10 @@
    * 
    * @param nsDelta the change of the tree size
    * @param dsDelta change to disk space occupied
-   * @throws QuotaExceededException if the changed size is greater 
-   *                                than the quota
    */
-  void updateNumItemsInTree(long nsDelta, long dsDelta) throws 
-                            QuotaExceededException {
-    long newCount = nsCount + nsDelta;
-    long newDiskspace = diskspace + dsDelta;
-    if (nsDelta>0 || dsDelta>0) {
-      verifyQuota(nsQuota, newCount, dsQuota, newDiskspace);
-    }
-    nsCount = newCount;
-    diskspace = newDiskspace;
+  void updateNumItemsInTree(long nsDelta, long dsDelta) {
+    nsCount += nsDelta;
+    diskspace += dsDelta;
   }
   
   /** 
@@ -146,14 +138,16 @@
   /** Verify if the namespace count disk space satisfies the quota restriction 
    * @throws QuotaExceededException if the given quota is less than the count
    */
-  private static void verifyQuota(long nsQuota, long nsCount, 
-                                  long dsQuota, long diskspace)
-                                  throws QuotaExceededException {
-    if (nsQuota >= 0 && nsQuota < nsCount) {
-      throw new NSQuotaExceededException(nsQuota, nsCount);
-    }
-    if (dsQuota >= 0 && dsQuota < diskspace) {
-      throw new DSQuotaExceededException(dsQuota, diskspace);
+  void verifyQuota(long nsDelta, long dsDelta) throws QuotaExceededException {
+    long newCount = nsCount + nsDelta;
+    long newDiskspace = diskspace + dsDelta;
+    if (nsDelta>0 || dsDelta>0) {
+      if (nsQuota >= 0 && nsQuota < newCount) {
+        throw new NSQuotaExceededException(nsQuota, newCount);
+      }
+      if (dsQuota >= 0 && dsQuota < newDiskspace) {
+        throw new DSQuotaExceededException(dsQuota, newDiskspace);
+      }
     }
   }
 }

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java?rev=825416&r1=825415&r2=825416&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java Thu Oct 15 07:28:04 2009
@@ -22,19 +22,21 @@
 
 import javax.security.auth.login.LoginException;
 
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestHDFSFileContextMainOperations extends
                                   FileContextMainOperationsBaseTest {
-  
   private static MiniDFSCluster cluster;
   private static Path defaultWorkingDirectory;
   
@@ -74,4 +76,44 @@
   public void testRenameFileAsExistingFile() throws Exception {
     // ignore base class test till hadoop-6240 is fixed
   }
+  
+  @Test
+  public void testRenameWithQuota() throws Exception {
+    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    Path src1 = getTestRootPath("testRenameWithQuota/srcdir/src1");
+    Path src2 = getTestRootPath("testRenameWithQuota/srcdir/src2");
+    Path dst1 = getTestRootPath("testRenameWithQuota/dstdir/dst1");
+    Path dst2 = getTestRootPath("testRenameWithQuota/dstdir/dst2");
+    createFile(src1);
+    createFile(src2);
+    fs.setQuota(src1.getParent(), FSConstants.QUOTA_DONT_SET,
+        FSConstants.QUOTA_DONT_SET);
+    fc.mkdir(dst1.getParent(), FileContext.DEFAULT_PERM, true);
+    fs.setQuota(dst1.getParent(), FSConstants.QUOTA_DONT_SET,
+        FSConstants.QUOTA_DONT_SET);
+
+    // Test1: src does not exceed quota and dst has quota to accommodate rename
+    rename(src1, dst1, true, false);
+
+    // Test2: src does not exceed quota and dst has *no* quota to accommodate
+    // rename
+    fs.setQuota(dst1.getParent(), 1, FSConstants.QUOTA_DONT_SET);
+    rename(src2, dst2, false, true);
+
+    // Test3: src exceeds quota and dst has *no* quota to accommodate rename
+    fs.setQuota(src1.getParent(), 1, FSConstants.QUOTA_DONT_SET);
+    rename(dst1, src1, false, true);
+  }
+  
+  private void rename(Path src, Path dst, boolean renameSucceeds,
+      boolean quotaException) throws Exception {
+    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    try {
+      Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
+    } catch (QuotaExceededException ex) {
+      Assert.assertTrue(quotaException);
+    }
+    Assert.assertEquals(renameSucceeds, !fc.exists(src));
+    Assert.assertEquals(renameSucceeds, fc.exists(dst));
+  }
 }

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java?rev=825416&r1=825415&r2=825416&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java Thu Oct 15 07:28:04 2009
@@ -90,6 +90,14 @@
         assertFalse(fs.rename(src, dst));
       }
       
+      { // test rename /a/b/c to /a/b/c
+        Path src = new Path("/a/b/c");
+        DataOutputStream a_out = fs.create(src);
+        a_out.writeBytes("something");
+        a_out.close();
+        assertTrue(fs.rename(src, src));
+      }
+      
       fs.delete(dir, true);
     } finally {
       if (cluster != null) {cluster.shutdown();}