You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dd...@apache.org on 2008/06/05 09:06:46 UTC

svn commit: r663483 - in /hadoop/core/trunk: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/shell/ src/java/org/apache/hadoop/ipc/ src/test/org/apache/hadoop/dfs/ src/test/org/apache/hadoop/fs/

Author: ddas
Date: Thu Jun  5 00:06:46 2008
New Revision: 663483

URL: http://svn.apache.org/viewvc?rev=663483&view=rev
Log:
HADOOP-3187. Some files missed in the commit due to which trunk build is broken. Reverting due to this.

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSAdmin.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSImage.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/INode.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
    hadoop/core/trunk/src/java/org/apache/hadoop/fs/ContentSummary.java
    hadoop/core/trunk/src/java/org/apache/hadoop/fs/FsShell.java
    hadoop/core/trunk/src/java/org/apache/hadoop/fs/shell/Count.java
    hadoop/core/trunk/src/java/org/apache/hadoop/ipc/RemoteException.java
    hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Thu Jun  5 00:06:46 2008
@@ -139,8 +139,6 @@
     HADOOP-1328. Implement user counters in streaming. (tomwhite via
     omalley)
 
-    HADOOP-3187. Quotas for namespace management. (hairong)
-
   IMPROVEMENTS
    
     HADOOP-2928. Remove deprecated FileSystem.getContentLength().

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java Thu Jun  5 00:06:46 2008
@@ -37,10 +37,9 @@
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
-   * 35 : Quota-related RPCs are introduced: getQuota, clearQuota;
-   * Besides, getContentSummary also returns the quota of the directory.
+   * 34 : remove abandonFileInProgress(...)
    */
-  public static final long versionID = 35L;
+  public static final long versionID = 34L;
   
   ///////////////////////////////////////
   // File contents
@@ -94,8 +93,6 @@
    * @throws AccessControlException if permission to create file is 
    * denied by the system. As usually on the client side the exception will 
    * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * @throws QuotaExceededException if the file creation violates 
-   *                                any quota restriction
    * @throws IOException if other errors occur.
    */
   public void create(String src, 
@@ -193,8 +190,6 @@
    * @return true if successful, or false if the old name does not exist
    * or if the new name already belongs to the namespace.
    * @throws IOException if the new name is invalid.
-   * @throws QuotaExceededException if the rename would violate 
-   *                                any quota restriction
    */
   public boolean rename(String src, String dst) throws IOException;
 
@@ -232,8 +227,6 @@
    * @throws {@link AccessControlException} if permission to create file is 
    * denied by the system. As usually on the client side the exception will 
    * be wraped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * @throws QuotaExceededException if the operation would violate 
-   *                                any quota restriction.
    */
   public boolean mkdirs(String src, FsPermission masked) throws IOException;
 
@@ -418,25 +411,6 @@
   public ContentSummary getContentSummary(String path) throws IOException;
 
   /**
-   * Set the quota for a directory.
-   * @param path  The string representation of the path to the directory
-   * @param quota The limit of the number of names in the tree rooted 
-   *              at the directory
-   * @throws FileNotFoundException if the path is a file or 
-   *                               does not exist 
-   * @throws QuotaExceededException if the directory size 
-   *                                is greater than the given quota
-   */
-  public void setQuota(String path, long quota) throws IOException;
-  
-  /**
-   * Remove the quota for a directory
-   * @param path The string representation of the path to the directory
-   * @throws FileNotFoundException if the path is not a directory
-   */
-  public void clearQuota(String path) throws IOException;
-  
-  /**
    * Write all metadata for this file into persistent storage.
    * The file must be currently open for writing.
    * @param src The string representation of the path

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSAdmin.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSAdmin.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSAdmin.java Thu Jun  5 00:06:46 2008
@@ -18,16 +18,11 @@
 package org.apache.hadoop.dfs;
 
 import java.io.IOException;
-import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.DistributedFileSystem.DiskStatus;
 import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.shell.Command;
-import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.ToolRunner;
@@ -38,106 +33,6 @@
 public class DFSAdmin extends FsShell {
 
   /**
-   * An abstract class for the execution of a file system command
-   */
-  abstract private static class DFSAdminCommand extends Command {
-    /** Constructor */
-    public DFSAdminCommand(FileSystem fs) {
-      super(fs);
-      if (!(fs instanceof DistributedFileSystem)) {
-        throw new IllegalArgumentException("FileSystem " + fs.getUri() + 
-            " is not a distributed file system");
-      }
-    }
-  }
-  
-  /** A class that supports command clearQuota */
-  private static class ClearQuotaCommand extends DFSAdminCommand {
-    private static final String NAME = "clrQuota";
-    private static final String USAGE = "-"+NAME+" <dirname>...<dirname>";
-    private static final String DESCRIPTION = USAGE + ": " +
-    "\tClear the quota for each directory <dirName>.\n" +
-    "\t\tBest effort for the directory. with fault reported if\n" +
-    "\t\t1. the directory does not exist or is a file, or\n" +
-    "\t\t2. user is not an administrator.\n" +
-    "\t\tIt does not fault if the directory has no quota.";
-    
-    /** Constructor */
-    ClearQuotaCommand(String[] args, int pos, FileSystem fs) {
-      super(fs);
-      CommandFormat c = new CommandFormat(NAME, 1, Integer.MAX_VALUE);
-      List<String> parameters = c.parse(args, pos);
-      this.args = parameters.toArray(new String[parameters.size()]);
-    }
-    
-    /** Check if a command is the clrQuota command
-     * 
-     * @param cmd A string representation of a command starting with "-"
-     * @return true if this is a clrQuota command; false otherwise
-     */
-    public static boolean matches(String cmd) {
-      return ("-"+NAME).equals(cmd); 
-    }
-
-    @Override
-    public String getCommandName() {
-      return NAME;
-    }
-
-    @Override
-    public void run(Path path) throws IOException {
-      ((DistributedFileSystem)fs).clearQuota(path);
-    }
-  }
-  
-  /** A class that supports command setQuota */
-  private static class SetQuotaCommand extends DFSAdminCommand {
-    private static final String NAME = "setQuota";
-    private static final String USAGE =
-      "-"+NAME+" <quota> <dirname>...<dirname>";
-    private static final String DESCRIPTION = 
-      "-setQuota <quota> <dirname>...<dirname>: " +
-      "\tSet the quota <quota> for each directory <dirName>.\n" + 
-      "\t\tThe directory quota is a long integer that puts a hard limit " +
-      "on the number of names in the directory tree\n" +
-      "\t\tBest effort for the directory, with faults reported if\n" +
-      "\t\t1. N is not a positive integer, 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.";
-    
-    private final long quota; // the quota to be set
-    
-    /** Constructor */
-    SetQuotaCommand(String[] args, int pos, FileSystem fs) {
-      super(fs);
-      CommandFormat c = new CommandFormat(NAME, 2, Integer.MAX_VALUE);
-      List<String> parameters = c.parse(args, pos);
-      this.quota = Long.parseLong(parameters.remove(0));
-      this.args = parameters.toArray(new String[parameters.size()]);
-    }
-    
-    /** Check if a command is the setQuota command
-     * 
-     * @param cmd A string representation of a command starting with "-"
-     * @return true if this is a count command; false otherwise
-     */
-    public static boolean matches(String cmd) {
-      return ("-"+NAME).equals(cmd); 
-    }
-
-    @Override
-    public String getCommandName() {
-      return NAME;
-    }
-
-    @Override
-    public void run(Path path) throws IOException {
-      ((DistributedFileSystem)fs).setQuota(path, quota);
-    }
-  }
-  
-  /**
    * Construct a DFSAdmin object.
    */
   public DFSAdmin() {
@@ -277,10 +172,7 @@
     String summary = "hadoop dfsadmin is the command to execute DFS administrative commands.\n" +
       "The full syntax is: \n\n" +
       "hadoop dfsadmin [-report] [-safemode <enter | leave | get | wait>]\n" +
-      "\t[-refreshNodes]\n" +
-      "\t[" + SetQuotaCommand.USAGE + "]\n" +
-      "\t[" + ClearQuotaCommand.USAGE +"]\n" +
-      "\t[-help [cmd]]\n";
+      "\t[-refreshNodes] [-help [cmd]]\n";
 
     String report ="-report: \tReports basic filesystem information and statistics.\n";
         
@@ -330,10 +222,6 @@
       System.out.println(upgradeProgress);
     } else if ("metasave".equals(cmd)) {
       System.out.println(metaSave);
-    } else if (SetQuotaCommand.matches(cmd)) {
-      System.out.println(SetQuotaCommand.DESCRIPTION);
-    } else if (ClearQuotaCommand.matches(cmd)) {
-      System.out.println(ClearQuotaCommand.DESCRIPTION);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
     } else {
@@ -344,8 +232,6 @@
       System.out.println(finalizeUpgrade);
       System.out.println(upgradeProgress);
       System.out.println(metaSave);
-      System.out.println(SetQuotaCommand.DESCRIPTION);
-      System.out.println(ClearQuotaCommand.DESCRIPTION);
       System.out.println(help);
       System.out.println();
       ToolRunner.printGenericCommandUsage(System.out);
@@ -433,7 +319,7 @@
    * Displays format of commands.
    * @param cmd The command that is being executed.
    */
-  private static void printUsage(String cmd) {
+  public void printUsage(String cmd) {
     if ("-report".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
                          + " [-report]");
@@ -452,12 +338,6 @@
     } else if ("-metasave".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
                          + " [-metasave filename]");
-    } else if (SetQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
-                         + " [" + SetQuotaCommand.USAGE+"]");
-    } else if (ClearQuotaCommand.matches(cmd)) {
-      System.err.println("Usage: java DFSAdmin"
-                         + " ["+ClearQuotaCommand.USAGE+"]");
     } else {
       System.err.println("Usage: java DFSAdmin");
       System.err.println("           [-report]");
@@ -466,8 +346,6 @@
       System.err.println("           [-finalizeUpgrade]");
       System.err.println("           [-upgradeProgress status | details | force]");
       System.err.println("           [-metasave filename]");
-      System.err.println("           ["+SetQuotaCommand.USAGE+"]");
-      System.err.println("           ["+ClearQuotaCommand.USAGE+"]");
       System.err.println("           [-help [cmd]]");
       System.err.println();
       ToolRunner.printGenericCommandUsage(System.err);
@@ -525,7 +403,8 @@
         return exitCode;
       }
     }
