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/10/28 03:17:37 UTC

svn commit: r1190111 - in /hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common: ./ src/main/java/ src/main/java/org/apache/hadoop/fs/shell/ src/test/java/org/apache/hadoop/cli/ src/test/java/org/apache/hadoop/fs/shell/

Author: szetszwo
Date: Fri Oct 28 01:17:37 2011
New Revision: 1190111

URL: http://svn.apache.org/viewvc?rev=1190111&view=rev
Log:
svn merge -c 1190109 from trunk for HADOOP-7360.

Modified:
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/   (props changed)
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
    hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java

Propchange: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Oct 28 01:17:37 2011
@@ -1 +1 @@
-/hadoop/common/trunk/hadoop-common-project/hadoop-common:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1164339,1166009,1166402,1167383,1170379,1170459,1171297,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1179869,1182641,1183132,1189357,1189932,1189982
+/hadoop/common/trunk/hadoop-common-project/hadoop-common:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1164339,1166009,1166402,1167383,1170379,1170459,1171297,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1179869,1182641,1183132,1189357,1189932,1189982,1190109

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt Fri Oct 28 01:17:37 2011
@@ -420,6 +420,9 @@ Release 0.23.0 - Unreleased
 
     MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
 
+    HADOOP-7360. Preserve relative paths that do not contain globs in FsShell.
+    (Daryn Sharp and Kihwal Lee via szetszwo)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

Propchange: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Oct 28 01:17:37 2011
@@ -1,5 +1,5 @@
 /hadoop/common/branches/yahoo-merge/CHANGES.txt:1079157,1079163-1079164,1079167
-/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1166009,1166402,1167383,1169986,1170046,1170379,1170459,1171297,1171894,1171909,1172186,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1179869,1182641,1183132,1189357,1189932,1189982
+/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1166009,1166402,1167383,1169986,1170046,1170379,1170459,1171297,1171894,1171909,1172186,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1179869,1182641,1183132,1189357,1189932,1189982,1190109
 /hadoop/core/branches/branch-0.18/CHANGES.txt:727226
 /hadoop/core/branches/branch-0.19/CHANGES.txt:713112
 /hadoop/core/trunk/CHANGES.txt:776175-785643,785929-786278

