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/05/04 23:34:15 UTC

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

Author: szetszwo
Date: Wed May  4 21:34:15 2011
New Revision: 1099612

URL: http://svn.apache.org/viewvc?rev=1099612&view=rev
Log:
HADOOP-7236. Refactor the mkdir command to conform to new FsCommand class.  Contributed by Daryn Sharp

Added:
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.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/FsCommand.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/testConf.xml
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed May  4 21:34:15 2011
@@ -108,6 +108,9 @@ Trunk (unreleased changes)
     HADOOP-7227. Remove protocol version check at proxy creation in Hadoop
     RPC. (jitendra)
 
+    HADOOP-7236. Refactor the mkdir command to conform to new FsCommand 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=1099612&r1=1099611&r2=1099612&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 Wed May  4 21:34:15 2011
@@ -736,30 +736,6 @@ public class FsShell extends Configured 
   }
 
   /**
-   * Create the given dir
-   */
-  void mkdir(String src) throws IOException {
-    Path f = new Path(src);
-    FileSystem srcFs = f.getFileSystem(getConf());
-    FileStatus fstatus = null;
-    try {
-      fstatus = srcFs.getFileStatus(f);
-      if (fstatus.isDirectory()) {
-        throw new IOException("cannot create directory " 
-            + src + ": File exists");
-      }
-      else {
-        throw new IOException(src + " exists but " +
-            "is not a directory");
-      }
-    } catch(FileNotFoundException e) {
-        if (!srcFs.mkdirs(f)) {
-          throw new IOException("failed to create " + src);
-        }
-    }
-  }
-
-  /**
    * (Re)create zero-length file at the specified path.
    * This will be replaced by a more UNIX-like touch when files may be
    * modified.
@@ -1274,7 +1250,7 @@ public class FsShell extends Configured 
       GET_SHORT_USAGE + "\n\t" +
       "[-getmerge <src> <localdst> [addnl]] [-cat <src>]\n\t" +
       "[" + COPYTOLOCAL_SHORT_USAGE + "] [-moveToLocal <src> <localdst>]\n\t" +
-      "[-mkdir <path>] [-report] [" + SETREP_SHORT_USAGE + "]\n\t" +
+      "[-report] [" + SETREP_SHORT_USAGE + "]\n\t" +
       "[-touchz <path>] [-test -[ezd] <path>] [-stat [format] <path>]\n\t" +
       "[-text <path>]\n\t" +
       "[" + FsShellPermissions.CHMOD_USAGE + "]\n\t" +
@@ -1364,8 +1340,6 @@ public class FsShell extends Configured 
 
     String moveToLocal = "-moveToLocal <src> <localdst>:  Not implemented yet \n";
         
-    String mkdir = "-mkdir <path>: \tCreate a directory in specified location. \n";
-
     String setrep = SETREP_SHORT_USAGE
       + ":  Set the replication level of a file. \n"
       + "\t\tThe -R flag requests a recursive change of replication level \n"
@@ -1440,8 +1414,6 @@ public class FsShell extends Configured 
       System.out.println(expunge);
     } else if ("rmr".equals(cmd)) {
       System.out.println(rmr);
-    } else if ("mkdir".equals(cmd)) {
-      System.out.println(mkdir);
     } else if ("mv".equals(cmd)) {
       System.out.println(mv);
     } else if ("cp".equals(cmd)) {
@@ -1506,7 +1478,6 @@ public class FsShell extends Configured 
       System.out.println(cat);
       System.out.println(copyToLocal);
       System.out.println(moveToLocal);
-      System.out.println(mkdir);
       System.out.println(setrep);
       System.out.println(touchz);
       System.out.println(test);
@@ -1566,8 +1537,6 @@ public class FsShell extends Configured 
         //
         if ("-cat".equals(cmd)) {
           cat(argv[i], true);
-        } else if ("-mkdir".equals(cmd)) {
-          mkdir(argv[i]);
         } else if ("-rm".equals(cmd)) {
           delete(argv[i], false, rmSkipTrash);
         } else if ("-rmr".equals(cmd)) {
@@ -1632,7 +1601,7 @@ public class FsShell extends Configured 
       System.err.println("Usage: java FsShell" + 
                          " [-D <[property=value>]");
     } else if ("-du".equals(cmd) || "-dus".equals(cmd) ||
-               "-touchz".equals(cmd) || "-mkdir".equals(cmd) ||
+               "-touchz".equals(cmd) ||
                "-text".equals(cmd)) {
       System.err.println("Usage: java FsShell" + 
                          " [" + cmd + " <path>]");
@@ -1686,7 +1655,6 @@ public class FsShell extends Configured 
       System.err.println("           [-text <src>]");
       System.err.println("           [" + COPYTOLOCAL_SHORT_USAGE + "]");
       System.err.println("           [-moveToLocal [-crc] <src> <localdst>]");
-      System.err.println("           [-mkdir <path>]");
       System.err.println("           [" + SETREP_SHORT_USAGE + "]");
       System.err.println("           [-touchz <path>]");
       System.err.println("           [-test -[ezd] <path>]");
@@ -1743,7 +1711,7 @@ public class FsShell extends Configured 
         return exitCode;
       }
     } else if ("-rm".equals(cmd) || "-rmr".equals(cmd) ||
-               "-cat".equals(cmd) || "-mkdir".equals(cmd) ||
+               "-cat".equals(cmd) ||
                "-touchz".equals(cmd) || "-stat".equals(cmd) ||
                "-text".equals(cmd)) {
       if (argv.length < 2) {
@@ -1829,8 +1797,6 @@ public class FsShell extends Configured 
         du(argv, i);
       } else if ("-dus".equals(cmd)) {
         dus(argv, i);
-      } else if ("-mkdir".equals(cmd)) {
-        exitCode = doall(cmd, argv, i);
       } else if ("-touchz".equals(cmd)) {
         exitCode = doall(cmd, argv, i);
       } else if ("-test".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=1099612&r1=1099611&r2=1099612&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 Wed May  4 21:34:15 2011
@@ -31,8 +31,6 @@ import org.apache.hadoop.classification.
 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.util.StringUtils;
 
@@ -104,8 +102,8 @@ abstract public class Command extends Co
     int exitCode = 0;
     for (String src : args) {
       try {
-        List<PathData> srcs = expandGlob(src);
-        for(PathData s : srcs) {
+        PathData[] srcs = PathData.expandAsGlob(src, getConf());
+        for (PathData s : srcs) {
           run(s.path);
         }
       } catch (IOException e) {
@@ -128,7 +126,7 @@ abstract public class Command extends Co
    *         \-> {@link #processPathArgument(PathData)}
    *             \-> {@link #processPaths(PathData, PathData...)}
    *                 \-> {@link #processPath(PathData)}*
-   *         \-> {@link #processNonexistentPathArgument(PathData)}
+   *         \-> {@link #processNonexistentPath(PathData)}
    * </pre>
    * Most commands will chose to implement just
    * {@link #processOptions(LinkedList)} and {@link #processPath(PathData)}
@@ -191,7 +189,12 @@ abstract public class Command extends Co
    * @throws IOException if anything goes wrong...
    */
   protected List<PathData> expandArgument(String arg) throws IOException {
-    return expandGlob(arg);
+    PathData[] items = PathData.expandAsGlob(arg, getConf());
+    if (items.length == 0) {
+      // it's a glob that failed to match
+      throw new FileNotFoundException(getFnfText(new Path(arg)));
+    }
+    return Arrays.asList(items);
   }
 
   /**
@@ -216,7 +219,7 @@ abstract public class Command extends Co
   /**
    * Processes a {@link PathData} item, calling
    * {@link #processPathArgument(PathData)} or
-   * {@link #processNonexistentPathArgument(PathData)} on each item.
+   * {@link #processNonexistentPath(PathData)} on each item.
    * @param arg {@link PathData} item to process
    * @throws IOException if anything goes wrong...
    */