-    
+
+
     // initialize DFSAdmin
     try {
       init();
@@ -552,10 +431,6 @@
         exitCode = upgradeProgress(argv, i);
       } else if ("-metasave".equals(cmd)) {
         exitCode = metaSave(argv, i);
-      } else if (ClearQuotaCommand.matches(cmd)) {
-        exitCode = new ClearQuotaCommand(argv, i, fs).runAll();
-      } else if (SetQuotaCommand.matches(cmd)) {
-        exitCode = new SetQuotaCommand(argv, i, fs).runAll();
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);
@@ -567,10 +442,6 @@
         System.err.println(cmd.substring(1) + ": Unknown command");
         printUsage("");
       }
-    } catch (IllegalArgumentException arge) {
-      exitCode = -1;
-      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
-      printUsage(cmd);
     } catch (RemoteException e) {
       //
       // This is a error returned by hadoop server. Print
@@ -585,11 +456,16 @@
         System.err.println(cmd.substring(1) + ": "
                            + ex.getLocalizedMessage());
       }
-    } catch (Exception e) {
+    } catch (IOException e) {
+      //
+      // IO exception encountered locally.
+      //
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": "
                          + e.getLocalizedMessage());
-    } 
+    } finally {
+      fs.close();
+    }
     return exitCode;
   }
 

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Thu Jun  5 00:06:46 2008
@@ -505,8 +505,7 @@
     try {
       return namenode.rename(src, dst);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     QuotaExceededException.class);
+      throw re.unwrapRemoteException(AccessControlException.class);
     }
   }
 
@@ -703,8 +702,7 @@
     try {
       return namenode.mkdirs(src, masked);
     } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     QuotaExceededException.class);
+      throw re.unwrapRemoteException(AccessControlException.class);
     }
   }
 
@@ -718,39 +716,6 @@
   }
 
   /**
-   * Remove the quota for a directory
-   * @param path The string representation of the path to the directory
-   * @throws FileNotFoundException if the path is not a directory
-   */
-  void clearQuota(String src) throws IOException {
-    try {
-      namenode.clearQuota(src);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class);
-    }
-  }
-  
-  /**
-   * Set the quota for a directory.
-   * @param path  The string representation of the path to the directory
-   * @param quota The limit of the number of names in the tree rooted 
-   *              at the directory
-   * @throws FileNotFoundException if the path is a file or 
-   *                               does not exist 
-   * @throws QuotaExceededException if the directory size 
-   *                                is greater than the given quota
-   */
-  void setQuota(String src, long quota) throws IOException {
-    try {
-      namenode.setQuota(src, quota);
-    } catch(RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     FileNotFoundException.class,
-                                     QuotaExceededException.class);
-    }
-  }
-  /**
    * Pick the best node from which to stream the data.
    * Entries in <i>nodes</i> are already in the priority order
    */
@@ -2291,8 +2256,7 @@
         namenode.create(
             src, masked, clientName, overwrite, replication, blockSize);
       } catch(RemoteException re) {
-        throw re.unwrapRemoteException(AccessControlException.class,
-                                       QuotaExceededException.class);
+        throw re.unwrapRemoteException(AccessControlException.class);
       }
       streamer = new DataStreamer();
       streamer.setDaemon(true);

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java Thu Jun  5 00:06:46 2008
@@ -182,24 +182,6 @@
     return dfs.getContentSummary(getPathName(f));
   }
 
-  /** Clear a directory's quota
-   * @see ClientProtocol#clearQuota(String)
-   */
-  void clearQuota(Path src) throws IOException {
-    dfs.clearQuota(getPathName(src));
-  }
-  
-  /** Set a directory's quota
-   * @see ClientProtocol#setQuota(String, long) 
-   */
-  void setQuota(Path src, long quota) throws IOException {
-    if (quota <= 0) {
-      throw new IllegalArgumentException("Quota should be a positive number: "
-          + quota);
-    }
-    dfs.setQuota(getPathName(src), quota);
-  }
-  
   public FileStatus[] listStatus(Path p) throws IOException {
     DFSFileInfo[] infos = dfs.listPaths(getPathName(p));
     if (infos == null) return null;

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java Thu Jun  5 00:06:46 2008
@@ -189,7 +189,7 @@
   // Version is reflected in the data storage file.
   // Versions are negative.
   // Decrement LAYOUT_VERSION to define a new version.
-  public static final int LAYOUT_VERSION = -16;
+  public static final int LAYOUT_VERSION = -15;
   // Current version: 
-  // Change edit log and fsimage to support quotas
+  // Store generation stamp with each Block
 }

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java Thu Jun  5 00:06:46 2008
@@ -41,12 +41,14 @@
 class FSDirectory implements FSConstants {
 
   FSNamesystem namesystem = null;
-  final INodeDirectoryWithQuota rootDir;
+  final INodeDirectory rootDir;
   FSImage fsImage;  
   boolean ready = false;
   // Metrics record
   private MetricsRecord directoryMetrics = null;
 
+  volatile private long totalInodes = 1;   // number of inodes, for rootdir
+    
   /** Access an existing dfs name directory. */
   public FSDirectory(FSNamesystem ns, Configuration conf) throws IOException {
     this(new FSImage(), ns, conf);
@@ -54,9 +56,8 @@
   }
 
   public FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) throws IOException {
-    rootDir = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME,
-        ns.createFsOwnerPermissions(new FsPermission((short)0755)),
-        Integer.MAX_VALUE);
+    rootDir = new INodeDirectory(INodeDirectory.ROOT_NAME,
+        ns.createFsOwnerPermissions(new FsPermission((short)0755)));
     this.fsImage = fsImage;
     namesystem = ns;
     initialize(conf);