Propchange: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Fri Oct 28 01:17:37 2011
@@ -1,3 +1,3 @@
-/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1164339,1166402,1167383,1170379,1170459,1171297,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1182641,1183132,1189932,1189982
+/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:1161777,1161781,1162008,1162188,1162421,1162491,1162499,1162613,1162928,1162954,1162979,1163050,1163069,1163456,1163465,1163490,1163768,1163852,1163858,1164255,1164301,1164339,1166402,1167383,1170379,1170459,1171297,1172916,1173402,1176550,1176986,1177035,1177487,1177531,1177859,1177864,1182641,1183132,1189932,1189982,1190109
 /hadoop/core/branches/branch-0.19/core/src/java:713112
 /hadoop/core/trunk/src/core:776175-785643,785929-786278

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java Fri Oct 28 01:17:37 2011
@@ -27,10 +27,8 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.shell.PathExceptions.PathExistsException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathOperationException;
@@ -131,12 +129,10 @@ class CopyCommands {  
     private static final String COPYTOLOCAL_PREFIX = "_copyToLocal_";
     private boolean copyCrc;
     private boolean verifyChecksum;
-    private LocalFileSystem localFs;
 
     @Override
     protected void processOptions(LinkedList<String> args)
     throws IOException {
-      localFs = FileSystem.getLocal(getConf());
       CommandFormat cf = new CommandFormat(
           1, Integer.MAX_VALUE, "crc", "ignoreCrc");
       cf.parse(args);
@@ -157,16 +153,15 @@ class CopyCommands {  
         copyCrc = false;
       }      
 
-      File targetFile = localFs.pathToFile(target.path);
       if (src.stat.isFile()) {
         // copy the file and maybe its crc
-        copyFileToLocal(src, target.path);
+        copyFileToLocal(src, target);
         if (copyCrc) {
-          copyCrcToLocal(src, target.path);
+          copyFileToLocal(src.getChecksumFile(), target.getChecksumFile());
         }
       } else if (src.stat.isDirectory()) {
         // create the remote directory structure locally
-        if (!targetFile.mkdirs()) {
+        if (!target.toFile().mkdirs()) {
           throw new PathIOException(target.toString());
         }
       } else {
@@ -174,9 +169,9 @@ class CopyCommands {  
       }
     }
 
-    private void copyFileToLocal(PathData src, Path target)
+    private void copyFileToLocal(PathData src, PathData target)
     throws IOException {
-      File targetFile = localFs.pathToFile(target);
+      File targetFile = target.toFile();
       File tmpFile = FileUtil.createLocalTempFile(
           targetFile, COPYTOLOCAL_PREFIX, true);
       // too bad we can't tell exactly why it failed...
@@ -195,14 +190,6 @@ class CopyCommands {  
         throw e;
       }
     }
-
-    private void copyCrcToLocal(PathData src, Path target)
-    throws IOException {
-      ChecksumFileSystem srcFs = (ChecksumFileSystem)src.fs;
-      Path srcPath = srcFs.getChecksumFile(src.path);
-      src = new PathData(srcFs.getRawFileSystem(), srcPath);
-      copyFileToLocal(src, localFs.getChecksumFile(target));    
-    }
   }
 
   /**

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Count.java Fri Oct 28 01:17:37 2011
@@ -81,6 +81,6 @@ public class Count extends FsCommand {
   @Override
   protected void processPath(PathData src) throws IOException {
     ContentSummary summary = src.fs.getContentSummary(src.path);
-    out.println(summary.toString(showQuotas) + src.path);
+    out.println(summary.toString(showQuotas) + src);
   }
 }

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java Fri Oct 28 01:17:37 2011
@@ -113,7 +113,7 @@ class Ls extends FsCommand {
         stat.getGroup(),
         formatSize(stat.getLen()),
         dateFormat.format(new Date(stat.getModificationTime())),
-        item.path.toUri().getPath()
+        item
     );
     out.println(line);
   }

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java Fri Oct 28 01:17:37 2011
@@ -21,27 +21,34 @@ package org.apache.hadoop.fs.shell;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIOException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathIsDirectoryException;
 import org.apache.hadoop.fs.shell.PathExceptions.PathIsNotDirectoryException;
+import org.apache.hadoop.fs.shell.PathExceptions.PathNotFoundException;
 
 /**
  * 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
+@InterfaceStability.Unstable
 
 public class PathData {
-  protected String string = null;
+  protected final URI uri;
+  public final FileSystem fs;
   public final Path path;
   public FileStatus stat;
-  public final FileSystem fs;
   public boolean exists;
 
   /**
@@ -53,10 +60,7 @@ public class PathData {
    * @throws IOException if anything goes wrong...
    */
   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));
+    this(FileSystem.get(URI.create(pathString), conf), pathString);
   }
   
   /**
@@ -68,106 +72,130 @@ public class PathData {
    * @throws IOException if anything goes wrong...
    */
   public PathData(File localPath, Configuration conf) throws IOException {
-    this.string = localPath.toString();
-    this.path = new Path(this.string);
-    this.fs = FileSystem.getLocal(conf);
-    setStat(getStat(fs, path));
+    this(FileSystem.getLocal(conf), localPath.toString());
   }
 
   /**
-   * 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);
-  }
-
-  /**
-   * Convenience ctor that looks up the file status for a path.  If the path
+   * 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 
+   * @param pathString a string for a path 
    * @throws IOException if anything goes wrong
    */
-  public PathData(FileSystem fs, Path path) throws IOException {
-    this(fs, path, getStat(fs, path));
+  private PathData(FileSystem fs, String pathString) throws IOException {
+    this(fs, pathString, lookupStat(fs, pathString, true));
   }
 
   /**
    * 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.
+   * return exactly the same string used to initialize it.
    * @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);
+  private PathData(FileSystem fs, String pathString, FileStatus stat)
+  throws IOException {
     this.fs = fs;
+    this.uri = stringToUri(pathString);
+    this.path = fs.makeQualified(new Path(uri));
     setStat(stat);
   }
 
   // need a static method for the ctor above
-  private static FileStatus getStat(FileSystem fs, Path path)
-  throws IOException {  
+  /**
+   * Get the FileStatus info
+   * @param ignoreFNF if true, stat will be null if the path doesn't exist
+   * @return FileStatus for the given path
+   * @throws IOException if anything goes wrong
+   */
+  private static
+  FileStatus lookupStat(FileSystem fs, String pathString, boolean ignoreFNF)
+  throws IOException {
     FileStatus status = null;
     try {
-      status = fs.getFileStatus(path);
-    } catch (FileNotFoundException e) {} // ignore FNF
+      status = fs.getFileStatus(new Path(pathString));
+    } catch (FileNotFoundException e) {
+      if (!ignoreFNF) throw new PathNotFoundException(pathString);
+    }
+    // TODO: should consider wrapping other exceptions into Path*Exceptions
     return status;
   }
   
