Pārlūkot izejas kodu

HDFS-76. Better error message to users when commands fail because of
lack of quota. Allow quota to be set even if the limit is lower than
current consumption. (Boris Shkolnik via rangadi)
Also added CHANGES.txt

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@787357 13f79535-47bb-0310-9956-ffa450edef68

Raghu Angadi 16 gadi atpakaļ
vecāks
revīzija
1ada062e4b

+ 10 - 0
CHANGES.txt

@@ -0,0 +1,10 @@
+Hadoop HDFS Change Log
+
+Trunk (unreleased changes)
+
+    HDFS-76. Better error message to users when commands fail because of 
+    lack of quota. Allow quota to be set even if the limit is lower than
+    current consumption. (Boris Shkolnik via rangadi)
+
+    HADOOP-4687. HDFS is split from Hadoop Core. It is a subproject under 
+    Hadoop (Owen O'Malley)

+ 2 - 3
src/docs/src/documentation/content/xdocs/hdfs_quota_admin_guide.xml

@@ -32,7 +32,7 @@ implementation of the two types of quotas are closely parallel. </p>
 
 
  <p> The name quota is a hard limit on the number of file and directory names in the tree rooted at that directory. File and
  <p> The name quota is a hard limit on the number of file and directory names in the tree rooted at that directory. File and
 directory creations fail if the quota would be exceeded. Quotas stick with renamed directories; the rename operation fails if
 directory creations fail if the quota would be exceeded. Quotas stick with renamed directories; the rename operation fails if
-operation would result in a quota violation. The attempt to set a quota fails if the directory would be in violation of the new
+operation would result in a quota violation. The attempt to set a quota will still succeed even if the directory would be in violation of the new
 quota. A newly created directory has no associated quota. The largest quota is <code>Long.Max_Value</code>. A quota of one
 quota. A newly created directory has no associated quota. The largest quota is <code>Long.Max_Value</code>. A quota of one
 forces a directory to remain empty. (Yes, a directory counts against its own quota!) </p>
 forces a directory to remain empty. (Yes, a directory counts against its own quota!) </p>
 
 
@@ -44,8 +44,7 @@ a warning is printed for each of such violations. Setting or removing a quota cr
 
 
  <p> The space quota is a hard limit on the number of bytes used by files in the tree rooted at that directory. Block
  <p> The space quota is a hard limit on the number of bytes used by files in the tree rooted at that directory. Block
 allocations fail if the quota would not allow a full block to be written. Each replica of a block counts against the quota. Quotas
 allocations fail if the quota would not allow a full block to be written. Each replica of a block counts against the quota. Quotas
-stick with renamed directories; the rename operation fails if the operation would result in a quota violation. The attempt to
-set a quota fails if the directory would be in violation of the new quota. A newly created directory has no associated quota.
+stick with renamed directories; the rename operation fails if the operation would result in a quota violation. A newly created directory has no associated quota.
 The largest quota is <code>Long.Max_Value</code>. A quota of zero still permits files to be created, but no blocks can be added to the files.
 The largest quota is <code>Long.Max_Value</code>. A quota of zero still permits files to be created, but no blocks can be added to the files.
 Directories don't use host file system space and don't count against the space quota. The host file system space used to save
 Directories don't use host file system space and don't count against the space quota. The host file system space used to save
 the file meta data is not counted against the quota. Quotas are charged at the intended replication factor for the file;
 the file meta data is not counted against the quota. Quotas are charged at the intended replication factor for the file;

+ 14 - 7
src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -461,7 +461,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
       throw re.unwrapRemoteException(FileNotFoundException.class,
                                      AccessControlException.class,
                                      AccessControlException.class,
-                                     QuotaExceededException.class);
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class);
     }
     }
     OutputStream result = new DFSOutputStream(src, buffersize, progress,
     OutputStream result = new DFSOutputStream(src, buffersize, progress,
         lastBlock, stat, conf.getInt("io.bytes.per.checksum", 512));
         lastBlock, stat, conf.getInt("io.bytes.per.checksum", 512));