@@ -224,7 +227,7 @@ abstract public class Command extends Co
     if (item.exists) {
       processPathArgument(item);
     } else {
-      processNonexistentPathArgument(item);
+      processNonexistentPath(item);
     }
   }
 
@@ -250,8 +253,7 @@ abstract public class Command extends Co
    *  @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 {
+  protected void processNonexistentPath(PathData item) throws IOException {
     // TODO: this should be more posix-like: ex. "No such file or directory"
     throw new FileNotFoundException(getFnfText(item.path));
   }
@@ -312,38 +314,6 @@ abstract public class Command extends Co
   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(getFnfText(path));
-    }
-    
-    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

Modified: 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=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/FsCommand.java Wed May  4 21:34:15 2011
@@ -44,6 +44,7 @@ abstract public class FsCommand extends 
   public static void registerCommands(CommandFactory factory) {
     factory.registerCommands(Count.class);
     factory.registerCommands(Ls.class);
+    factory.registerCommands(Mkdir.class);
     factory.registerCommands(Tail.class);
   }
 

Added: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java?rev=1099612&view=auto
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java (added)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Mkdir.java Wed May  4 21:34:15 2011
@@ -0,0 +1,64 @@
+/**
+ * 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 java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Create the given dir
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+
+class Mkdir extends FsCommand {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Mkdir.class, "-mkdir");
+  }
+  
+  public static final String NAME = "mkdir";
+  public static final String USAGE = "<path> ...";
+  public static final String DESCRIPTION =
+    "Create a directory in specified location.";
+
+  @Override
+  protected void processOptions(LinkedList<String> args) {
+    CommandFormat cf = new CommandFormat(null, 1, Integer.MAX_VALUE);
+    cf.parse(args);
+  }
+
+  @Override
+  protected void processPath(PathData item) throws IOException {
+    if (item.stat.isDirectory()) {
+      throw new IOException("cannot create directory " + item + ": File exists");
+    } else {
+      throw new IOException(item + " exists but is not a directory");
+    }
+  }
+
+  @Override
+  protected void processNonexistentPath(PathData item) throws IOException {
+    if (!item.fs.mkdirs(item.path)) {
+      throw new IOException("failed to create " + item);
+    }
+  }
+}

Modified: 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=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/PathData.java Wed May  4 21:34:15 2011
@@ -23,6 +23,7 @@ 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,22 +36,38 @@ import org.apache.hadoop.fs.Path;
 @InterfaceStability.Evolving
 
 public class PathData {
+  protected String string = null;
   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)
+   * Creates an object to wrap the given parameters as fields.  The string
+   * used to create the path will be recorded since the Path object does not
+   * return exactly the same string used to initialize it
+   * @param pathString a string for a path
+   * @param conf the configuration file
+   * @throws IOException if anything goes wrong...
    */