@@ -148,7 +149,14 @@
                                  preferredBlockSize, modTime, clientName, 
                                  clientMachine, clientNode);
     synchronized (rootDir) {
-      newNode = addNode(path, newNode, false);
+      try {
+        newNode = rootDir.addNode(path, newNode);
+      } catch (FileNotFoundException e) {
+        newNode = null;
+      }
+      if (newNode != null) {
+        totalInodes++;
+      }
     }
     if (newNode == null) {
       NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
@@ -180,7 +188,7 @@
                               modificationTime, preferredBlockSize);
     synchronized (rootDir) {
       try {
-        newNode = addNode(path, newNode, false);
+        newNode = rootDir.addNode(path, newNode);
         if(newNode != null && blocks != null) {
           int nrBlocks = blocks.length;
           // Add file->block mapping
@@ -189,9 +197,12 @@
             newF.setBlock(i, namesystem.blocksMap.addINode(blocks[i], newF));
           }
         }
-      } catch (IOException e) {
+      } catch (FileNotFoundException e) {
         return null;
       }
+      if (newNode != null) {
+        totalInodes++;
+      }
       return newNode;
     }
   }
@@ -202,18 +213,12 @@
                               Block[] blocks, 
                               short replication,
                               long modificationTime,
-                              long quota,
                               long preferredBlockSize) {
     // create new inode
     INode newNode;
-    if (blocks == null) {
-      if (quota >= 0) {
-        newNode = new INodeDirectoryWithQuota(
-            permissions, modificationTime, quota);
-      } else {
-        newNode = new INodeDirectory(permissions, modificationTime);
-      }
-    } else 
+    if (blocks == null)
+      newNode = new INodeDirectory(permissions, modificationTime);
+    else 
       newNode = new INodeFile(permissions, blocks.length, replication,
                               modificationTime, preferredBlockSize);
     // add new node to the parent
@@ -226,6 +231,7 @@
       }
       if(newParent == null)
         return null;
+      totalInodes++;
       if(blocks != null) {
         int nrBlocks = blocks.length;
         // Add file->block mapping
@@ -315,10 +321,9 @@
   }
 
   /**
-   * @see #unprotectedRenameTo(String, String, long)
+   * Change the filename
    */
-  public boolean renameTo(String src, String dst)
-  throws QuotaExceededException {
+  public boolean renameTo(String src, String dst) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
                                   +src+" to "+dst);
@@ -331,90 +336,47 @@
     return true;
   }
 
-  /** Change a path name
-   * 
-   * @param src source path
-   * @param dst destination path
-   * @return true if rename succeeds; false otherwise
-   * @throws QuotaExceededException if the operation violates any quota limit
-   */
-  boolean unprotectedRenameTo(String src, String dst, long timestamp) 
-  throws QuotaExceededException {
-    byte[][] srcComponents = INode.getPathComponents(src);
-    INode[] srcInodes = new INode[srcComponents.length];
-    synchronized (rootDir) {
-      rootDir.getExistingPathINodes(srcComponents, srcInodes);
-
-      // check the validation of the source
-      if (srcInodes[srcInodes.length-1] == null) {
+  /**
+   */
+  boolean unprotectedRenameTo(String src, String dst, long timestamp) {
+    synchronized(rootDir) {
+      INode renamedNode = rootDir.getNode(src);
+      if (renamedNode == null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                                      +"failed to rename "+src+" to "+dst+ " because source does not exist");
         return false;
-      } else if (srcInodes.length == 1) {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            +"failed to rename "+src+" to "+dst+ " because source is the root");
-        return false;
       }
       if (isDir(dst)) {
         dst += Path.SEPARATOR + new Path(src).getName();
       }
-      
-      byte[][] dstComponents = INode.getPathComponents(dst);
-      INode[] dstInodes = new INode[dstComponents.length];
-      rootDir.getExistingPathINodes(dstComponents, dstInodes);
-      
-      // check the existence of the destination
-      if (dstInodes[dstInodes.length-1] != null) {
+      if (rootDir.getNode(dst) != null) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                                      +"failed to rename "+src+" to "+dst+ " because destination exists");
         return false;
       }
-      
-      // remove source
-      INode srcChild = null;
+      INodeDirectory oldParent = renamedNode.getParent();
+      oldParent.removeChild(renamedNode);
+            
+      // the renamed node can be reused now
       try {
-        srcChild = removeChild(srcInodes, srcInodes.length-1);
-      } catch (IOException e) {
-        // srcChild == null; go to next if statement
-      }
-      if (srcChild == null) {
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            +"failed to rename "+src+" to "+dst+ " because the source can not be removed");
-        return false;
-      }
-
-      // add to the destination
-      INode dstChild = null;
-      QuotaExceededException failureByQuota = null;
-      try {
-        // set the destination's name
-        srcChild.setLocalName(dstComponents[dstInodes.length-1]);
-        // add it to the namespace
-        dstChild = addChild(dstInodes, dstInodes.length-1, srcChild, false);
-      } catch (QuotaExceededException qe) {
-        failureByQuota = qe;
-      }
-      if (dstChild != null) {
-        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;
-      } else {
+        if (rootDir.addNode(dst, renamedNode) != null) {
+          NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
+                                        +src+" is renamed to "+dst);
+
+          // update modification time of old parent as well as new parent dir
+          oldParent.setModificationTime(timestamp);
+          renamedNode.getParent().setModificationTime(timestamp);
+          return true;
+        }
+      } catch (FileNotFoundException e) {
         NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
-            +"failed to rename "+src+" to "+dst);
+                                     +"failed to rename "+src+" to "+dst);
         try {
-          // put it back
-          addChild(srcInodes, srcInodes.length-1, srcChild, false);
-        } catch (IOException ignored) {}
-        if (failureByQuota != null) {
-          throw failureByQuota;
-        } else {
-          return false;
+          rootDir.addNode(src, renamedNode); // put it back
+        }catch(FileNotFoundException e2) {                
         }
       }
+      return false;
     }
   }
 
@@ -534,7 +496,7 @@
                                   +src);
     waitForReady();
     long now = FSNamesystem.now();
-    INode deletedNode = unprotectedDelete(src, now, deletedBlocks);
+    INode deletedNode = unprotectedDelete(src, now, deletedBlocks); 
     if (deletedNode != null) {
       fsImage.getEditLog().logDelete(src, now);
     }
@@ -558,58 +520,42 @@
   }
   
   /**
-   * Delete a path from the name space
-   * Update the count at each ancestor directory with quota
-   * @param src a string representation of a path to an inode
-   * @param modificationTime the time the inode is removed
-   * @param deletedBlocks the place holder for the blocks to be removed
-   * @return if the deletion succeeds
-   */ 
+   */
   INode unprotectedDelete(String src, long modificationTime, 
                           Collection<Block> deletedBlocks) {
-    src = normalizePath(src);
-    String[] names = INode.getPathNames(src);
-    byte[][] components = INode.getPathComponents(names);
-    INode[] inodes = new INode[components.length];
-
     synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
-      INode targetNode = inodes[inodes.length-1];
-
-      if (targetNode == null) { // non-existent src
+      INode targetNode = rootDir.getNode(src);
+      if (targetNode == null) {
         NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
-            +"failed to remove "+src+" because it does not exist");
-        return null;
-      } else if (inodes.length == 1) { // src is the root
-        NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
-            "failed to remove " + src +
-            " because the root is not allowed to be deleted");
+                                     +"failed to remove "+src+" because it does not exist");
         return null;
       } else {
-        try {
-          // Remove the node from the namespace
-          removeChild(inodes, inodes.length-1);
-          // set the parent's modification time
-          inodes[inodes.length-2].setModificationTime(modificationTime);
-          // GC all the blocks underneath the node.
+        //
+        // Remove the node from the namespace and GC all
+        // the blocks underneath the node.
+        //
+        if (targetNode.getParent() == null) {
+          NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: "
+                                       +"failed to remove "+src+" because it does not have a parent");
+          return null;
+        } else {
+          targetNode.getParent().setModificationTime(modificationTime);
+          targetNode.removeNode();
+          NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
+                                        +src+" is removed");
           ArrayList<Block> v = new ArrayList<Block>();
           int filesRemoved = targetNode.collectSubtreeBlocksAndClear(v);
           incrDeletedFileCount(filesRemoved);
+          totalInodes -= filesRemoved;
           for (Block b : v) {
             namesystem.blocksMap.removeINode(b);
-            // remove the block from corruptReplicasMap
+            // If block is removed from blocksMap remove it from corruptReplicasMap
             namesystem.corruptReplicas.removeFromCorruptReplicasMap(b);
             if (deletedBlocks != null) {
               deletedBlocks.add(b);
             }
           }
-          NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
-              +src+" is removed");
           return targetNode;