@@ -484,7 +485,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       return namenode.setReplication(src, replication);
       return namenode.setReplication(src, replication);
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     QuotaExceededException.class);
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class);
     }
     }
   }
   }
 
 
@@ -498,7 +500,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       return namenode.rename(src, dst);
       return namenode.rename(src, dst);
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     QuotaExceededException.class);
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class);
     }
     }
   }
   }
 
 
@@ -861,7 +864,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
       return namenode.mkdirs(src, masked);
       return namenode.mkdirs(src, masked);
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
-                                     QuotaExceededException.class);
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class);
     }
     }
   }
   }
 
 
@@ -896,7 +900,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     } catch(RemoteException re) {
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      FileNotFoundException.class,
-                                     QuotaExceededException.class);
+                                     NSQuotaExceededException.class,
+                                     DSQuotaExceededException.class);
     }
     }
   }
   }
 
 
@@ -2784,7 +2789,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
               IOException ue = 
               IOException ue = 
                 e.unwrapRemoteException(FileNotFoundException.class,
                 e.unwrapRemoteException(FileNotFoundException.class,
                     AccessControlException.class,
                     AccessControlException.class,
-                    QuotaExceededException.class);
+                    NSQuotaExceededException.class,
+                    DSQuotaExceededException.class);
               if (ue != e) { 
               if (ue != e) { 
                 throw ue; // no need to retry these exceptions
                 throw ue; // no need to retry these exceptions
               }
               }
@@ -2933,7 +2939,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
             src, masked, clientName, new EnumSetWritable<CreateFlag>(flag), replication, blockSize);
             src, masked, clientName, new EnumSetWritable<CreateFlag>(flag), replication, blockSize);
       } catch(RemoteException re) {
       } catch(RemoteException re) {
         throw re.unwrapRemoteException(AccessControlException.class,
         throw re.unwrapRemoteException(AccessControlException.class,
-                                       QuotaExceededException.class);
+                                       NSQuotaExceededException.class,
+                                       DSQuotaExceededException.class);
       }
       }
       streamer.start();
       streamer.start();
     }
     }

+ 43 - 0
src/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java

@@ -0,0 +1,43 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.util.StringUtils;
+
+public class DSQuotaExceededException extends QuotaExceededException {
+  protected static final long serialVersionUID = 1L;
+
+  public DSQuotaExceededException(String msg) {
+    super(msg);
+  }
+
+  public DSQuotaExceededException(long quota, long count) {
+    super(quota, count);
+  }
+
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg == null) {
+      return "The DiskSpace quota" + (pathName==null?"":(" of " + pathName)) + 
+          " is exceeded: quota=" + quota + " diskspace consumed=" + StringUtils.humanReadableInt(count);
+    } else {
+      return msg;
+    }
+  }
+}

+ 42 - 0
src/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java

@@ -0,0 +1,42 @@
+/**
+ * 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.protocol;
+
+public final class NSQuotaExceededException extends QuotaExceededException {
+  protected static final long serialVersionUID = 1L;
+  
+  public NSQuotaExceededException(String msg) {
+    super(msg);
+  }
+  
+  public NSQuotaExceededException(long quota, long count) {
+    super(quota, count);
+  }
+
+  public String getMessage() {
+    String msg = super.getMessage();
+    if (msg == null) {
+      return "The NameSpace quota (directories and files)" + 
+      (pathName==null?"":(" of directory " + pathName)) + 
+          " is exceeded: quota=" + quota + " file count=" + count; 
+    } else {
+      return msg;
+    }
+  }
+}

+ 14 - 24
src/java/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java

@@ -27,26 +27,24 @@ import java.io.IOException;
  * taken by all the file under the directory tree). <br> <br>
  * taken by all the file under the directory tree). <br> <br>
  * 
  * 
  * The message for the exception specifies the directory where the quota
  * The message for the exception specifies the directory where the quota
- * was violated and actual quotas.
+ * was violated and actual quotas. Specific message is generated in the 
+ * corresponding Exception class: 
+ *  DSQuotaExceededException or
+ *  NSQuotaExceededException
  */
  */