-  private void setStat(FileStatus theStat) {
-    stat = theStat;
+  private void setStat(FileStatus stat) {
+    this.stat = stat;
     exists = (stat != null);
   }
 
   /**
-   * 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 {
-    setStat(fs.getFileStatus(path));
-    return stat;
+    FileStatus status = null;
+    try {
+      status = lookupStat(fs, toString(), false);
+    } finally {
+      // always set the status.  the caller must get the correct result
+      // if it catches the exception and later interrogates the status
+      setStat(status);
+    }
+    return status;
+  }
+
+  protected enum FileTypeRequirement {
+    SHOULD_NOT_BE_DIRECTORY, SHOULD_BE_DIRECTORY
+  };
+
+  /**
+   * Ensure that the file exists and if it is or is not a directory
+   * @param typeRequirement Set it to the desired requirement.
+   * @throws PathIOException if file doesn't exist or the type does not match
+   * what was specified in typeRequirement.
+   */
+  private void checkIfExists(FileTypeRequirement typeRequirement) 
+  throws PathIOException {
+    if (!exists) {
+      throw new PathNotFoundException(toString());      
+    }
+
+    if ((typeRequirement == FileTypeRequirement.SHOULD_BE_DIRECTORY)
+       && !stat.isDirectory()) {
+      throw new PathIsNotDirectoryException(toString());
+    } else if ((typeRequirement == FileTypeRequirement.SHOULD_NOT_BE_DIRECTORY)
+              && stat.isDirectory()) {
+      throw new PathIsDirectoryException(toString());
+    }
   }
   
   /**
+   * Return the corresponding crc data for a file.  Avoids exposing the fs
+   * contortions to the caller.  
+   * @return PathData of the crc file
+   * @throws IOException is anything goes wrong
+   */
+  public PathData getChecksumFile() throws IOException {
+    checkIfExists(FileTypeRequirement.SHOULD_NOT_BE_DIRECTORY);
+    ChecksumFileSystem srcFs = (ChecksumFileSystem)fs;
+    Path srcPath = srcFs.getChecksumFile(path);
+    return new PathData(srcFs.getRawFileSystem(), srcPath.toString());
+  }
+
+  /**
    * 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 PathIsNotDirectoryException(string);
-    }
-
+    checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY);
     FileStatus[] stats = fs.listStatus(path);
     PathData[] items = new PathData[stats.length];
     for (int i=0; i < stats.length; i++) {
       // preserve relative paths
-      String basename = stats[i].getPath().getName();
-      String parent = string;
-      if (!parent.endsWith(Path.SEPARATOR)) parent += Path.SEPARATOR;
-      items[i] = new PathData(fs, parent + basename, stats[i]);
+      String child = getStringForChildPath(stats[i].getPath());
+      items[i] = new PathData(fs, child, stats[i]);
     }
     return items;
   }
@@ -179,13 +207,31 @@ public class PathData {
    * @throws IOException if this object does not exist or is not a directory
    */
   public PathData getPathDataForChild(PathData child) throws IOException {
-    if (!stat.isDirectory()) {
-      throw new PathIsNotDirectoryException(string);
-    }
-    return new PathData(fs, new Path(path, child.path.getName()));
+    checkIfExists(FileTypeRequirement.SHOULD_BE_DIRECTORY);
+    return new PathData(fs, getStringForChildPath(child.path));
   }
 
   /**
+   * Given a child of this directory, use the directory's path and the child's
+   * basename to construct the string to the child.  This preserves relative
+   * paths since Path will fully qualify.
+   * @param child a path contained within this directory
+   * @return String of the path relative to this directory
+   */
+  private String getStringForChildPath(Path childPath) {
+    String basename = childPath.getName();
+    if (Path.CUR_DIR.equals(toString())) {
+      return basename;
+    }
+    // check getPath() so scheme slashes aren't considered part of the path
+    String separator = uri.getPath().endsWith(Path.SEPARATOR)
+        ? "" : Path.SEPARATOR;
+    return uri + separator + basename;
+  }
+  
+  protected enum PathType { HAS_SCHEME, SCHEMELESS_ABSOLUTE, RELATIVE };
+  
+  /**
    * 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
@@ -207,35 +253,184 @@ public class PathData {
     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 {
+      // figure out what type of glob path was given, will convert globbed
+      // paths to match the type to preserve relativity
+      PathType globType;
+      URI globUri = globPath.toUri();
+      if (globUri.getScheme() != null) {
+        globType = PathType.HAS_SCHEME;
+      } else if (new File(globUri.getPath()).isAbsolute()) {
+        globType = PathType.SCHEMELESS_ABSOLUTE;
+      } else {
+        globType = PathType.RELATIVE;
+      }
+
       // convert stats to PathData
       items = new PathData[stats.length];
       int i=0;
       for (FileStatus stat : stats) {
-        items[i++] = new PathData(fs, stat);
+        URI matchUri = stat.getPath().toUri();
+        String globMatch = null;
+        switch (globType) {
+          case HAS_SCHEME: // use as-is, but remove authority if necessary
+            if (globUri.getAuthority() == null) {
+              matchUri = removeAuthority(matchUri);
+            }
+            globMatch = matchUri.toString();
+            break;
+          case SCHEMELESS_ABSOLUTE: // take just the uri's path
+            globMatch = matchUri.getPath();
+            break;
+          case RELATIVE: // make it relative to the current working dir
+            URI cwdUri = fs.getWorkingDirectory().toUri();
+            globMatch = relativize(cwdUri, matchUri, stat.isDirectory());
+            break;
+        }
+        items[i++] = new PathData(fs, globMatch, stat);
       }
     }
     return items;
   }
 
+  private static URI removeAuthority(URI uri) {
+    try {
+      uri = new URI(
+          uri.getScheme(), "",
+          uri.getPath(), uri.getQuery(), uri.getFragment()
+      );
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e.getLocalizedMessage());
+    }
+    return uri;
+  }
+  
+  private static String relativize(URI cwdUri, URI srcUri, boolean isDir) {
+    String uriPath = srcUri.getPath();
+    String cwdPath = cwdUri.getPath();
+    if (cwdPath.equals(uriPath)) {
+      return Path.CUR_DIR;
+    }
+
+    // find common ancestor
+    int lastSep = findLongestDirPrefix(cwdPath, uriPath, isDir);
+    
+    StringBuilder relPath = new StringBuilder();    
+    // take the remaining path fragment after the ancestor
+    if (lastSep < uriPath.length()) {
+      relPath.append(uriPath.substring(lastSep+1));
+    }
+
+    // if cwd has a path fragment after the ancestor, convert them to ".."
+    if (lastSep < cwdPath.length()) {
+      while (lastSep != -1) {
+        if (relPath.length() != 0) relPath.insert(0, Path.SEPARATOR);
+        relPath.insert(0, "..");
+        lastSep = cwdPath.indexOf(Path.SEPARATOR, lastSep+1);
+      }
+    }
+    return relPath.toString();
+  }
+
+  private static int findLongestDirPrefix(String cwd, String path, boolean isDir) {
+    // add the path separator to dirs to simplify finding the longest match
+    if (!cwd.endsWith(Path.SEPARATOR)) {
+      cwd += Path.SEPARATOR;
+    }
+    if (isDir && !path.endsWith(Path.SEPARATOR)) {
+      path += Path.SEPARATOR;
+    }
+
+    // find longest directory prefix 
+    int len = Math.min(cwd.length(), path.length());
+    int lastSep = -1;
+    for (int i=0; i < len; i++) {
+      if (cwd.charAt(i) != path.charAt(i)) break;
+      if (cwd.charAt(i) == Path.SEPARATOR_CHAR) lastSep = i;
+    }
+    return lastSep;
+  }
+  
   /**
    * 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 (string != null) ? string : path.toString();
+    String scheme = uri.getScheme();
+    // No interpretation of symbols. Just decode % escaped chars.
+    String decodedRemainder = uri.getSchemeSpecificPart();
+
+    if (scheme == null) {
+      return decodedRemainder;
+    } else {
+      StringBuilder buffer = new StringBuilder();
+      buffer.append(scheme);
+      buffer.append(":");
+      buffer.append(decodedRemainder);
+      return buffer.toString();
+    }
+  }
+  
+  /**
+   * Get the path to a local file
+   * @return File representing the local path
+   * @throws IllegalArgumentException if this.fs is not the LocalFileSystem
+   */
+  public File toFile() {
+    if (!(fs instanceof LocalFileSystem)) {
+       throw new IllegalArgumentException("Not a local path: " + path);
+    }
+    return ((LocalFileSystem)fs).pathToFile(path);
+  }
+
+  /** Construct a URI from a String with unescaped special characters
+   *  that have non-standard sematics. e.g. /, ?, #. A custom parsing
+   *  is needed to prevent misbihaviors.
+   *  @param pathString The input path in string form
+   *  @return URI
+   */
+  private static URI stringToUri(String pathString) {
+    // We can't use 'new URI(String)' directly. Since it doesn't do quoting
+    // internally, the internal parser may fail or break the string at wrong
+    // places. Use of multi-argument ctors will quote those chars for us,
+    // but we need to do our own parsing and assembly.
+    
+    // parse uri components
+    String scheme = null;
+    String authority = null;
+
+    int start = 0;
+
+    // parse uri scheme, if any
+    int colon = pathString.indexOf(':');
+    int slash = pathString.indexOf('/');
+    if (colon > 0 && (slash == colon +1)) {
+      // has a non zero-length scheme
+      scheme = pathString.substring(0, colon);
+      start = colon + 1;
+    }
+
+    // parse uri authority, if any
+    if (pathString.startsWith("//", start) &&
+        (pathString.length()-start > 2)) {
+      start += 2;
+      int nextSlash = pathString.indexOf('/', start);
+      int authEnd = nextSlash > 0 ? nextSlash : pathString.length();
+      authority = pathString.substring(start, authEnd);
+      start = authEnd;
+    }
+
+    // uri path is the rest of the string. ? or # are not interpreated,
+    // but any occurrence of them will be quoted by the URI ctor.
+    String path = pathString.substring(start, pathString.length());
+
+    // Construct the URI
+    try {
+      return new URI(scheme, authority, path, null, null);
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException(e);
+    }
   }
+
 }

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/CLITestHelper.java Fri Oct 28 01:17:37 2011
@@ -180,7 +180,7 @@ public class CLITestHelper {
           LOG.info("         Comparision result:   [" + 
                    (resultBoolean ? "pass" : "fail") + "]");
           LOG.info("            Expected output:   [" + 
-                   cd.getExpectedOutput() + "]");
+                   expandCommand(cd.getExpectedOutput()) + "]");
           LOG.info("              Actual output:   [" + 
                    cd.getActualOutput() + "]");
         }