-        } catch (IOException e) {
-          NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
-              "failed to remove " + src + " because " + e.getMessage());
-          return null;
         }
       }
     }
@@ -737,79 +683,20 @@
     }
   }
 
-  /** update count of each inode with quota
-   * 
-   * @param inodes an array of inodes on a path
-   * @param numOfINodes the number of inodes to update starting from index 0
-   * @param deltaCount the delta change of the count
-   * @throws QuotaExceededException if the new count violates any quota limit
-   */
-  private static void updateCount(
-      INode[] inodes, int numOfINodes, long deltaCount )
-  throws QuotaExceededException {
-    if (numOfINodes>inodes.length) {
-      numOfINodes = inodes.length;
-    }
-    // check existing components in the path  
-    List<INodeDirectoryWithQuota> inodesWithQuota = 
-      new ArrayList<INodeDirectoryWithQuota>(numOfINodes);
-    int i=0;
-    try {
-      for(; i < numOfINodes; i++) {
-        if (inodes[i].getQuota() >= 0) { // a directory with quota
-          INodeDirectoryWithQuota quotaINode =(INodeDirectoryWithQuota)inodes[i]; 
-          quotaINode.updateNumItemsInTree(deltaCount);
-          inodesWithQuota.add(quotaINode);
-        }
-      }
-    } catch (QuotaExceededException e) {
-      for (INodeDirectoryWithQuota quotaINode:inodesWithQuota) {
-        try {
-          quotaINode.updateNumItemsInTree(-deltaCount);
-        } catch (IOException ingored) {
-        }
-      }
-      e.setPathName(getFullPathName(inodes, i));
-      throw e;
-    }
-  }
-  
-  /** Return the name of the path represented by inodes at [0, pos] */
-  private static String getFullPathName(INode[] inodes, int pos) {
-    StringBuilder fullPathName = new StringBuilder();
-    for (int i=1; i<=pos; i++) {
-      fullPathName.append(Path.SEPARATOR_CHAR).append(inodes[i].getLocalName());
-    }
-    return fullPathName.toString();
-  }
-  
   /**
-   * Create a directory 
-   * If ancestor directories do not exist, automatically create them.
-
-   * @param src string representation of the path to the directory
-   * @param permissions the permission of the directory
-   * @param inheritPermission if the permission of the directory should inherit
-   *                          from its parent or not. The automatically created
-   *                          ones always inherit its permission from its parent
-   * @param now creation time
-   * @return true if the operation succeeds false otherwise
-   * @throws FileNotFoundException if an ancestor or itself is a file
-   * @throws QuotaExceededException if directory creation violates 
-   *                                any quota limit
+   * Create directory entries for every item
    */
   boolean mkdirs(String src, PermissionStatus permissions,
-      boolean inheritPermission, long now)
-      throws FileNotFoundException, QuotaExceededException {
+      boolean inheritPermission, long now) throws IOException {
     src = normalizePath(src);
     String[] names = INode.getPathNames(src);
     byte[][] components = INode.getPathComponents(names);
-    INode[] inodes = new INode[components.length];
 
     synchronized(rootDir) {
+      INode[] inodes = new INode[components.length];
       rootDir.getExistingPathINodes(components, inodes);
 
-      // find the index of the first null in inodes[]
+      // find the index of the first null in inodes[]  
       StringBuilder pathbuilder = new StringBuilder();
       int i = 1;
       for(; i < inodes.length && inodes[i] != null; i++) {
@@ -824,14 +711,17 @@
       for(; i < inodes.length; i++) {
         pathbuilder.append(Path.SEPARATOR + names[i]);
         String cur = pathbuilder.toString();
-        unprotectedMkdir(inodes, i, components[i], permissions,
-            inheritPermission || i != components.length-1, now);
-        if (inodes[i] == null) {
-          return false;
-        }
-        fsImage.getEditLog().logMkDir(cur, inodes[i]);
+  
+        inodes[i] = new INodeDirectory(permissions, now);
+        inodes[i].name = components[i];
+        INode inserted = ((INodeDirectory)inodes[i-1]).addChild(
+            inodes[i], inheritPermission || i != inodes.length-1);
+
+        assert inserted == inodes[i];
+        totalInodes++;
         NameNode.stateChangeLog.debug(
             "DIR* FSDirectory.mkdirs: created directory " + cur);
+        fsImage.getEditLog().logMkDir(cur, inserted);
       }
     }
     return true;
@@ -840,74 +730,17 @@
   /**
    */
   INode unprotectedMkdir(String src, PermissionStatus permissions,
-                          long timestamp) throws QuotaExceededException {
-    byte[][] components = INode.getPathComponents(src);
-    INode[] inodes = new INode[components.length];
+                          long timestamp) throws FileNotFoundException {
     synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
-      unprotectedMkdir(inodes, inodes.length-1, components[inodes.length-1],
-          permissions, false, timestamp);
-      return inodes[inodes.length-1];
+      INode newNode = rootDir.addNode(src,
+                                new INodeDirectory(permissions, timestamp));
+      if (newNode != null) {
+        totalInodes++;
+      }
+      return newNode;
     }
   }
 
-  /** create a directory at index pos.
-   * The parent path to the directory is at [0, pos-1].
-   * All ancestors exist. Newly created one stored at index pos.
-   */
-  private void unprotectedMkdir(INode[] inodes, int pos,
-      byte[] name, PermissionStatus permission, boolean inheritPermission,
-      long timestamp) throws QuotaExceededException {
-    inodes[pos] = addChild(inodes, pos, 
-        new INodeDirectory(name, permission, timestamp),
-        inheritPermission );
-  }
-  
-  /** Add a node child to the namespace. The full path name of the node is src. 
-   * QuotaExceededException is thrown if it violates quota limit */
-  private <T extends INode> T addNode(String src, T child, 
-      boolean inheritPermission) 
-  throws QuotaExceededException {
-    byte[][] components = INode.getPathComponents(src);
-    child.setLocalName(components[components.length-1]);
-    INode[] inodes = new INode[components.length];
-    synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
-      return addChild(inodes, inodes.length-1, child, inheritPermission);
-    }
-  }
-  
-  /** 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 {
-    long childSize = child.numItemsInTree();
-    updateCount(pathComponents, pos, childSize);
-    T addedNode = ((INodeDirectory)pathComponents[pos-1]).addChild(
-        child, inheritPermission);
-    if (addedNode == null) {
-      updateCount(pathComponents, pos, -childSize);
-    }
-    return addedNode;
-  }
-  
-  /** 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 {
-    INode removedNode = 
-      ((INodeDirectory)pathComponents[pos-1]).removeChild(pathComponents[pos]);
-    if (removedNode != null) {
-      updateCount(pathComponents, pos, 
-          -removedNode.numItemsInTree());
-    }
-    return removedNode;
-  }
-  
   /**
    */
   String normalizePath(String src) {
@@ -930,122 +763,9 @@
     }
   }
 