-public final class QuotaExceededException extends IOException {
-  private static final long serialVersionUID = 1L;
-  private String pathName;
-  private long nsQuota;
-  private long nsCount;
-  private long dsQuota;
-  private long diskspace;
+public class QuotaExceededException extends IOException {
+  protected static final long serialVersionUID = 1L;
+  protected String pathName=null;
+  protected long quota; // quota
+  protected long count; // actual value
   
   
-  public QuotaExceededException(String msg) {
+  protected QuotaExceededException(String msg) {
     super(msg);
     super(msg);
   }
   }
   
   
-  public QuotaExceededException(long nsQuota, long nsCount,
-                                long dsQuota, long diskspace) {
-    this.nsQuota = nsQuota;
-    this.nsCount = nsCount;
-    this.dsQuota = dsQuota;
-    this.diskspace = diskspace;
+  protected QuotaExceededException(long quota, long count) {
+    this.quota = quota;
+    this.count = count;
   }
   }
   
   
   public void setPathName(String path) {
   public void setPathName(String path) {
@@ -54,14 +52,6 @@ public final class QuotaExceededException extends IOException {
   }
   }
   
   
   public String getMessage() {
   public String getMessage() {
-    String msg = super.getMessage();
-    if (msg == null) {
-      return "The quota" + (pathName==null?"":(" of " + pathName)) + 
-          " is exceeded: namespace quota=" + nsQuota + " file count=" + 
-          nsCount + ", diskspace quota=" + dsQuota + 
-          " diskspace=" + diskspace; 
-    } else {
-      return msg;
-    }
+    return super.getMessage();
   }
   }
 }
 }

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

@@ -1129,7 +1129,7 @@ class FSDirectory implements Closeable {
           path.append(n.getLocalName());
           path.append(n.getLocalName());
         }
         }
         
         