@@ -290,7 +290,7 @@ public class CLITestHelper {
           comparatorType);
         ComparatorBase comp = (ComparatorBase) comparatorClass.newInstance();
         compareOutput = comp.compare(cmdResult.getCommandOutput(), 
-          compdata.getExpectedOutput());
+          expandCommand(compdata.getExpectedOutput()));
       } catch (Exception e) {
         LOG.info("Error in instantiating the comparator" + e);
       }

Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java?rev=1190111&r1=1190110&r2=1190111&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java Fri Oct 28 01:17:37 2011
@@ -17,80 +17,180 @@
  */
 package org.apache.hadoop.fs.shell;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Arrays;
+
 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 String dirString;
-  protected static Path dir;
+  protected static Path testDir;
   protected static PathData item;
-
+  
+  protected static String[] d1Paths =
+    new String[] { "d1/f1", "d1/f1.1", "d1/f2" };
+  protected static String[] d2Paths =
+    new String[] { "d2/f3" };
+        
   @BeforeClass
   public static void initialize() throws Exception {
     conf = new Configuration();
-    fs = FileSystem.getLocal(conf); 
+    fs = FileSystem.getLocal(conf);
+    testDir = new Path(
+        System.getProperty("test.build.data", "build/test/data") + "/testPD"
+    );
+    // don't want scheme on the path, just an absolute path
+    testDir = new Path(fs.makeQualified(testDir).toUri().getPath());
+    FileSystem.setDefaultUri(conf, fs.getUri());    
+    fs.setWorkingDirectory(testDir);
+    fs.mkdirs(new Path("d1"));
+    fs.createNewFile(new Path("d1", "f1"));
+    fs.createNewFile(new Path("d1", "f1.1"));
+    fs.createNewFile(new Path("d1", "f2"));
+    fs.mkdirs(new Path("d2"));
+    fs.create(new Path("d2","f3"));
   }
 
   @Test