-  /** Update the count of each directory with quota in the namespace
-   * A directory's count is defined as the total number inodes in the tree
-   * rooted at the directory.
-   * 
-   * @throws QuotaExceededException if the count update violates 
-   *                                any quota limitation
-   */
-  void updateCountForINodeWithQuota() throws QuotaExceededException {
-    updateCountForINodeWithQuota(rootDir);
-  }
-  
-  /** Update the count of the directory if it has a quota and return the count
-   * 
-   * @param node the root of the tree that represents the directory
-   * @return the size of the tree
-   * @throws QuotaExceededException if the count is greater than its quota
-   */
-  private static long updateCountForINodeWithQuota(INode node) throws QuotaExceededException {
-    long count = 1L;
-    if (node.isDirectory()) {
-      INodeDirectory dNode = (INodeDirectory)node;
-      for (INode child : dNode.getChildren()) {
-        count += updateCountForINodeWithQuota(child);
-      }
-      if (dNode.getQuota()>=0) {
-        ((INodeDirectoryWithQuota)dNode).setCount(count);
-      }
-    }
-    return count;
-  }
-  
-  /**
-   * Set the quota for a directory.
-   * @param path The string representation of the path to the directory
-   * @param quota The limit of the number of names in or below the directory
-   * @throws FileNotFoundException if the path does not exist or is a file
-   * @throws QuotaExceededException if the directory tree size is 
-   *                                greater than the given quota
-   */
-  void unprotectedSetQuota(String src, long quota)
-  throws FileNotFoundException, QuotaExceededException {
-    String srcs = normalizePath(src);
-    byte[][] components = INode.getPathComponents(src);
-    INode[] inodes = new INode[components.length==1?1:2];
-    synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
-      INode targetNode = inodes[inodes.length-1];
-      if (targetNode == null || !targetNode.isDirectory()) {
-        throw new FileNotFoundException("Directory does not exist: " + srcs);
-      } else { // a directory inode
-        INodeDirectory dirNode = (INodeDirectory)targetNode;
-        if (dirNode instanceof INodeDirectoryWithQuota) { 
-          // a directory with quota; so set the quota to the new value
-          ((INodeDirectoryWithQuota)dirNode).setQuota(quota);
-        } else {
-          // a non-quota directory; so replace it with a directory with quota
-          INodeDirectoryWithQuota newNode = 
-            new INodeDirectoryWithQuota(quota, dirNode);
-          // non-root directory node; parent != null
-          assert inodes.length==2;
-          INodeDirectory parent = (INodeDirectory)inodes[0];
-          parent.replaceChild(newNode);
-        }
-      }
-    }
-  }
-  
-  /**
-   * @see #unprotectedSetQuota(String, long)
-   */
-  void setQuota(String src, long quota) 
-  throws FileNotFoundException, QuotaExceededException {
-    unprotectedSetQuota(src, quota);
-    fsImage.getEditLog().logSetQuota(src, quota);
-  }
-  
-  /**
-   * Remove the quota for a directory
-   * @param src The string representation of the path to the directory
-   * @throws FileNotFoundException if the path does not exist or it is a file
-   */
-  void unprotectedClearQuota(String src) throws IOException {
-    String srcs = normalizePath(src);
-    byte[][] components = INode.getPathComponents(src);
-    INode[] inodes = new INode[components.length==1?1:2];
-    synchronized (rootDir) {
-      rootDir.getExistingPathINodes(components, inodes);
-      INode targetNode = inodes[inodes.length-1];
-      if (targetNode == null || !targetNode.isDirectory()) {
-        throw new FileNotFoundException("Directory does not exist: " + srcs);
-      } else if (targetNode instanceof INodeDirectoryWithQuota) {
-        // a directory inode with quota
-        // replace the directory with quota with a non-quota one
-        INodeDirectoryWithQuota dirNode = (INodeDirectoryWithQuota)targetNode;
-        INodeDirectory newNode = new INodeDirectory(dirNode);
-        if (dirNode == rootDir) { // root
-          throw new IOException("Can't clear the root's quota");
-        } else { // non-root directory node; parent != null
-          INodeDirectory parent = (INodeDirectory)inodes[0];
-          parent.replaceChild(newNode);
-        }
-      }
-    }
-  }
-  
-  /**
-   * @see #unprotectedClearQuota(String)
-   */
-  void clearQuota(String src) throws IOException {
-    unprotectedClearQuota(src);
-    fsImage.getEditLog().logClearQuota(src);
-  }
-  
   long totalInodes() {
     synchronized (rootDir) {
-      return rootDir.numItemsInTree();
+      return totalInodes;
     }
   }
 }

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java Thu Jun  5 00:06:46 2008
@@ -28,6 +28,8 @@
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.util.ArrayList;
 import java.lang.Math;
 import java.nio.channels.FileChannel;
@@ -52,8 +54,6 @@
   private static final byte OP_SET_OWNER = 8;
   private static final byte OP_CLOSE = 9;    // close after write
   private static final byte OP_SET_GENSTAMP = 10;    // store genstamp
-  private static final byte OP_SET_QUOTA = 11; // set a directory's quota
-  private static final byte OP_CLEAR_QUOTA = 12; // clear a directory's quota
   private static int sizeFlushBuffer = 512*1024;
 
   private ArrayList<EditLogOutputStream> editStreams = null;
@@ -632,23 +632,6 @@
                 FSImage.readString(in), FSImage.readString(in));
             break;
           }
-          case OP_SET_QUOTA: {
-            if (logVersion > -16) {
-              throw new IOException("Unexpected opcode " + opcode
-                  + " for version " + logVersion);
-            }
-            fsDir.unprotectedSetQuota(FSImage.readString(in), 
-                readLongWritable(in) );
-            break;
-          }
-          case OP_CLEAR_QUOTA: {
-            if (logVersion > -16) {
-              throw new IOException("Unexpected opcode " + opcode
-                  + " for version " + logVersion);
-            }
-            fsDir.unprotectedClearQuota(FSImage.readString(in));
-            break;
-          }
           default: {
             throw new IOException("Never seen opcode " + opcode);
           }
@@ -677,17 +660,6 @@
     return numEdits;
   }
 
-  // a place holder for reading a long
-  private static final LongWritable longWritable = new LongWritable();
-
-  /** Read an integer from an input stream */
-  private static long readLongWritable(DataInputStream in) throws IOException {
-    synchronized (longWritable) {
-      longWritable.readFields(in);
-      return longWritable.get();
-    }
-  }
-  
   static short adjustReplication(short replication) {
     FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
     short minReplication = fsNamesys.getMinReplication();
@@ -902,23 +874,6 @@
             FSEditLog.toLogReplication(replication));
   }
   
-  /** Add set quota record to edit log
-   * 
-   * @param src the string representation of the path to a directory
-   * @param quota the directory size limit
-   */
-  void logSetQuota(String src, long quota) {
-    logEdit(OP_SET_QUOTA, new UTF8(src), new LongWritable(quota));
-  }
-
-  /** Add clear quota record to edit log
-   * 
-   * @param src the string representation of the path to a directory
-   */
-  void logClearQuota(String src) {
-    logEdit(OP_CLEAR_QUOTA, new UTF8(src));
-  }
-  
   /**  Add set permissions record to edit log */
   void logSetPermissions(String src, FsPermission permissions) {
     logEdit(OP_SET_PERMISSIONS, new UTF8(src), permissions);

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSImage.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSImage.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSImage.java Thu Jun  5 00:06:46 2008
@@ -709,12 +709,8 @@
       this.namespaceID = in.readInt();
 
       // read number of files
-      long numFiles;
-      if (imgVersion <= -16) {
-        numFiles = in.readLong();
-      } else {
-        numFiles = in.readInt();
-      }
+      int numFiles = 0;
+      numFiles = in.readInt();
 
       this.layoutVersion = imgVersion;
       // read in the last generation stamp.
@@ -733,7 +729,7 @@
       String path;
       String parentPath = "";
       INodeDirectory parentINode = fsDir.rootDir;
-      for (long i = 0; i < numFiles; i++) {
+      for (int i = 0; i < numFiles; i++) {
         long modificationTime = 0;
         long blockSize = 0;
         path = readString(in);
@@ -773,20 +769,13 @@
             blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
           }
         }
-        
-        // get quota only when the node is a directory
-        long quota = -1L;
-        if (imgVersion <= -16 && blocks == null) {
-          quota = in.readLong();
-        }
-        
         PermissionStatus permissions = fsNamesys.getUpgradePermission();
         if (imgVersion <= -11) {
           permissions = PermissionStatus.read(in);
         }
-        if (path.length() == 0) { // it is the root
+        // check if this is a root node
+        if (path.length() == 0) {
           // update the root's attributes
-          fsDir.rootDir.setQuota(quota);
           fsDir.rootDir.setModificationTime(modificationTime);
           fsDir.rootDir.setPermissionStatus(permissions);
           continue;
@@ -798,7 +787,7 @@
         }
         // add new inode
         parentINode = fsDir.addToParent(path, parentINode, permissions,
-            blocks, replication, modificationTime, quota, blockSize);
+            blocks, replication, modificationTime, blockSize);
       }
       
       // load datanode info
@@ -806,9 +795,6 @@
 
       // load Files Under Construction
       this.loadFilesUnderConstruction(imgVersion, in, fsNamesys);
-      
-      // update the count of each directory with quota
-      fsDir.updateCountForINodeWithQuota();
     } finally {
       in.close();
     }