-        NameNode.LOG.warn("Unexpected quota violation in image for " + path + 
+        NameNode.LOG.warn("Quota violation in image for " + path + 
                           " (Namespace quota : " + dir.getNsQuota() +
                           " (Namespace quota : " + dir.getNsQuota() +
                           " consumed : " + counts.nsCount + ")" +
                           " consumed : " + counts.nsCount + ")" +
                           " (Diskspace quota : " + dir.getDsQuota() +
                           " (Diskspace quota : " + dir.getDsQuota() +

+ 7 - 12
src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryWithQuota.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
 
 /**
 /**
@@ -84,18 +86,9 @@ class INodeDirectoryWithQuota extends INodeDirectory {
    * 
    * 
    * @param nsQuota Namespace quota to be set
    * @param nsQuota Namespace quota to be set
    * @param dsQuota diskspace quota to be set
    * @param dsQuota diskspace quota to be set
-   * @throws QuotaExceededException if quota is modified and the modified quota
-   *         is too low.
    *                                
    *                                
    */
    */
   void setQuota(long newNsQuota, long newDsQuota) throws QuotaExceededException {
   void setQuota(long newNsQuota, long newDsQuota) throws QuotaExceededException {
-    // if a quota is not chaged, ignore that in verification.
-    if ((newNsQuota >=0 && newNsQuota != nsQuota && newNsQuota < nsCount)  ||
-        (newDsQuota >=0 && newDsQuota != dsQuota && newDsQuota < diskspace)) {
-      throw new QuotaExceededException(newNsQuota, nsCount, 
-                                       newDsQuota, diskspace);
-    }
-
     nsQuota = newNsQuota;
     nsQuota = newNsQuota;
     dsQuota = newDsQuota;
     dsQuota = newDsQuota;
   }
   }
@@ -156,9 +149,11 @@ class INodeDirectoryWithQuota extends INodeDirectory {
   private static void verifyQuota(long nsQuota, long nsCount, 
   private static void verifyQuota(long nsQuota, long nsCount, 
                                   long dsQuota, long diskspace)
                                   long dsQuota, long diskspace)
                                   throws QuotaExceededException {
                                   throws QuotaExceededException {
-    if ((nsQuota >= 0 && nsQuota < nsCount) || 
-        (dsQuota >= 0 && dsQuota < diskspace)) {
-      throw new QuotaExceededException(nsQuota, nsCount, dsQuota, diskspace);
+    if (nsQuota >= 0 && nsQuota < nsCount) {
+      throw new NSQuotaExceededException(nsQuota, nsCount);
+    }
+    if (dsQuota >= 0 && dsQuota < diskspace) {
+      throw new DSQuotaExceededException(dsQuota, diskspace);
     }
     }
   }
   }
 }
 }

+ 4 - 6
src/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -122,9 +122,8 @@ public class DFSAdmin extends FsShell {
       "\t\tBest effort for the directory, with faults reported if\n" +
       "\t\tBest effort for the directory, with faults reported if\n" +
       "\t\t1. N is not a positive integer, or\n" +
       "\t\t1. N is not a positive integer, or\n" +
       "\t\t2. user is not an administrator, or\n" +
       "\t\t2. user is not an administrator, or\n" +
-      "\t\t3. the directory does not exist or is a file, or\n" +
-      "\t\t4. the directory would immediately exceed the new quota.";
-    
+      "\t\t3. the directory does not exist or is a file, or\n";
+
     private final long quota; // the quota to be set
     private final long quota; // the quota to be set
     
     
     /** Constructor */
     /** Constructor */
@@ -209,9 +208,8 @@ public class DFSAdmin extends FsShell {
       "\t\tBest effort for the directory, with faults reported if\n" +
       "\t\tBest effort for the directory, with faults reported if\n" +
       "\t\t1. N is not a positive integer, or\n" +
       "\t\t1. N is not a positive integer, or\n" +
       "\t\t2. user is not an administrator, or\n" +
       "\t\t2. user is not an administrator, or\n" +
-      "\t\t3. the directory does not exist or is a file, or\n" +
-      "\t\t4. the directory would immediately exceed the new space quota.";
-    
+      "\t\t3. the directory does not exist or is a file, or\n";
+
     private long quota; // the quota to be set
     private long quota; // the quota to be set
     
     
     /** Constructor */
     /** Constructor */

+ 3 - 7
src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml

@@ -16020,10 +16020,6 @@
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
           <expected-output>^( |\t)*3. the directory does not exist or is a file, or( )*</expected-output>
           <expected-output>^( |\t)*3. the directory does not exist or is a file, or( )*</expected-output>
         </comparator>
         </comparator>
-        <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>^( |\t)*4. the directory would immediately exceed the new quota.( )*</expected-output>
-        </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
 
 
@@ -16206,7 +16202,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>The quota of /test is exceeded</expected-output>
+          <expected-output>The NameSpace quota (directories and files) of directory /test is exceeded</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -16274,7 +16270,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
-          <expected-output>put:( )*org.apache.hadoop.hdfs.protocol.QuotaExceededException:( )*The( )*quota( )*of( )*/dir1( )*is( )*exceeded:( )*namespace( )*quota=-1( )*file( )*count=[0-9],( )*diskspace( )*quota=1024( )*diskspace=[0-9]+</expected-output>
+          <expected-output>put: org.apache.hadoop.hdfs.protocol.DSQuotaExceededException: The DiskSpace quota of /dir1 is exceeded: quota=1024 diskspace consumed=[0-9.]+[kmg]*</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>
@@ -16294,7 +16290,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>SubstringComparator</type>
           <type>SubstringComparator</type>
-          <expected-output>mkdir: org.apache.hadoop.hdfs.protocol.QuotaExceededException: The quota of /dir1 is exceeded: namespace quota=1 file count=2, diskspace quota=-1 diskspace=0</expected-output>
+          <expected-output>mkdir: org.apache.hadoop.hdfs.protocol.NSQuotaExceededException: The NameSpace quota (directories and files) of directory /dir1 is exceeded: quota=1 file count=2</expected-output>
         </comparator>
         </comparator>
       </comparators>
       </comparators>
     </test>
     </test>

+ 14 - 10
src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java

@@ -23,11 +23,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
@@ -167,9 +170,10 @@ public class TestQuota extends TestCase {
       DFSTestUtil.createFile(dfs, childFile1, fileLen, replication, 0);
       DFSTestUtil.createFile(dfs, childFile1, fileLen, replication, 0);
       
       
       // 11: set the quota of /test to be 1
       // 11: set the quota of /test to be 1
+      // HADOOP-5872 - we can set quota even if it is immediately violated 
       args = new String[]{"-setQuota", "1", parent.toString()};
       args = new String[]{"-setQuota", "1", parent.toString()};
-      runCommand(admin, args, true);
-      runCommand(admin, true, "-setSpaceQuota",  // for space quota
+      runCommand(admin, args, false);
+      runCommand(admin, false, "-setSpaceQuota",  // for space quota
                  Integer.toString(fileLen), args[2]);
                  Integer.toString(fileLen), args[2]);
       
       
       // 12: set the quota of /test/data0 to be 1
       // 12: set the quota of /test/data0 to be 1
@@ -303,7 +307,7 @@ public class TestQuota extends TestCase {
       boolean hasException = false;
       boolean hasException = false;
       try {
       try {
         assertFalse(dfs.mkdirs(tempPath));
         assertFalse(dfs.mkdirs(tempPath));
-      } catch (QuotaExceededException e) {
+      } catch (NSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -326,7 +330,7 @@ public class TestQuota extends TestCase {
       hasException = false;
       hasException = false;
       try {
       try {
         assertFalse(dfs.mkdirs(tempPath));
         assertFalse(dfs.mkdirs(tempPath));
-      } catch (QuotaExceededException e) {
+      } catch (NSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -345,7 +349,7 @@ public class TestQuota extends TestCase {
       hasException = false;
       hasException = false;
       try {
       try {
         assertFalse(dfs.rename(tempPath, quotaDir3));
         assertFalse(dfs.rename(tempPath, quotaDir3));
-      } catch (QuotaExceededException e) {
+      } catch (NSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -379,7 +383,7 @@ public class TestQuota extends TestCase {
       hasException = false;
       hasException = false;
       try {
       try {
         assertFalse(dfs.rename(new Path("/nqdir0/nqdir30"), tempPath));
         assertFalse(dfs.rename(new Path("/nqdir0/nqdir30"), tempPath));
-      } catch (QuotaExceededException e) {
+      } catch (NSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -480,7 +484,7 @@ public class TestQuota extends TestCase {
       try {
       try {
         DFSTestUtil.createFile(dfs, new Path(quotaDir21, "nqdir33/file2"), 
         DFSTestUtil.createFile(dfs, new Path(quotaDir21, "nqdir33/file2"), 
                                2*fileLen, replication, 0);
                                2*fileLen, replication, 0);
-      } catch (QuotaExceededException e) {
+      } catch (DSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -523,7 +527,7 @@ public class TestQuota extends TestCase {
       hasException = false;
       hasException = false;
       try {
       try {
         assertFalse(dfs.rename(dstPath, srcPath));
         assertFalse(dfs.rename(dstPath, srcPath));
-      } catch (QuotaExceededException e) {
+      } catch (DSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);
@@ -569,7 +573,7 @@ public class TestQuota extends TestCase {
         out.write(new byte[fileLen + 1024]);
         out.write(new byte[fileLen + 1024]);
         out.flush();
         out.flush();
         out.close();
         out.close();
-      } catch (QuotaExceededException e) {
+      } catch (DSQuotaExceededException e) {
         hasException = true;
         hasException = true;
         IOUtils.closeStream(out);
         IOUtils.closeStream(out);
       }
       }
@@ -594,7 +598,7 @@ public class TestQuota extends TestCase {
       hasException = false;
       hasException = false;
       try {
       try {
         dfs.setReplication(file2, (short)(replication+1));
         dfs.setReplication(file2, (short)(replication+1));
-      } catch (QuotaExceededException e) {
+      } catch (DSQuotaExceededException e) {
         hasException = true;
         hasException = true;
       }
       }
       assertTrue(hasException);
       assertTrue(hasException);