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 sz...@apache.org on 2011/04/07 23:59:38 UTC

svn commit: r1090039 - in /hadoop/common/trunk: ./ src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/fs/shell/ src/test/core/org/apache/hadoop/fs/shell/

Author: szetszwo
Date: Thu Apr  7 21:59:37 2011
New Revision: 1090039

URL: http://svn.apache.org/viewvc?rev=1090039&view=rev
Log:
HADOOP-7202. Improve shell Command base class.  Contributed by Daryn Sharp

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1090039&r1=1090038&r2=1090039&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Thu Apr  7 21:59:37 2011
@@ -82,6 +82,8 @@ Trunk (unreleased changes)
     HADOOP-7180. Better support on CommandFormat on the API and exceptions.
     (Daryn Sharp via szetszwo)
 
+    HADOOP-7202. Improve shell Command base class.  (Daryn Sharp via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java?rev=1090039&r1=1090038&r2=1090039&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FsShell.java Thu Apr  7 21:59:37 2011
@@ -23,7 +23,12 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.TimeZone;
 import java.util.zip.GZIPInputStream;
 
 import org.apache.commons.logging.Log;
@@ -44,9 +49,9 @@ import org.apache.hadoop.io.compress.Com
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.util.StringUtils;
 
 /** Provide command line access to a FileSystem. */
 @InterfaceAudience.Private
@@ -370,7 +375,7 @@ public class FsShell extends Configured 
 
   private class TextRecordInputStream extends InputStream {
     SequenceFile.Reader r;
-    WritableComparable key;
+    WritableComparable<?> key;
     Writable val;
 
     DataInputBuffer inbuf;
@@ -1691,8 +1696,6 @@ public class FsShell extends Configured 
           delete(argv[i], true, rmSkipTrash);
         } else if ("-df".equals(cmd)) {
           df(argv[i]);
-        } else if (Count.matches(cmd)) {
-          new Count(argv, i, getConf()).runAll();
         } else if ("-ls".equals(cmd)) {
           exitCode = ls(argv[i], false);
         } else if ("-lsr".equals(cmd)) {
@@ -1759,7 +1762,7 @@ public class FsShell extends Configured 
     } else if ("-df".equals(cmd) ) {
       System.err.println("Usage: java FsShell" +
                          " [" + cmd + " [<path>]]");
-    } else if (Count.matches(cmd)) {
+    } else if ("-count".equals(cmd)) {
       System.err.println(prefix + " [" + Count.USAGE + "]");
     } else if ("-rm".equals(cmd) || "-rmr".equals(cmd)) {
       System.err.println("Usage: java FsShell [" + cmd + 
@@ -1948,8 +1951,14 @@ public class FsShell extends Configured 
         du(argv, i);
       } else if ("-dus".equals(cmd)) {
         dus(argv, i);
-      } else if (Count.matches(cmd)) {
-        exitCode = new Count(argv, i, getConf()).runAll();
+      } else if ("-count".equals(cmd)) {
+        // TODO: next two lines are a temporary crutch until this entire
+        // block is overhauled
+        LinkedList<String> args = new LinkedList<String>(Arrays.asList(argv));
+        String cmdName = args.removeFirst();
+        Count runner = ReflectionUtils.newInstance(Count.class, getConf());
+        runner.setCommandName(cmdName); // TODO: will change with factory
+        exitCode = runner.run(args);
       } else if ("-mkdir".equals(cmd)) {
         exitCode = doall(cmd, argv, i);
       } else if ("-touchz".equals(cmd)) {

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java?rev=1090039&r1=1090038&r2=1090039&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Command.java Thu Apr  7 21:59:37 2011
@@ -17,29 +17,67 @@
  */
 package org.apache.hadoop.fs.shell;
 
-import java.io.*;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * An abstract class for the execution of a file system command
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
 abstract public class Command extends Configured {
   protected String[] args;
+  protected String name;
+  protected int exitCode = 0;
+  protected int numErrors = 0;
+  protected boolean recursive = false;
+  protected ArrayList<Exception> exceptions = new ArrayList<Exception>();
+
+  private static final Log LOG = LogFactory.getLog(Command.class);
+
+  /** allows stdout to be captured if necessary */
+  public PrintStream out = System.out;
+  /** allows stderr to be captured if necessary */
+  public PrintStream err = System.err;
+
+  /** Constructor */
+  protected Command() {
+    out = System.out;
+    err = System.err;
+  }
   
   /** Constructor */
   protected Command(Configuration conf) {
     super(conf);
   }
   
-  /** Return the command's name excluding the leading character - */
+  /** @return the command's name excluding the leading character - */
   abstract public String getCommandName();
   
+  /**
+   * Name the command
+   * @param cmdName as invoked
+   */
+  public void setCommandName(String cmdName) {
+    name = cmdName;
+  }
+  
   /** 
    * Execute the command on the input path
    * 
@@ -57,30 +95,272 @@ abstract public class Command extends Co
     int exitCode = 0;
     for (String src : args) {
       try {
-        Path srcPath = new Path(src);
-        FileSystem fs = srcPath.getFileSystem(getConf());
-        FileStatus[] statuses = fs.globStatus(srcPath);
-        if (statuses == null) {
-          System.err.println("Can not find listing for " + src);
-          exitCode = -1;
-        } else {
-          for(FileStatus s : statuses) {
-            run(s.getPath());
-          }
+        List<PathData> srcs = expandGlob(src);
+        for(PathData s : srcs) {
+          run(s.path);
         }
-      } catch (RemoteException re) {
-        exitCode = -1;
-        String content = re.getLocalizedMessage();
-        int eol = content.indexOf('\n');
-        if (eol>=0) {
-          content = content.substring(0, eol);
-        }
-        System.err.println(getCommandName() + ": " + content);
       } catch (IOException e) {
         exitCode = -1;
-        System.err.println(getCommandName() + ": " + e.getLocalizedMessage());
+        displayError(e);
       }
     }
     return exitCode;
   }
+
+  /**
+   * Invokes the command handler.  The default behavior is to process options,
+   * expand arguments, and then process each argument.
+   * <pre>
+   * run
+   * \-> {@link #processOptions(LinkedList)}
+   * \-> {@link #expandArguments(LinkedList)} -> {@link #expandArgument(String)}*
+   * \-> {@link #processArguments(LinkedList)}
+   *     \-> {@link #processArgument(PathData)}*
+   *         \-> {@link #processPathArgument(PathData)}
+   *             \-> {@link #processPaths(PathData, PathData...)}
+   *                 \-> {@link #processPath(PathData)}*
+   *         \-> {@link #processNonexistentPathArgument(PathData)}
+   * </pre>
+   * Most commands will chose to implement just
+   * {@link #processOptions(LinkedList)} and {@link #processPath(PathData)}
+   * 
+   * @param args the list of command line arguments
+   * @return the exit code for the command
+   * @throws IllegalArgumentException if called with invalid arguments
+   */
+  public int run(LinkedList<String> args) {
+    try {
+      processOptions(args);
+      processArguments(expandArguments(args));
+    } catch (IOException e) {
+      displayError(e);
+    }
+    
+    return (numErrors == 0) ? exitCode : 1;
+  }
+
+  /**
+   * Must be implemented by commands to process the command line flags and
+   * check the bounds of the remaining arguments.  If an
+   * IllegalArgumentException is thrown, the FsShell object will print the
+   * short usage of the command.
+   * @param args the command line arguments
+   * @throws IOException
+   */
+  protected void processOptions(LinkedList<String> args) throws IOException {}
+
+  /**
+   *  Expands a list of arguments into {@link PathData} objects.  The default
+   *  behavior is to call {@link #expandArgument(String)} on each element
+   *  which by default globs the argument.  The loop catches IOExceptions,
+   *  increments the error count, and displays the exception.
+   * @param args strings to expand into {@link PathData} objects
+   * @return list of all {@link PathData} objects the arguments
+   * @throws IOException if anything goes wrong...
+   */
+  protected LinkedList<PathData> expandArguments(LinkedList<String> args)
+  throws IOException {
+    LinkedList<PathData> expandedArgs = new LinkedList<PathData>();
+    for (String arg : args) {
+      try {
+        expandedArgs.addAll(expandArgument(arg));
+      } catch (IOException e) { // other exceptions are probably nasty
+        displayError(e);
+      }
+    }
+    return expandedArgs;
+  }
+
+  /**
+   * Expand the given argument into a list of {@link PathData} objects.
+   * The default behavior is to expand globs.  Commands may override to
+   * perform other expansions on an argument.
+   * @param arg string pattern to expand
+   * @return list of {@link PathData} objects
+   * @throws IOException if anything goes wrong...
+   */
+  protected List<PathData> expandArgument(String arg) throws IOException {
+    return expandGlob(arg);
+  }
+
+  /**
+   *  Processes the command's list of expanded arguments.
+   *  {@link #processArgument(PathData)} will be invoked with each item
+   *  in the list.  The loop catches IOExceptions, increments the error
+   *  count, and displays the exception.
+   *  @param args a list of {@link PathData} to process
+   *  @throws IOException if anything goes wrong... 
+   */
+  protected void processArguments(LinkedList<PathData> args)
+  throws IOException {
+    for (PathData arg : args) {
+      try {
+        processArgument(arg);
+      } catch (IOException e) {
+        displayError(e);
+      }
+    }
+  }
+
+  /**
+   * Processes a {@link PathData} item, calling
+   * {@link #processPathArgument(PathData)} or
+   * {@link #processNonexistentPathArgument(PathData)} on each item.
+   * @param arg {@link PathData} item to process
+   * @throws IOException if anything goes wrong...
+   */
+  protected void processArgument(PathData item) throws IOException {
+    if (item.exists) {
+      processPathArgument(item);
+    } else {
+      processNonexistentPathArgument(item);
+    }
+  }
+
+  /**
+   *  This is the last chance to modify an argument before going into the
+   *  (possibly) recursive {@link #processPaths(PathData, PathData...)}
+   *  -> {@link #processPath(PathData)} loop.  Ex.  ls and du use this to
+   *  expand out directories.
+   *  @param item a {@link PathData} representing a path which exists
+   *  @throws IOException if anything goes wrong... 
+   */
+  protected void processPathArgument(PathData item) throws IOException {
+    // null indicates that the call is not via recursion, ie. there is
+    // no parent directory that was expanded
+    processPaths(null, item);
+  }
+  
+  /**
+   *  Provides a hook for handling paths that don't exist.  By default it
+   *  will throw an exception.  Primarily overriden by commands that create
+   *  paths such as mkdir or touch.
+   *  @param item the {@link PathData} that doesn't exist
+   *  @throws FileNotFoundException if arg is a path and it doesn't exist
+   *  @throws IOException if anything else goes wrong... 
+   */
+  protected void processNonexistentPathArgument(PathData item)
+  throws IOException {
+    // TODO: this should be more posix-like: ex. "No such file or directory"
+    throw new FileNotFoundException("Can not find listing for " + item);
+  }
+
+  /**
+   *  Iterates over the given expanded paths and invokes
+   *  {@link #processPath(PathData)} on each element.  If "recursive" is true,
+   *  will do a post-visit DFS on directories.
+   *  @param parent if called via a recurse, will be the parent dir, else null
+   *  @param items a list of {@link PathData} objects to process
+   *  @throws IOException if anything goes wrong...
+   */
+  protected void processPaths(PathData parent, PathData ... items)
+  throws IOException {
+    // TODO: this really should be iterative
+    for (PathData item : items) {
+      try {
+        processPath(item);
+        if (recursive && item.stat.isDirectory()) {
+          recursePath(item);
+        }
+      } catch (IOException e) {
+        displayError(e);
+      }
+    }
+  }
+
+  /**
+   * Hook for commands to implement an operation to be applied on each
+   * path for the command.  Note implementation of this method is optional
+   * if earlier methods in the chain handle the operation.
+   * @param item a {@link PathData} object
+   * @throws RuntimeException if invoked but not implemented
+   * @throws IOException if anything else goes wrong in the user-implementation
+   */  
+  protected void processPath(PathData item) throws IOException {
+    throw new RuntimeException("processPath() is not implemented");    
+  }
+
+  /**
+   *  Gets the directory listing for a path and invokes
+   *  {@link #processPaths(PathData, PathData...)}
+   *  @param item {@link PathData} for directory to recurse into
+   *  @throws IOException if anything goes wrong...
+   */
+  protected void recursePath(PathData item) throws IOException {
+    processPaths(item, item.getDirectoryContents());
+  }
+ 
+  /**
+   * Expand the given path as a glob pattern.  Non-existent paths do not 
+   * throw an exception because creation commands like touch and mkdir need
+   * to create them.  The "stat" field will be null if the path does not
+   * exist.
+   * @param pattern the glob to expand
+   * @return list of {@link PathData} objects
+   * @throws FileNotFoundException the path is a glob with no matches
+   * @throws IOException anything else goes wrong...
+   */
+  protected List<PathData> expandGlob(String pattern) throws IOException {
+    Path path = new Path(pattern);
+    FileSystem fs = path.getFileSystem(getConf());
+    FileStatus[] stats = fs.globStatus(path);
+    
+    if (stats != null && stats.length == 0) { // glob failed to match
+      // TODO: this should be more posix-like: ex. "No such file or directory"
+      throw new FileNotFoundException("Can not find listing for " + pattern);
+    }
+    
+    List<PathData> items = new LinkedList<PathData>();
+    if (stats == null) { // not a glob & file not found, so null stat block
+      items.add(new PathData(fs, path, null));
+    } else {
+      // convert all the stats to PathData objs
+      for (FileStatus stat : stats) {
+        items.add(new PathData(fs, stat));
+      }
+    }
+    return items;
+  }
+
+  /**
+   * Display an exception prefaced with the command name.  Also increments
+   * the error count for the command which will result in a non-zero exit
+   * code.
+   * @param e exception to display
+   */
+  public void displayError(Exception e) {
+    // build up a list of exceptions that occurred
+    exceptions.add(e);
+    
+    String errorMessage = e.getLocalizedMessage();
+    if (errorMessage == null) {
+      // this is an unexpected condition, so dump the whole exception since
+      // it's probably a nasty internal error where the backtrace would be
+      // useful
+      errorMessage = StringUtils.stringifyException(e);
+      LOG.debug(errorMessage);
+    } else {
+      errorMessage = errorMessage.split("\n", 2)[0];
+    }
+    displayError(errorMessage);
+  }
+  
+  /**
+   * Display an error string prefaced with the command name.  Also increments
+   * the error count for the command which will result in a non-zero exit
+   * code.
+   * @param message error message to display
+   */
+  public void displayError(String message) {
+    numErrors++;
+    displayWarning(message);
+  }
+  
+  /**
+   * Display an warning string prefaced with the command name.
+   * @param message warning message to display
+   */
+  public void displayWarning(String message) {
+    err.println(getCommandName() + ": " + message);
+  }
 }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java?rev=1090039&r1=1090038&r2=1090039&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Count.java Thu Apr  7 21:59:37 2011
@@ -17,17 +17,23 @@
  */
 package org.apache.hadoop.fs.shell;
 
-import java.io.*;
-import java.util.List;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FsShell;
 
 /**
  * Count the number of directories, files, bytes, quota, and remaining quota.
  */
-public class Count extends Command {
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+public class Count extends FsCommand {
   public static final String NAME = "count";
   public static final String USAGE = "-" + NAME + "[-q] <path>";
   public static final String DESCRIPTION = CommandUtils.formatDescription(USAGE, 
@@ -37,41 +43,42 @@ public class Count extends Command {
       "QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA ",
       "      DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME");
   
-  private boolean qOption;
+  private boolean showQuotas;
 
+  /** Constructor */
+  public Count() {
+    setCommandName(NAME);
+  }
+  
   /** Constructor
-   * 
+   * @deprecated invoke via {@link FsShell}
    * @param cmd the count command
    * @param pos the starting index of the arguments 
+   * @param conf configuration
    */
+  @Deprecated
   public Count(String[] cmd, int pos, Configuration conf) {
     super(conf);
-    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); 
+    setCommandName(NAME);
+    LinkedList<String> parameters = new LinkedList<String>(Arrays.asList(cmd));
+    parameters.subList(0, pos).clear();
+    processOptions(parameters);
+    this.args = parameters.toArray(new String[0]);
   }
 
   @Override
-  public String getCommandName() {
-    return NAME;
+  protected void processOptions(LinkedList<String> args) {
+    CommandFormat cf = new CommandFormat(NAME, 1, Integer.MAX_VALUE, "q");
+    cf.parse(args);
+    if (args.isEmpty()) { // default path is the current working directory
+      args.add(".");
+    }
+    showQuotas = cf.getOpt("q");
   }
 
   @Override
-  protected void run(Path path) throws IOException {
-    FileSystem fs = path.getFileSystem(getConf());
-    System.out.println(fs.getContentSummary(path).toString(qOption) + path);
+  protected void processPath(PathData src) throws IOException {
+    ContentSummary summary = src.fs.getContentSummary(src.path);
+    out.println(summary.toString(showQuotas) + src.path);
   }
 }

Added: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java?rev=1090039&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java Thu Apr  7 21:59:37 2011
@@ -0,0 +1,53 @@
+/**
+ * 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.fs.shell;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Base class for all "hadoop fs" commands
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+// this class may not look useful now, but it's a placeholder for future
+// functionality to act as a registry for fs commands.  currently it's being
+// used to implement unnecessary abstract methods in the base class
+
+abstract public class FsCommand extends Command {
+  protected FsCommand() {}
+  
+  protected FsCommand(Configuration conf) {
+    super(conf);
+  }
+
+  public String getCommandName() {
+    return name.startsWith("-") ? name.substring(1) : name; 
+  }
+  
+  protected void run(Path path) throws IOException {
+    throw new RuntimeException("not supposed to get here");
+  }
+}

Added: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java?rev=1090039&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java Thu Apr  7 21:59:37 2011
@@ -0,0 +1,123 @@
+/**
+ * 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.fs.shell;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Encapsulates a Path (path), its FileStatus (stat), and its FileSystem (fs).
+ * The stat field will be null if the path does not exist.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+public class PathData {
+  public final Path path;
+  public FileStatus stat;
+  public final FileSystem fs;
+  public boolean exists;
+
+  /**
+   * Creates an object to wrap the given parameters as fields.
+   * @param theFs the FileSystem
+   * @param thePath a Path
+   * @param theStat the FileStatus (may be null if the path doesn't exist)
+   */
+  public PathData(FileSystem theFs, Path thePath, FileStatus theStat) {
+    path = thePath;
+    stat = theStat;
+    fs = theFs;
+    exists = (stat != null);
+  }
+
+  /**
+   * Convenience ctor that looks up the file status for a path.  If the path
+   * doesn't exist, then the status will be null
+   * @param fs the FileSystem for the path
+   * @param path the pathname to lookup 
+   * @throws IOException if anything goes wrong
+   */
+  public PathData(FileSystem fs, Path path) throws IOException {
+    this(fs, path, getStat(fs, path));
+  }
+
+  // need a static method for the ctor above
+  private static FileStatus getStat(FileSystem fs, Path path)
+  throws IOException {  
+    FileStatus status = null;
+    try {
+      status = fs.getFileStatus(path);
+    } catch (FileNotFoundException e) {} // ignore FNF
+    return status;
+  }
+
+  /**
+   * Convenience ctor that extracts the path from the given file status
+   * @param fs the FileSystem for the FileStatus
+   * @param stat the FileStatus 
+   */
+  public PathData(FileSystem fs, FileStatus stat) {
+    this(fs, stat.getPath(), stat);
+  }
+  
+  /**
+   * Updates the paths's file status
+   * @return the updated FileStatus
+   * @throws IOException if anything goes wrong...
+   */
+  public FileStatus refreshStatus() throws IOException {
+    stat = fs.getFileStatus(path);
+    return stat;
+  }
+  
+  /**
+   * Returns a list of PathData objects of the items contained in the given
+   * directory.
+   * @return list of PathData objects for its children
+   * @throws IOException if anything else goes wrong...
+   */
+  public PathData[] getDirectoryContents() throws IOException {
+    if (!stat.isDirectory()) {
+      throw new IOException(path + ": Not a directory");
+    }
+
+    FileStatus[] stats = fs.listStatus(path);
+    PathData[] items = new PathData[stats.length];
+    for (int i=0; i < stats.length; i++) {
+      items[i] = new PathData(fs, stats[i]);
+    }
+    return items;
+  }
+
+  /**
+   * Returns the printable version of the path that is just the
+   * filesystem path instead of the full uri
+   * @return String of the path
+   */
+  public String toString() {
+    return path.toString();
+  }
+}

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java?rev=1090039&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java Thu Apr  7 21:59:37 2011
@@ -0,0 +1,47 @@
+/**
+ * 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.fs.shell;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestPathData {
+  protected static Configuration conf;
+  protected static FileSystem fs;
+  protected static Path dir;
+  
+  @BeforeClass
+  public static void initialize() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf); 
+    dir = new Path("/tmp");
+  }
+
+  @Test
+  public void testWithPath() throws Exception {
+    PathData item = new PathData(fs, dir);
+    assertEquals(fs, item.fs);
+    assertEquals(dir, item.path);
+    assertTrue(item.stat != null);
+    assertTrue(item.stat.isDirectory());
+  }
+}