@@ -861,7 +847,7 @@
     try {
       out.writeInt(FSConstants.LAYOUT_VERSION);
       out.writeInt(namespaceID);
-      out.writeLong(fsDir.rootDir.numItemsInTree());
+      out.writeInt(fsDir.rootDir.numItemsInTree());
       out.writeLong(fsNamesys.getGenerationStamp());
       byte[] byteStore = new byte[4*FSConstants.MAX_PATH_LENGTH];
       ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
@@ -972,7 +958,6 @@
       out.writeLong(node.getModificationTime());
       out.writeLong(0);   // preferred block size
       out.writeInt(-1);    // # of blocks
-      out.writeLong(node.getQuota());
       FILE_PERM.fromShort(node.getFsPermissionShort());
       PermissionStatus.write(out, node.getUserName(),
                              node.getGroupName(),

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Thu Jun  5 00:06:46 2008
@@ -1590,36 +1590,6 @@
     return dir.getContentSummary(src);
   }
 
-  /**
-   * Set the quota for a directory.
-   * @param path The string representation of the path to the directory
-   * @param quota The limit of the number of names in or below the directory
-   * @throws IOException if the path is not a directory or the number of
-   * existing names in or below the directory is greater than the given quota
-   */
-  void setQuota(String path, long quota) throws IOException {
-    if (isPermissionEnabled) {
-      checkSuperuserPrivilege();
-    }
-    
-    dir.setQuota(path, quota);
-    getEditLog().logSync();
-  }
-  
-  /**
-   * Remove the quota for a directory
-   * @param path The string representation of the path to the directory
-   * @throws IOException if the path is not a directory
-   */
-  void clearQuota(String path) throws IOException {
-    if (isPermissionEnabled) {
-      checkSuperuserPrivilege();
-    }
-    
-    dir.clearQuota(path);
-    getEditLog().logSync();
-  }
-  
   /** Persist all metadata about this file.
    * @param src The string representation of the path
    * @param clientName The string representation of the client

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/INode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/INode.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/INode.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/INode.java Thu Jun  5 00:06:46 2008
@@ -85,17 +85,6 @@
     this(permissions, 0L);
     setLocalName(name);
   }
-  
-  /** copy constructor
-   * 
-   * @param other Other node to be copied
-   */
-  INode(INode other) {
-    setLocalName(other.getLocalName());
-    this.parent = other.getParent();
-    setPermissionStatus(other.getPermissionStatus());
-    setModificationTime(other.getModificationTime());
-  }
 
   /**
    * Check whether this is the root inode.
@@ -165,32 +154,15 @@
   /** Compute {@link ContentSummary}. */
   final ContentSummary computeContentSummary() {
     long[] a = computeContentSummary(new long[]{0,0,0});
-    return new ContentSummary(a[0], a[1], a[2], getQuota());
+    return new ContentSummary(a[0], a[1], a[2]);
   }
   /**
    * @return an array of three longs. 
    * 0: length, 1: file count, 2: directory count
    */
   abstract long[] computeContentSummary(long[] summary);
-  
-  /**
-   * Get the quota set for this inode
-   * @return the quota if it is set; -1 otherwise
-   */
-  long getQuota() {
-    return -1;
-  }
 
   /**
-   * Get the total number of names in the tree
-   * rooted at this inode including the root
-   * @return The total number of names in this tree
-   */
-  long numItemsInTree() {
-    return 1;
-  }
-    
-  /**
    * Get local file name
    * @return local file name
    */
@@ -389,21 +361,6 @@
     this.children = null;
   }
 
-  /** constructor */
-  INodeDirectory(byte[] localName, PermissionStatus permissions, long mTime) {
-    this(permissions, mTime);
-    this.name = localName;
-  }
-  
-  /** copy constructor
-   * 
-   * @param other
-   */
-  INodeDirectory(INodeDirectory other) {
-    super(other);
-    this.children = other.getChildren();
-  }
-  
   /**
    * Check whether it's a directory
    */
@@ -411,32 +368,14 @@
     return true;
   }
 
-  INode removeChild(INode node) {
+  void removeChild(INode node) {
     assert children != null;
     int low = Collections.binarySearch(children, node.name);
     if (low >= 0) {
-      return children.remove(low);
-    } else {
-      return null;
+      children.remove(low);
     }
   }
 
-  /** Replace a child that has the same name as newChild by newChild.
-   * 
-   * @param newChild Child node to be added
-   */
-  void replaceChild(INode newChild) {
-    if ( children == null ) {
-      throw new IllegalArgumentException("The directory is empty");
-    }
-    int low = Collections.binarySearch(children, newChild.name);
-    if (low>=0) { // an old child exists so replace by the newChild
-      children.set(low, newChild);
-    } else {
-      throw new IllegalArgumentException("No child exists to be replaced");
-    }
-  }
-  
   INode getChild(String name) {
     return getChildINode(string2Bytes(name));
   }
@@ -645,13 +584,16 @@
 
   /**
    */
-  long numItemsInTree() {
-    long total = 1L;
+  int numItemsInTree() {
+    int total = 1;
     if (children == null) {
       return total;
     }
     for (INode child : children) {
-      total += child.numItemsInTree();
+      if(!child.isDirectory())
+        total++;
+      else
+        total += ((INodeDirectory)child).numItemsInTree();
     }
     return total;
   }
@@ -690,100 +632,6 @@
   }
 }
 
-/**
- * Directory INode class that has a quota restriction
- */
-class INodeDirectoryWithQuota extends INodeDirectory {
-  private long quota;
-  private long count;
-  
-  /** Convert an existing directory inode to one with the given quota
-   * 
-   * @param quota Quota to be assigned to this inode
-   * @param other The other inode from which all other properties are copied
-   */
-  INodeDirectoryWithQuota(long quota, INodeDirectory other)
-  throws QuotaExceededException {
-    super(other);
-    this.count = other.numItemsInTree();
-    setQuota(quota);
-  }
-  
-  /** constructor with no quota verification */
-  INodeDirectoryWithQuota(
-      PermissionStatus permissions, long modificationTime, long quota)
-  {
-    super(permissions, modificationTime);
-    this.quota = quota;
-  }
-  
-  /** constructor with no quota verification */
-  INodeDirectoryWithQuota(String name, PermissionStatus permissions, long quota)
-  {
-    super(name, permissions);
-    this.quota = quota;
-  }
-  
-  /** Get this directory's quota
-   * @return this directory's quota
-   */
-  long getQuota() {
-    return quota;
-  }
-  
-  /** Set this directory's quota
-   * 
-   * @param quota Quota to be set
-   * @throws QuotaExceededException if the given quota is less than 
-   *                                the size of the tree
-   */
-  void setQuota(long quota) throws QuotaExceededException {
-    verifyQuota(quota, this.count);
-    this.quota = quota;
-  }
-  
-  /** Get the number of names in the subtree rooted at this directory
-   * @return the size of the subtree rooted at this directory
-   */
-  long numItemsInTree() {
-    return count;
-  }
-  
-  /** Update the size of the tree
-   * 
-   * @param delta the change of the tree size
-   * @throws QuotaExceededException if the changed size is greater 
-   *                                than the quota
-   */
-  void updateNumItemsInTree(long delta) throws QuotaExceededException {
-    long newCount = this.count + delta;
-    if (delta>0) {
-      verifyQuota(this.quota, newCount);
-    }
-    this.count = newCount;
-  }
-  
-  /** Set the size of the tree rooted at this directory
-   * 
-   * @param count size of the directory to be set
-   * @throws QuotaExceededException if the given count is greater than quota
-   */
-  void setCount(long count) throws QuotaExceededException {
-    verifyQuota(this.quota, count);
-    this.count = count;
-  }
-  
-  /** Verify if the count satisfies the quota restriction 
-   * @throws QuotaExceededException if the given quota is less than the count
-   */
-  private static void verifyQuota(long quota, long count)
-  throws QuotaExceededException {
-    if (quota < count) {
-      throw new QuotaExceededException(quota, count);
-    }
-  }
-}
-
 class INodeFile extends INode {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/NameNode.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/NameNode.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/NameNode.java Thu Jun  5 00:06:46 2008
@@ -550,16 +550,6 @@
   }
 
   /** {@inheritDoc} */