-  public void testWithFsAndPath() throws Exception {
-    dirString = "/tmp";
-    dir = new Path(dirString);
-    item = new PathData(fs, dir);
+  public void testWithDirStringAndConf() throws Exception {
+    dirString = "d1";
+    item = new PathData(dirString, conf);
     checkPathData();
-  }
 
-  @Test
-  public void testWithStringAndConf() throws Exception {
-    dirString = "/tmp";
-    dir = new Path(dirString);
+    // properly implementing symlink support in various commands will require
+    // trailing slashes to be retained
+    dirString = "d1/";
     item = new PathData(dirString, conf);
     checkPathData();
   }
 
   @Test
   public void testUnqualifiedUriContents() throws Exception {
-    dirString = "/tmp";
+    dirString = "d1";
     item = new PathData(dirString, conf);
     PathData[] items = item.getDirectoryContents();
-    for (PathData item : items) {
-      assertTrue(item.toString().startsWith(dirString));
-    }
+    assertEquals(
+        sortedString("d1/f1", "d1/f1.1", "d1/f2"),
+        sortedString(items)
+    );
   }
 
   @Test
   public void testQualifiedUriContents() throws Exception {
-    dirString = "file:/tmp";
+    dirString = fs.makeQualified(new Path("d1")).toString();
     item = new PathData(dirString, conf);
     PathData[] items = item.getDirectoryContents();
-    for (PathData item : items) {
-      assertTrue(item.toString().startsWith(dirString));
-    }
+    assertEquals(
+        sortedString(dirString+"/f1", dirString+"/f1.1", dirString+"/f2"),
+        sortedString(items)
+    );
+  }
+
+  @Test
+  public void testCwdContents() throws Exception {
+    dirString = Path.CUR_DIR;
+    item = new PathData(dirString, conf);
+    PathData[] items = item.getDirectoryContents();
+    assertEquals(
+        sortedString("d1", "d2"),
+        sortedString(items)
+    );
+  }
+
+
+	@Test
+	public void testToFile() throws Exception {
+    item = new PathData(".", conf);
+    assertEquals(new File(testDir.toString()), item.toFile());
+	  item = new PathData("d1/f1", conf);
+	  assertEquals(new File(testDir+"/d1/f1"), item.toFile());
+    item = new PathData(testDir+"/d1/f1", conf);
+    assertEquals(new File(testDir+"/d1/f1"), item.toFile());
+	}
+	
+  @Test
+  public void testAbsoluteGlob() throws Exception {
+    PathData[] items = PathData.expandAsGlob(testDir+"/d1/f1*", conf);
+    assertEquals(
+        sortedString(testDir+"/d1/f1", testDir+"/d1/f1.1"),
+        sortedString(items)
+    );
+  }
+
+  @Test
+  public void testRelativeGlob() throws Exception {
+    PathData[] items = PathData.expandAsGlob("d1/f1*", conf);
+    assertEquals(
+        sortedString("d1/f1", "d1/f1.1"),
+        sortedString(items)
+    );
+  }
+
+  @Test
+  public void testRelativeGlobBack() throws Exception {
+    fs.setWorkingDirectory(new Path("d1"));
+    PathData[] items = PathData.expandAsGlob("../d2/*", conf);
+    assertEquals(
+        sortedString("../d2/f3"),
+        sortedString(items)
+    );
   }
 
   @Test
   public void testWithStringAndConfForBuggyPath() throws Exception {
     dirString = "file:///tmp";
-    dir = new Path(dirString);
+    testDir = 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());
+    assertEquals("file:/tmp", testDir.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());
+    assertEquals("checking fs", fs, item.fs);
+    assertEquals("checking string", dirString, item.toString());
+    assertEquals("checking path",
+        fs.makeQualified(new Path(item.toString())), item.path
+    );
+    assertTrue("checking exist", item.stat != null);
+    assertTrue("checking isDir", item.stat.isDirectory());
+  }
+  
+  /* junit does a lousy job of comparing arrays
+   * if the array lengths differ, it just says that w/o showing contents
+   * this sorts the paths, and builds a string of "i:<value>, ..." suitable
+   * for a string compare
+   */
+  private static String sortedString(Object ... list) {
+    String[] strings = new String[list.length];
+    for (int i=0; i < list.length; i++) {
+      strings[i] = String.valueOf(list[i]);
+    }
+    Arrays.sort(strings);
+    
+    StringBuilder result = new StringBuilder();
+    for (int i=0; i < strings.length; i++) {
+      if (result.length() > 0) {
+        result.append(", ");
+      }
+      result.append(i+":<"+strings[i]+">");
+    }
+    return result.toString();
+  }
+  
+  private static String sortedString(PathData ... items) {
+    return sortedString((Object[])items);
   }
 }