-  public PathData(FileSystem theFs, Path thePath, FileStatus theStat) {
-    path = thePath;
-    stat = theStat;
-    fs = theFs;
-    exists = (stat != null);
+  public PathData(String pathString, Configuration conf) throws IOException {
+    this.string = pathString;
+    this.path = new Path(pathString);
+    this.fs = path.getFileSystem(conf);
+    setStat(getStat(fs, path));
+  }
+  
+  /**
+   * Creates an object to wrap the given parameters as fields. 
+   * @param fs the FileSystem
+   * @param path a Path
+   * @param stat the FileStatus (may be null if the path doesn't exist)
+   */
+  public PathData(FileSystem fs, Path path, FileStatus stat) {
+    this.string = path.toString();
+    this.path = path;
+    this.fs = fs;
+    setStat(stat);
   }
 
   /**
@@ -64,6 +81,23 @@ public class PathData {
     this(fs, path, getStat(fs, path));
   }
 
+  /**
+   * Creates an object to wrap the given parameters as fields.  The string
+   * used to create the path will be recorded since the Path object does not
+   * return exactly the same string used to initialize it.  If the FileStatus
+   * is not null, then its Path will be used to initialized the path, else
+   * the string of the path will be used.
+   * @param fs the FileSystem
+   * @param pathString a String of the path
+   * @param stat the FileStatus (may be null if the path doesn't exist)
+   */
+  public PathData(FileSystem fs, String pathString, FileStatus stat) {
+    this.string = pathString;
+    this.path = (stat != null) ? stat.getPath() : new Path(pathString);
+    this.fs = fs;
+    setStat(stat);
+  }
+
   // need a static method for the ctor above
   private static FileStatus getStat(FileSystem fs, Path path)
   throws IOException {  
@@ -73,6 +107,11 @@ public class PathData {
     } catch (FileNotFoundException e) {} // ignore FNF
     return status;
   }
+  
+  private void setStat(FileStatus theStat) {
+    stat = theStat;
+    exists = (stat != null);
+  }
 
   /**
    * Convenience ctor that extracts the path from the given file status
@@ -89,7 +128,7 @@ public class PathData {
    * @throws IOException if anything goes wrong...
    */
   public FileStatus refreshStatus() throws IOException {
-    stat = fs.getFileStatus(path);
+    setStat(fs.getFileStatus(path));
     return stat;
   }
   
@@ -113,11 +152,56 @@ public class PathData {
   }
 
   /**
-   * Returns the printable version of the path that is just the
-   * filesystem path instead of the full uri
+   * 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 pattern to expand as a glob
+   * @param conf the hadoop configuration
+   * @return list of {@link PathData} objects.  if the pattern is not a glob,
+   * and does not exist, the list will contain a single PathData with a null
+   * stat 
+   * @throws IOException anything else goes wrong...
+   */
+  public static PathData[] expandAsGlob(String pattern, Configuration conf)
+  throws IOException {
+    Path globPath = new Path(pattern);
+    FileSystem fs = globPath.getFileSystem(conf);    
+    FileStatus[] stats = fs.globStatus(globPath);
+    PathData[] items = null;
+    
+    if (stats == null) {
+      // not a glob & file not found, so add the path with a null stat
+      items = new PathData[]{ new PathData(fs, pattern, null) };
+    } else if (
+        // this is very ugly, but needed to avoid breaking hdfs tests...
+        // if a path has no authority, then the FileStatus from globStatus
+        // will add the "-fs" authority into the path, so we need to sub
+        // it back out to satisfy the tests
+        stats.length == 1
+        &&
+        stats[0].getPath().equals(fs.makeQualified(globPath)))
+    {
+      // if the fq path is identical to the pattern passed, use the pattern
+      // to initialize the string value
+      items = new PathData[]{ new PathData(fs, pattern, stats[0]) };
+    } else {
+      // convert stats to PathData
+      items = new PathData[stats.length];
+      int i=0;
+      for (FileStatus stat : stats) {
+        items[i++] = new PathData(fs, stat);
+      }
+    }
+    return items;
+  }
+
+  /**
+   * Returns the printable version of the path that is either the path
+   * as given on the commandline, or the full path
    * @return String of the path
    */
   public String toString() {
-    return path.toString();
+    return (string != null) ? string : path.toString();
   }
 }

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java?rev=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/shell/Tail.java Wed May  4 21:34:15 2011
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 
@@ -60,11 +59,8 @@ class Tail extends FsCommand {
   // TODO: HADOOP-7234 will add glob support; for now, be backwards compat
   @Override
   protected List<PathData> expandArgument(String arg) throws IOException {
-    Path path = new Path(arg);
-    FileSystem fs = path.getFileSystem(getConf());
-    
     List<PathData> items = new LinkedList<PathData>();
-    items.add(new PathData(fs, path));
+    items.add(new PathData(arg, getConf()));
     return items;
   }
       

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/testConf.xml?rev=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/testConf.xml (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/cli/testConf.xml Wed May  4 21:34:15 2011
@@ -503,7 +503,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-mkdir &lt;path&gt;:( |\t)*Create a directory in specified location.( )*</expected-output>
+          <expected-output>^-mkdir &lt;path&gt; \.\.\.:( |\t)*Create a directory in specified location.( )*</expected-output>
         </comparator>
       </comparators>
     </test>

Modified: 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=1099612&r1=1099611&r2=1099612&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/shell/TestPathData.java Wed May  4 21:34:15 2011
@@ -27,19 +27,48 @@ import static org.junit.Assert.*;
 public class TestPathData {
   protected static Configuration conf;
   protected static FileSystem fs;
+  protected static String dirString;
   protected static Path dir;
-  
+  protected static PathData item;
+
   @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);
+  public void testWithFsAndPath() throws Exception {
+    dirString = "/tmp";
+    dir = new Path(dirString);
+    item = new PathData(fs, dir);
+    checkPathData();
+  }
+
+  @Test
+  public void testWithStringAndConf() throws Exception {
+    dirString = "/tmp";
+    dir = new Path(dirString);
+    item = new PathData(dirString, conf);
+    checkPathData();
+  }
+
+  @Test
+  public void testWithStringAndConfForBuggyPath() throws Exception {
+    dirString = "file:///tmp";
+    dir = new Path(dirString);
+    item = new PathData(dirString, conf);
+    // this may fail some day if Path is fixed to not crunch the uri
+    // if the authority is null, however we need to test that the PathData
+    // toString() returns the given string, while Path toString() does
+    // the crunching
+    assertEquals("file:/tmp", dir.toString());
+    checkPathData();
+  }
+
+  public void checkPathData() throws Exception {
     assertEquals(fs, item.fs);
+    assertEquals(dirString, item.toString());
     assertEquals(dir, item.path);
     assertTrue(item.stat != null);
     assertTrue(item.stat.isDirectory());