-  public void setQuota(String path, long quota) throws IOException {
-    namesystem.setQuota(path, quota);
-  }
-  
-  /** {@inheritDoc} */
-  public void clearQuota(String path) throws IOException {
-    namesystem.clearQuota(path);
-  }
-  
-  /** {@inheritDoc} */
   public void fsync(String src, String clientName) throws IOException {
     namesystem.fsync(src, clientName);
   }

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/fs/ContentSummary.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/fs/ContentSummary.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/fs/ContentSummary.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/fs/ContentSummary.java Thu Jun  5 00:06:46 2008
@@ -28,7 +28,6 @@
   private long length;
   private long fileCount;
   private long directoryCount;
-  private long quota;
 
   /** Constructor */
   public ContentSummary() {}
@@ -38,16 +37,6 @@
     this.length = length;
     this.fileCount = fileCount;
     this.directoryCount = directoryCount;
-    this.quota = -1L;
-  }
-
-  /** Constructor */
-  public ContentSummary(
-      long length, long fileCount, long directoryCount, long quota) {
-    this.length = length;
-    this.fileCount = fileCount;
-    this.directoryCount = directoryCount;
-    this.quota = quota;
   }
 
   /** @return the length */
@@ -59,15 +48,11 @@
   /** @return the file count */
   public long getFileCount() {return fileCount;}
   
-  /** Return the directory quota */
-  public long getQuota() {return quota;}
-  
   /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
     out.writeLong(length);
     out.writeLong(fileCount);
     out.writeLong(directoryCount);
-    out.writeLong(quota);
   }
 
   /** {@inheritDoc} */
@@ -75,7 +60,6 @@
     this.length = in.readLong();
     this.fileCount = in.readLong();
     this.directoryCount = in.readLong();
-    this.quota = in.readLong();
   }
   
   /** 
@@ -84,58 +68,13 @@
    *    DIR_COUNT   FILE_COUNT       CONTENT_SIZE FILE_NAME    
    */
   private static final String STRING_FORMAT = "%12d %12d %18d ";
-  /** 
-   * Output format:
-   * <----12----> <----15----> <----12----> <----12----> <-------18------->
-   *    QUOTA   REMAINING_QUATA  DIR_COUNT   FILE_COUNT        CONTENT_SIZE FILE_NAME    
-   */
-  private static final String QUOTA_STRING_FORMAT = "%12d %15d "+STRING_FORMAT;
-  private static final String NON_QUOTA_STRING_FORMAT =
-    "%12s %15s "+STRING_FORMAT;
 
   /** The header string */
-  private static final String HEADER = String.format(
+  static String HEADER = String.format(
       STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
 
-  private static final String QUOTA_HEADER = String.format(
-      QUOTA_STRING_FORMAT.replace('d', 's'), 
-      "quota", "remaining quota", "directories", "files", "bytes");
-  
-  /** Return the header of the output.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the header of the output
-   */
-  public static String getHeader(boolean qOption) {
-    return qOption ? QUOTA_HEADER : HEADER;
-  }
-  
   /** {@inheritDoc} */
   public String toString() {
-    return toString(true);
-  }
-
-  /** Return the string representation of the object in the output format.
-   * if qOption is false, output directory count, file count, and content size;
-   * if qOption is true, output quota and remaining quota as well.
-   * 
-   * @param qOption a flag indicating if quota needs to be printed or not
-   * @return the string representation of the object
-   */
-  public String toString(boolean qOption) {
-    if (qOption) {
-      if (quota>0) {
-        long remainingQuota = quota-(directoryCount+fileCount);
-        return String.format(QUOTA_STRING_FORMAT, quota, remainingQuota,
-            directoryCount, fileCount, length);
-      } else {
-        return String.format(NON_QUOTA_STRING_FORMAT, "none", "inf",
-            directoryCount, fileCount, length);
-      }
-    } else {
-      return String.format(STRING_FORMAT, directoryCount, fileCount, length);
-    }
+    return String.format(STRING_FORMAT, directoryCount, fileCount, length);
   }
 }
\ No newline at end of file

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/fs/FsShell.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/fs/FsShell.java Thu Jun  5 00:06:46 2008
@@ -30,7 +30,6 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.fs.shell.Count;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -176,8 +175,8 @@
       System.err.println("Usage: java FsShell " + GET_SHORT_USAGE);
       throw iae;
     }
-    final boolean copyCrc = cf.getOpt("crc");
-    final boolean verifyChecksum = !cf.getOpt("ignoreCrc");
+    final boolean copyCrc = cf.options.get("crc");
+    final boolean verifyChecksum = !cf.options.get("ignoreCrc");
 
     if (dststr.equals("-")) {
       if (copyCrc) {
@@ -422,6 +421,42 @@
   }
 
   /**
+   * Parse the args of a command and check the format of args.
+   */
+  static class CommandFormat {
+    final String name;
+    final int minPar, maxPar;
+    final Map<String, Boolean> options = new HashMap<String, Boolean>();
+
+    CommandFormat(String n, int min, int max, String ... possibleOpt) {
+      name = n;
+      minPar = min;
+      maxPar = max;
+      for(String opt : possibleOpt)
+        options.put(opt, Boolean.FALSE);
+    }
+
+    List<String> parse(String[] args, int pos) {
+      List<String> parameters = new ArrayList<String>();
+      for(; pos < args.length; pos++) {
+        if (args[pos].charAt(0) == '-' && args[pos].length() > 1) {
+          String opt = args[pos].substring(1);
+          if (options.containsKey(opt))
+            options.put(opt, Boolean.TRUE);
+          else
+            throw new IllegalArgumentException("Illegal option " + args[pos]);
+        }
+        else
+          parameters.add(args[pos]);
+      }
+      int psize = parameters.size();
+      if (psize < minPar || psize > maxPar)
+        throw new IllegalArgumentException("Illegal number of arguments");
+      return parameters;
+    }
+  }
+
+  /**
    * Parse the incoming command string
    * @param cmd
    * @param pos ignore anything before this pos in cmd
@@ -450,8 +485,8 @@
       throw new IllegalArgumentException("replication must be >= 1");
     }
 
-    List<Path> waitList = c.getOpt("w")? new ArrayList<Path>(): null;
-    setReplication(rep, dst, c.getOpt("R"), waitList);
+    List<Path> waitList = c.options.get("w")? new ArrayList<Path>(): null;
+    setReplication(rep, dst, c.options.get("R"), waitList);
 
     if (waitList != null) {
       waitForReplication(waitList, rep);
@@ -1082,7 +1117,7 @@
       System.err.println("Usage: java FsShell " + TAIL_USAGE);
       throw iae;
     }
-    boolean foption = c.getOpt("f") ? true: false;
+    boolean foption = c.options.get("f") ? true: false;
     path = new Path(src);
     FileSystem srcFs = path.getFileSystem(getConf());
     if (srcFs.isDirectory(path)) {
@@ -1458,7 +1493,7 @@
       System.out.println(chown);
     } else if ("chgrp".equals(cmd)) {
       System.out.println(chgrp);
-    } else if (Count.matches(cmd)) {
+    } else if (Count.NAME.equals(cmd)) {
       System.out.println(Count.DESCRIPTION);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
@@ -1497,7 +1532,8 @@
    * Apply operation specified by 'cmd' on all parameters
    * starting from argv[startindex].
    */
-  private int doall(String cmd, String argv[], int startindex) {
+  private int doall(String cmd, String argv[], Configuration conf, 
+                    int startindex) {
     int exitCode = 0;
     int i = startindex;
     //
@@ -1521,7 +1557,7 @@
         } else if ("-dus".equals(cmd)) {
           dus(argv[i]);
         } else if (Count.matches(cmd)) {
-          new Count(argv, i, fs).runAll();
+          Count.count(argv[i], getConf(), System.out);
         } else if ("-ls".equals(cmd)) {
           ls(argv[i], false);
         } else if ("-lsr".equals(cmd)) {
@@ -1566,7 +1602,7 @@
    * Displays format of commands.
    * 
    */
-  private static void printUsage(String cmd) {
+  void printUsage(String cmd) {
     String prefix = "Usage: java " + FsShell.class.getSimpleName();
     if ("-fs".equals(cmd)) {
       System.err.println("Usage: java FsShell" + 
@@ -1725,9 +1761,9 @@
         else
           copyMergeToLocal(argv[i++], new Path(argv[i++]));
       } else if ("-cat".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-text".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-moveToLocal".equals(cmd)) {
         moveToLocal(argv[i++], new Path(argv[i++]));
       } else if ("-setrep".equals(cmd)) {
@@ -1738,13 +1774,13 @@
         FsShellPermissions.changePermissions(fs, cmd, argv, i, this);
       } else if ("-ls".equals(cmd)) {
         if (i < argv.length) {
-          exitCode = doall(cmd, argv, i);
+          exitCode = doall(cmd, argv, getConf(), i);
         } else {
           ls(Path.CUR_DIR, false);
         } 
       } else if ("-lsr".equals(cmd)) {
         if (i < argv.length) {
-          exitCode = doall(cmd, argv, i);
+          exitCode = doall(cmd, argv, getConf(), i);
         } else {
           ls(Path.CUR_DIR, true);
         } 
@@ -1753,29 +1789,33 @@
       } else if ("-cp".equals(cmd)) {
         exitCode = copy(argv, getConf());
       } else if ("-rm".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-rmr".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-expunge".equals(cmd)) {
         expunge();
       } else if ("-du".equals(cmd)) {
         if (i < argv.length) {
-          exitCode = doall(cmd, argv, i);
+          exitCode = doall(cmd, argv, getConf(), i);
         } else {
           du(".");
         }
       } else if ("-dus".equals(cmd)) {
         if (i < argv.length) {
-          exitCode = doall(cmd, argv, i);
+          exitCode = doall(cmd, argv, getConf(), i);
         } else {
           dus(".");
         }         
       } else if (Count.matches(cmd)) {
-        exitCode = new Count(argv, i, fs).runAll();
+        if (i < argv.length) {
+          exitCode = doall(cmd, argv, getConf(), i);
+        } else {
+          Count.count(".", getConf(), System.out);
+        }         
       } else if ("-mkdir".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-touchz".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
+        exitCode = doall(cmd, argv, getConf(), i);
       } else if ("-test".equals(cmd)) {
         exitCode = test(argv, i);
       } else if ("-stat".equals(cmd)) {
@@ -1797,10 +1837,6 @@
         System.err.println(cmd.substring(1) + ": Unknown command");
         printUsage("");
       }
-    } catch (IllegalArgumentException arge) {
-      exitCode = -1;
-      System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
-      printUsage(cmd);
     } catch (RemoteException e) {
       //
       // This is a error returned by hadoop server. Print
@@ -1822,7 +1858,7 @@
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": " + 
                          e.getLocalizedMessage());  
-    } catch (Exception re) {
+    } catch (RuntimeException re) {
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": " + re.getLocalizedMessage());  
     } finally {

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/fs/shell/Count.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/fs/shell/Count.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/fs/shell/Count.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/fs/shell/Count.java Thu Jun  5 00:06:46 2008
@@ -18,58 +18,40 @@
 package org.apache.hadoop.fs.shell;
 
 import java.io.*;
-import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Count the number of directories, files, bytes, quota, and remaining quota.
+ * Count the number of directories, files and bytes.
  */
-public class Count extends Command {
+public class Count {
   public static final String NAME = "count";
-  public static final String USAGE = "-" + NAME + "[-q] <path>";
+  public static final String USAGE = "-" + NAME + " <path>";
   public static final String DESCRIPTION = CommandUtils.formatDescription(USAGE, 
       "Count the number of directories, files and bytes under the paths",
       "that match the specified file pattern.  The output columns are:",
-      "DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME or",
-      "QUOTA REMAINING_QUATA DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME");
-  
-  private boolean qOption;
+      "DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME");
 
-  /** Constructor
-   * 
-   * @param cmd the count command
-   * @param pos the starting index of the arguments 
-   * @param fs the file system handler
-   */
-  public Count(String[] cmd, int pos, FileSystem fs) {
-    super(fs);
-    CommandFormat c = new CommandFormat(NAME, 1, Integer.MAX_VALUE, "q");
-    List<String> parameters = c.parse(cmd, pos);
-    this.args = parameters.toArray(new String[parameters.size()]);
-    if (this.args.length == 0) { // default path is the current working directory
-      this.args = new String[] {"."};
-    }
-    this.qOption = c.getOpt("q") ? true: false;
-  }
-  
-  /** Check if a command is the count command
-   * 
-   * @param cmd A string representation of a command starting with "-"
-   * @return true if this is a count command; false otherwise
-   */
   public static boolean matches(String cmd) {
     return ("-" + NAME).equals(cmd); 
   }
 
-  @Override
-  public String getCommandName() {
-    return NAME;
-  }
-
-  @Override
-  protected void run(Path path) throws IOException {
-    System.out.println(fs.getContentSummary(path).toString(qOption) + path);
+  public static void count(String src, Configuration conf, PrintStream out
+      ) throws IOException {
+    Path srcPath = new Path(src);
+    FileSystem srcFs = srcPath.getFileSystem(conf);
+    FileStatus[] statuses = srcFs.globStatus(srcPath);
+    if (statuses == null || statuses.length == 0) {
+      throw new FileNotFoundException(src + " not found.");
+    }
+    for(FileStatus s : statuses) {
+      Path p = s.getPath();
+      String pathstr = p.toString();
+      out.println(srcFs.getContentSummary(p)
+          + ("".equals(pathstr)? ".": pathstr));
+    }
   }
 }
\ No newline at end of file

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/ipc/RemoteException.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/ipc/RemoteException.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/ipc/RemoteException.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/ipc/RemoteException.java Thu Jun  5 00:06:46 2008
@@ -82,12 +82,7 @@
       throws Exception {
     Constructor<? extends IOException> cn = cls.getConstructor(String.class);
     cn.setAccessible(true);
-    String firstLine = this.getMessage();
-    int eol = firstLine.indexOf('\n');
-    if (eol>=0) {
-      firstLine = firstLine.substring(0, eol);
-    }
-    IOException ex = cn.newInstance(firstLine);
+    IOException ex = cn.newInstance(this.getMessage());
     ex.initCause(this);
     return ex;
   }

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java Thu Jun  5 00:06:46 2008
@@ -27,7 +27,6 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.shell.*;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
@@ -670,10 +669,10 @@
       String root = createTree(dfs, "count");
 
       // Verify the counts
-      runCount(root, 2, 4, dfs);
-      runCount(root + "2", 2, 1, dfs);
-      runCount(root + "2/f1", 0, 1, dfs);
-      runCount(root + "2/sub", 1, 0, dfs);
+      runCount(root, 2, 4, conf);
+      runCount(root + "2", 2, 1, conf);
+      runCount(root + "2/f1", 0, 1, conf);
+      runCount(root + "2/sub", 1, 0, conf);
     } finally {
       try {
         dfs.close();
@@ -682,25 +681,18 @@
       cluster.shutdown();
     }
   }
-  private void runCount(String path, long dirs, long files, FileSystem fs
-    ) throws IOException {
+  private void runCount(String path, long dirs, long files, Configuration conf
+      ) throws IOException {
     ByteArrayOutputStream bytes = new ByteArrayOutputStream(); 
     PrintStream out = new PrintStream(bytes);
-    PrintStream oldOut = System.out;
-    System.setOut(out);
-    Scanner in = null;
-    try {
-      new Count(new String[]{path}, 0, fs).runAll();
-      String results = bytes.toString();
-      System.out.println(results);
-      in = new Scanner(results);
-      assertEquals(dirs, in.nextLong());
-      assertEquals(files, in.nextLong());
-    } finally {
-      if (in!=null) in.close();
-      IOUtils.closeStream(out);
-      System.setOut(oldOut);
-    }
+    Count.count(path, conf, out);
+    String results = bytes.toString();
+    System.out.println(results);
+    Scanner in = new Scanner(results);
+    assertEquals(dirs, in.nextLong());
+    assertEquals(files, in.nextLong());
+    in.close();
+    out.close();
   }
 
   //throws IOException instead of Exception as shell.run() does.

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java?rev=663483&r1=663482&r2=663483&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java Thu Jun  5 00:06:46 2008
@@ -32,7 +32,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.fs.FsShell.CommandFormat;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.UTF8;