Browse Source

Fixed HADOOP-129. Replaced uses of java.io.File in FileSystem API with a new class named Path. Also dfs.local.dir and mapred.local.dir may no longer be space-separated, but must now be comma-separated lists of directories.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@394984 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 years ago
parent
commit
b8a11ba2ec
51 changed files with 1254 additions and 905 deletions
  1. 13 0
      CHANGES.txt
  2. 2 2
      conf/hadoop-default.xml
  3. 8 7
      src/examples/org/apache/hadoop/examples/Grep.java
  4. 3 2
      src/examples/org/apache/hadoop/examples/WordCount.java
  5. 50 19
      src/java/org/apache/hadoop/conf/Configuration.java
  6. 1 1
      src/java/org/apache/hadoop/dfs/DFSClient.java
  7. 0 101
      src/java/org/apache/hadoop/dfs/DFSFile.java
  8. 3 2
      src/java/org/apache/hadoop/dfs/DFSFileInfo.java
  9. 25 25
      src/java/org/apache/hadoop/dfs/DFSShell.java
  10. 2 2
      src/java/org/apache/hadoop/dfs/DFSck.java
  11. 44 0
      src/java/org/apache/hadoop/dfs/DfsPath.java
  12. 48 143
      src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
  13. 1 1
      src/java/org/apache/hadoop/dfs/FSDataset.java
  14. 9 8
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  15. 5 5
      src/java/org/apache/hadoop/fs/FSDataInputStream.java
  16. 2 2
      src/java/org/apache/hadoop/fs/FSDataOutputStream.java
  17. 153 78
      src/java/org/apache/hadoop/fs/FileSystem.java
  18. 150 85
      src/java/org/apache/hadoop/fs/FileUtil.java
  19. 69 128
      src/java/org/apache/hadoop/fs/LocalFileSystem.java
  20. 178 0
      src/java/org/apache/hadoop/fs/Path.java
  21. 30 0
      src/java/org/apache/hadoop/fs/PathFilter.java
  22. 20 22
      src/java/org/apache/hadoop/io/MapFile.java
  23. 42 31
      src/java/org/apache/hadoop/io/SequenceFile.java
  24. 14 5
      src/java/org/apache/hadoop/mapred/FileSplit.java
  25. 25 19
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  26. 11 10
      src/java/org/apache/hadoop/mapred/JobClient.java
  27. 60 21
      src/java/org/apache/hadoop/mapred/JobConf.java
  28. 16 15
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  29. 2 2
      src/java/org/apache/hadoop/mapred/JobTracker.java
  30. 8 18
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  31. 13 4
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  32. 16 12
      src/java/org/apache/hadoop/mapred/MapOutputFile.java
  33. 1 1
      src/java/org/apache/hadoop/mapred/MapTask.java
  34. 2 2
      src/java/org/apache/hadoop/mapred/OutputFormatBase.java
  35. 7 8
      src/java/org/apache/hadoop/mapred/ReduceTask.java
  36. 6 6
      src/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java
  37. 13 6
      src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
  38. 1 1
      src/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java
  39. 1 1
      src/java/org/apache/hadoop/mapred/TaskInProgress.java
  40. 16 20
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  41. 1 1
      src/java/org/apache/hadoop/mapred/TextInputFormat.java
  42. 2 2
      src/java/org/apache/hadoop/mapred/TextOutputFormat.java
  43. 1 1
      src/test/org/apache/hadoop/dfs/ClusterTestDFS.java
  44. 12 11
      src/test/org/apache/hadoop/dfs/TestLocalDFS.java
  45. 16 17
      src/test/org/apache/hadoop/fs/TestFileSystem.java
  46. 10 15
      src/test/org/apache/hadoop/fs/TestLocalFileSystem.java
  47. 97 0
      src/test/org/apache/hadoop/fs/TestPath.java
  48. 22 20
      src/test/org/apache/hadoop/io/TestSequenceFile.java
  49. 16 16
      src/test/org/apache/hadoop/mapred/MapredLoadTest.java
  50. 4 4
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
  51. 3 3
      src/test/org/apache/hadoop/mapred/TestTextInputFormat.java

+ 13 - 0
CHANGES.txt

@@ -21,6 +21,19 @@ Trunk (unreleased)
 
  6. Fix HADOOP-128.  Improved DFS error handling. (Owen O'Malley via cutting)
 
+ 7. Fix HADOOP-129.  Replace uses of java.io.File with new class named
+    Path.  This fixes bugs where java.io.File methods were called
+    directly when FileSystem methods were desired, and reduces the
+    likelihood of such bugs in the future.  It also makes the handling
+    of pathnames more consistent between local and dfs FileSystems and
+    between Windows and Unix. java.io.File-based methods are still
+    available for back-compatibility, but are deprecated and will be
+    removed once 0.2 is released. (cutting)
+
+ 8. Change dfs.data.dir and mapred.local.dir to be comma-separated
+    lists of directories, no longer be space-separated. This fixes
+    several bugs on Windows. (cutting)
+
 
 Release 0.1.1 - 2006-04-08
 

+ 2 - 2
conf/hadoop-default.xml

@@ -84,7 +84,7 @@
   <name>dfs.data.dir</name>
   <value>/tmp/hadoop/dfs/data</value>
   <description>Determines where on the local filesystem an DFS data node
-  should store its blocks.  If this is a comma- or space-delimited
+  should store its blocks.  If this is a comma-delimited
   list of directories, then data will be stored in all named
   directories, typically on different devices.</description>
 </property>
@@ -164,7 +164,7 @@
   <name>mapred.local.dir</name>
   <value>/tmp/hadoop/mapred/local</value>
   <description>The local directory where MapReduce stores intermediate
-  data files.  May be a space- or comma- separated list of
+  data files.  May be a comma-separated list of
   directories on different devices in order to spread disk i/o.
   </description>
 </property>

+ 8 - 7
src/examples/org/apache/hadoop/examples/Grep.java

@@ -29,7 +29,8 @@ import org.apache.hadoop.io.LongWritable;
 
 import org.apache.hadoop.conf.Configuration;
 
-import java.io.File;
+import org.apache.hadoop.fs.Path;
+
 import java.util.Random;
 
 /* Extracts matching regexs from input files and counts them. */
@@ -44,14 +45,14 @@ public class Grep {
 
     Configuration defaults = new Configuration();
 
-    File tempDir =
-      new File("grep-temp-"+
+    Path tempDir =
+      new Path("grep-temp-"+
                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
 
     JobConf grepJob = new JobConf(defaults, Grep.class);
     grepJob.setJobName("grep-search");
 
-    grepJob.setInputDir(new File(args[0]));
+    grepJob.setInputPath(new Path(args[0]));
 
     grepJob.setMapperClass(RegexMapper.class);
     grepJob.set("mapred.mapper.regex", args[2]);
@@ -61,7 +62,7 @@ public class Grep {
     grepJob.setCombinerClass(LongSumReducer.class);
     grepJob.setReducerClass(LongSumReducer.class);
 
-    grepJob.setOutputDir(tempDir);
+    grepJob.setOutputPath(tempDir);
     grepJob.setOutputFormat(SequenceFileOutputFormat.class);
     grepJob.setOutputKeyClass(UTF8.class);
     grepJob.setOutputValueClass(LongWritable.class);
@@ -71,7 +72,7 @@ public class Grep {
     JobConf sortJob = new JobConf(defaults, Grep.class);
     sortJob.setJobName("grep-sort");
 
-    sortJob.setInputDir(tempDir);
+    sortJob.setInputPath(tempDir);
     sortJob.setInputFormat(SequenceFileInputFormat.class);
     sortJob.setInputKeyClass(UTF8.class);
     sortJob.setInputValueClass(LongWritable.class);
@@ -79,7 +80,7 @@ public class Grep {
     sortJob.setMapperClass(InverseMapper.class);
 
     sortJob.setNumReduceTasks(1);                 // write a single file
-    sortJob.setOutputDir(new File(args[1]));
+    sortJob.setOutputPath(new Path(args[1]));
     sortJob.setOutputKeyComparatorClass           // sort by decreasing freq
       (LongWritable.DecreasingComparator.class);
 

+ 3 - 2
src/examples/org/apache/hadoop/examples/WordCount.java

@@ -20,6 +20,7 @@ import java.io.*;
 import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
@@ -134,8 +135,8 @@ public class WordCount {
           other_args.size() + " instead of 2.");
       printUsage();
     }
-    conf.setInputDir(new File((String) other_args.get(0)));
-    conf.setOutputDir(new File((String) other_args.get(1)));
+    conf.setInputPath(new Path((String) other_args.get(0)));
+    conf.setOutputPath(new Path((String) other_args.get(1)));
     
     // Uncomment to run locally in a single process
     // conf.set("mapred.job.tracker", "local");

+ 50 - 19
src/java/org/apache/hadoop/conf/Configuration.java

@@ -30,14 +30,16 @@ import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
 
 import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 /** Provides access to configuration parameters.  Configurations are specified
  * by resources.  A resource contains a set of name/value pairs.
  *
- * <p>Each resources is named by either a String or by a File.  If named by a
+ * <p>Each resources is named by either a String or by a Path.  If named by a
  * String, then the classpath is examined for a file with that name.  If a
- * File, then the filesystem is examined directly, without referring to the
- * CLASSPATH.
+ * File, then the local filesystem is examined directly, without referring to
+ * the CLASSPATH.
  *
  * <p>Configuration resources are of two types: default and
  * final.  Default values are loaded first and final values are loaded last, and
@@ -78,7 +80,7 @@ public class Configuration {
   }
 
   /** Add a default resource. */
-  public void addDefaultResource(File file) {
+  public void addDefaultResource(Path file) {
     addResource(defaultResources, file);
   }
 
@@ -88,7 +90,7 @@ public class Configuration {
   }
 
   /** Add a final resource. */
-  public void addFinalResource(File file) {
+  public void addFinalResource(Path file) {
     addResource(finalResources, file);
   }
 
@@ -210,13 +212,13 @@ public class Configuration {
 
   /** Returns the value of the <code>name</code> property as an array of
    * strings.  If no such property is specified, then <code>null</code>
-   * is returned.  Values are whitespace or comma delimted.
+   * is returned.  Values are comma delimited.
    */
   public String[] getStrings(String name) {
     String valueString = get(name);
     if (valueString == null)
       return null;
-    StringTokenizer tokenizer = new StringTokenizer (valueString,", \t\n\r\f");
+    StringTokenizer tokenizer = new StringTokenizer (valueString,",");
     List values = new ArrayList();
     while (tokenizer.hasMoreTokens()) {
       values.add(tokenizer.nextToken());
@@ -263,17 +265,39 @@ public class Configuration {
     set(propertyName, theClass.getName());
   }
 
-  /** Returns a file name under a directory named in <i>dirsProp</i> with the
-   * given <i>path</i>.  If <i>dirsProp</i> contains multiple directories, then
-   * one is chosen based on <i>path</i>'s hash code.  If the selected directory
-   * does not exist, an attempt is made to create it.
+  /** Returns a local file under a directory named in <i>dirsProp</i> with
+   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+   * then one is chosen based on <i>path</i>'s hash code.  If the selected
+   * directory does not exist, an attempt is made to create it.
    */
-  public File getFile(String dirsProp, String path) throws IOException {
+  public Path getLocalPath(String dirsProp, String path)
+    throws IOException {
     String[] dirs = getStrings(dirsProp);
     int hashCode = path.hashCode();
+    FileSystem fs = FileSystem.getNamed("local", this);
     for (int i = 0; i < dirs.length; i++) {  // try each local dir
       int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
-      File file = new File(dirs[index], path).getAbsoluteFile();
+      Path file = new Path(dirs[index], path);
+      Path dir = file.getParent();
+      if (fs.exists(dir) || fs.mkdirs(dir)) {
+        return file;
+      }
+    }
+    throw new IOException("No valid local directories in property: "+dirsProp);
+  }
+
+  /** Returns a local file name under a directory named in <i>dirsProp</i> with
+   * the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+   * then one is chosen based on <i>path</i>'s hash code.  If the selected
+   * directory does not exist, an attempt is made to create it.
+   */
+  public File getFile(String dirsProp, String path)
+    throws IOException {
+    String[] dirs = getStrings(dirsProp);
+    int hashCode = path.hashCode();
+    for (int i = 0; i < dirs.length; i++) {  // try each local dir
+      int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
+      File file = new File(dirs[index], path);
       File dir = file.getParentFile();
       if (dir.exists() || dir.mkdirs()) {
         return file;
@@ -283,6 +307,7 @@ public class Configuration {
   }
 
 
+
   /** Returns the URL for the named resource. */
   public URL getResource(String name) {
     return classLoader.getResource(name);
@@ -358,11 +383,17 @@ public class Configuration {
           LOG.info("parsing " + url);
           doc = builder.parse(url.toString());
         }
-      } else if (name instanceof File) {          // a file resource
-        File file = (File)name;
-        if (file.exists()) {
+      } else if (name instanceof Path) {          // a file resource
+        Path file = (Path)name;
+        FileSystem fs = FileSystem.getNamed("local", this);
+        if (fs.exists(file)) {
           LOG.info("parsing " + file);
-          doc = builder.parse(file);
+          InputStream in = new BufferedInputStream(fs.openRaw(file));
+          try {
+            doc = builder.parse(in);
+          } finally {
+            in.close();
+          }
         }
       }
 
@@ -466,8 +497,8 @@ public class Configuration {
         sb.append(" , ");
       }
       Object obj = i.next();
-      if (obj instanceof File) {
-        sb.append((File)obj);
+      if (obj instanceof Path) {
+        sb.append((Path)obj);
       } else {
         sb.append((String)obj);
       }

+ 1 - 1
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -157,7 +157,7 @@ class DFSClient implements FSConstants {
 
     /**
      */
-    public DFSFileInfo[] listFiles(UTF8 src) throws IOException {
+    public DFSFileInfo[] listPaths(UTF8 src) throws IOException {
         return namenode.getListing(src.toString());
     }
 

+ 0 - 101
src/java/org/apache/hadoop/dfs/DFSFile.java

@@ -1,101 +0,0 @@
-/**
- * Copyright 2005 The Apache Software Foundation
- *
- * Licensed 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.dfs;
-
-import java.io.*;
-
-
-/*****************************************************************
- * DFSFile is a traditional java File that's been annotated with
- * some extra information.
- *
- * @author Mike Cafarella
- *****************************************************************/
-class DFSFile extends File {
-    DFSFileInfo info;
-
-    /** Separator used in DFS filenames. */
-    public static final String DFS_FILE_SEPARATOR = "/";
-    
-    /**
-     */
-    public DFSFile(DFSFileInfo info) {
-        super(info.getPath());
-        this.info = info;
-    }
-
-    /**
-     * A number of File methods are unsupported in this subclass
-     */
-    public boolean canRead() {
-        return false;
-    }
-    public boolean canWrite() {
-        return false;
-    }
-    public boolean createNewFile() {
-        return false;
-    }
-    public boolean delete() {
-        return false;
-    }
-    public void deleteOnExit() {
-    }
-    public boolean isHidden() {
-        return false;
-    }
-    public boolean isAbsolute() {
-        return true;
-    }
-
-    /**
-     * We need to reimplement some of them
-     */
-    public boolean isDirectory() {
-        return info.isDir();
-    }
-    public boolean isFile() {
-        return ! isDirectory();
-    }
-    public long length() {
-        return info.getLen();
-    }
-
-    /**
-     * And add a few extras
-     */
-    public long getContentsLength() {
-        return info.getContentsLen();
-    }
-    
-    /**
-     * Retrieving parent path from DFS path string
-     * @param path - DFS path 
-     * @return - parent path of DFS path, or null if no parent exist.
-     */
-    public static String getDFSParent(String path) {
-        if (path == null)
-            return null;
-        if (DFS_FILE_SEPARATOR.equals(path))
-            return null;
-        int index = path.lastIndexOf(DFS_FILE_SEPARATOR); 
-        if (index == -1)
-            return null;
-        if (index == 0)
-            return DFS_FILE_SEPARATOR;
-        return path.substring(0, index);
-    }
-}

+ 3 - 2
src/java/org/apache/hadoop/dfs/DFSFileInfo.java

@@ -16,6 +16,7 @@
 package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.Path;
 
 import java.io.*;
 
@@ -68,13 +69,13 @@ class DFSFileInfo implements Writable {
     /**
      */
     public String getName() {
-        return new File(path.toString()).getName();
+        return new Path(path.toString()).getName();
     }
 
     /**
      */
     public String getParent() {
-        return DFSFile.getDFSParent(path.toString());
+      return new Path(path.toString()).getParent().toString();
     }
 
     /**

+ 25 - 25
src/java/org/apache/hadoop/dfs/DFSShell.java

@@ -38,37 +38,37 @@ public class DFSShell {
     /**
      * Add a local file to the indicated name in DFS. src is kept.
      */
-    void copyFromLocal(File src, String dstf) throws IOException {
-        fs.copyFromLocalFile(src, new File(dstf));
+    void copyFromLocal(Path src, String dstf) throws IOException {
+        fs.copyFromLocalFile(src, new Path(dstf));
     }
 
     /**
      * Add a local file to the indicated name in DFS. src is removed.
      */
-    void moveFromLocal(File src, String dstf) throws IOException {
-        fs.moveFromLocalFile(src, new File(dstf));
+    void moveFromLocal(Path src, String dstf) throws IOException {
+        fs.moveFromLocalFile(src, new Path(dstf));
     }
 
     /**
      * Obtain the indicated DFS file and copy to the local name.
      * srcf is kept.
      */
-    void copyToLocal(String srcf, File dst) throws IOException {
-        fs.copyToLocalFile(new File(srcf), dst);
+    void copyToLocal(String srcf, Path dst) throws IOException {
+        fs.copyToLocalFile(new Path(srcf), dst);
     }
 
     /**
      * Obtain the indicated DFS file and copy to the local name.
      * srcf is removed.
      */
-    void moveToLocal(String srcf, File dst) throws IOException {
+    void moveToLocal(String srcf, Path dst) throws IOException {
         System.err.println("Option '-moveToLocal' is not implemented yet.");
     }
 
     void cat(String srcf) throws IOException {
-      FSDataInputStream in = fs.open(new File(srcf));
+      FSDataInputStream in = fs.open(new Path(srcf));
       try {
-        DataInputStream din = new DataInputStream(new BufferedInputStream(in));
+        BufferedReader din = new BufferedReader(new InputStreamReader(in));
         String line;
         while((line = din.readLine()) != null) {
           System.out.println(line);      
@@ -82,7 +82,7 @@ public class DFSShell {
      * Get a listing of all files in DFS at the indicated name
      */
     public void ls(String src, boolean recursive) throws IOException {
-        File items[] = fs.listFiles(new File(src));
+        Path items[] = fs.listPaths(new Path(src));
         if (items == null) {
             System.out.println("Could not get listing for " + src);
         } else {
@@ -90,10 +90,10 @@ public class DFSShell {
             	System.out.println("Found " + items.length + " items");
             }
             for (int i = 0; i < items.length; i++) {
-                File cur = items[i];
-                System.out.println(cur.getPath() + "\t" + (cur.isDirectory() ? "<dir>" : ("" + cur.length())));
-                if(recursive && cur.isDirectory()) {
-									 ls(cur.getPath(), recursive);
+                Path cur = items[i];
+                System.out.println(cur + "\t" + (fs.isDirectory(cur) ? "<dir>" : ("" + fs.getLength(cur))));
+                if(recursive && fs.isDirectory(cur)) {
+                  ls(cur.toString(), recursive);
                 }
             }
         }
@@ -102,14 +102,14 @@ public class DFSShell {
     /**
      */
     public void du(String src) throws IOException {
-        File items[] = fs.listFiles(new File(src));
+        Path items[] = fs.listPaths(new Path(src));
         if (items == null) {
             System.out.println("Could not get listing for " + src);
         } else {
             System.out.println("Found " + items.length + " items");
             for (int i = 0; i < items.length; i++) {
-                DFSFile cur = (DFSFile) items[i];
-                System.out.println(cur.getPath() + "\t" + cur.getContentsLength());
+                DfsPath cur = (DfsPath) items[i];
+                System.out.println(cur + "\t" + cur.getContentsLength());
             }
         }
     }
@@ -118,7 +118,7 @@ public class DFSShell {
      * Create the given dir
      */
     public void mkdir(String src) throws IOException {
-        File f = new File(src);
+        Path f = new Path(src);
         fs.mkdirs(f);
     }
     
@@ -126,7 +126,7 @@ public class DFSShell {
      * Rename an DFS file
      */
     public void rename(String srcf, String dstf) throws IOException {
-        if (fs.rename(new File(srcf), new File(dstf))) {
+        if (fs.rename(new Path(srcf), new Path(dstf))) {
             System.out.println("Renamed " + srcf + " to " + dstf);
         } else {
             System.out.println("Rename failed");
@@ -137,14 +137,14 @@ public class DFSShell {
      * Copy an DFS file
      */
     public void copy(String srcf, String dstf, Configuration conf) throws IOException {
-      DistributedFileSystem.doCopy(fs, new File(srcf), fs, new File(dstf), true, conf);
+      FileUtil.copy(fs, new Path(srcf), fs, new Path(dstf), false, conf);
     }
 
     /**
      * Delete an DFS file
      */
     public void delete(String srcf) throws IOException {
-        if (fs.delete(new File(srcf))) {
+        if (fs.delete(new Path(srcf))) {
             System.out.println("Deleted " + srcf);
         } else {
             System.out.println("Delete failed");
@@ -228,15 +228,15 @@ public class DFSShell {
             DFSShell tc = new DFSShell(fs);
 
             if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
-                tc.copyFromLocal(new File(argv[i++]), argv[i++]);
+                tc.copyFromLocal(new Path(argv[i++]), argv[i++]);
             } else if ("-moveFromLocal".equals(cmd)) {
-                tc.moveFromLocal(new File(argv[i++]), argv[i++]);
+                tc.moveFromLocal(new Path(argv[i++]), argv[i++]);
             } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
-                tc.copyToLocal(argv[i++], new File(argv[i++]));
+                tc.copyToLocal(argv[i++], new Path(argv[i++]));
             } else if ("-cat".equals(cmd)) {
                 tc.cat(argv[i++]);
             } else if ("-moveToLocal".equals(cmd)) {
-                tc.moveToLocal(argv[i++], new File(argv[i++]));
+                tc.moveToLocal(argv[i++], new Path(argv[i++]));
             } else if ("-ls".equals(cmd)) {
                 String arg = i < argv.length ? argv[i++] : "";
                 tc.ls(arg, false);

+ 2 - 2
src/java/org/apache/hadoop/dfs/DFSck.java

@@ -105,7 +105,7 @@ public class DFSck {
    * @throws Exception
    */
   public Result fsck(String path) throws Exception {
-    DFSFileInfo[] files = dfs.listFiles(new UTF8(path));
+    DFSFileInfo[] files = dfs.listPaths(new UTF8(path));
     Result res = new Result();
     res.setReplication(conf.getInt("dfs.replication", 3));
     for (int i = 0; i < files.length; i++) {
@@ -119,7 +119,7 @@ public class DFSck {
       if (showFiles)
         System.out.println(file.getPath() + " <dir>");
       res.totalDirs++;
-      DFSFileInfo[] files = dfs.listFiles(new UTF8(file.getPath()));
+      DFSFileInfo[] files = dfs.listPaths(new UTF8(file.getPath()));
       for (int i = 0; i < files.length; i++) {
         check(files[i], res);
       }

+ 44 - 0
src/java/org/apache/hadoop/dfs/DfsPath.java

@@ -0,0 +1,44 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.dfs;
+
+import java.io.*;
+
+import org.apache.hadoop.fs.Path;
+
+
+/** DfsPath is a Path that's been annotated with some extra information. */
+class DfsPath extends Path {
+    DFSFileInfo info;
+
+    public DfsPath(DFSFileInfo info) {
+        super(info.getPath());
+        this.info = info;
+    }
+
+    public boolean isDirectory() {
+        return info.isDir();
+    }
+    public boolean isFile() {
+        return ! isDirectory();
+    }
+    public long length() {
+        return info.getLen();
+    }
+    public long getContentsLength() {
+        return info.getContentsLen();
+    }
+}

+ 48 - 143
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -32,10 +32,11 @@ import org.apache.hadoop.conf.Configuration;
  * @author Mike Cafarella
  *****************************************************************/
 public class DistributedFileSystem extends FileSystem {
-    private File workingDir = 
-      new File("/user", System.getProperty("user.name")).getAbsoluteFile();
+    private Path workingDir = 
+      new Path("/user", System.getProperty("user.name"));
 
     private String name;
+    private FileSystem localFs;
 
     DFSClient dfs;
 
@@ -45,40 +46,40 @@ public class DistributedFileSystem extends FileSystem {
       super(conf);
       this.dfs = new DFSClient(namenode, conf);
       this.name = namenode.getHostName() + ":" + namenode.getPort();
+      this.localFs = getNamed("local", conf);
     }
 
     public String getName() { return name; }
 
-    public File getWorkingDirectory() {
+    public Path getWorkingDirectory() {
       return workingDir;
     }
     
-    private File makeAbsolute(File f) {
-      if (isAbsolute(f)) {
+    private Path makeAbsolute(Path f) {
+      if (f.isAbsolute()) {
         return f;
       } else {
-        return new File(workingDir, f.getPath());
+        return new Path(workingDir, f);
       }
     }
     
-    public void setWorkingDirectory(File dir) {
+    public void setWorkingDirectory(Path dir) {
       workingDir = makeAbsolute(dir);
     }
     
-    private UTF8 getPath(File file) {
-      String path = getDFSPath(makeAbsolute(file));
-      return new UTF8(path);
+    private UTF8 getPath(Path file) {
+      return new UTF8(makeAbsolute(file).toString());
     }
 
-    public String[][] getFileCacheHints(File f, long start, long len) throws IOException {
+    public String[][] getFileCacheHints(Path f, long start, long len) throws IOException {
       return dfs.getHints(getPath(f), start, len);
     }
 
-    public FSInputStream openRaw(File f) throws IOException {
+    public FSInputStream openRaw(Path f) throws IOException {
       return dfs.open(getPath(f));
     }
 
-    public FSOutputStream createRaw(File f, boolean overwrite, short replication)
+    public FSOutputStream createRaw(Path f, boolean overwrite, short replication)
       throws IOException {
       return dfs.create(getPath(f), overwrite, replication);
     }
@@ -86,161 +87,85 @@ public class DistributedFileSystem extends FileSystem {
     /**
      * Rename files/dirs
      */
-    public boolean renameRaw(File src, File dst) throws IOException {
+    public boolean renameRaw(Path src, Path dst) throws IOException {
       return dfs.rename(getPath(src), getPath(dst));
     }
 
     /**
-     * Get rid of File f, whether a true file or dir.
+     * Get rid of Path f, whether a true file or dir.
      */
-    public boolean deleteRaw(File f) throws IOException {
+    public boolean deleteRaw(Path f) throws IOException {
         return dfs.delete(getPath(f));
     }
 
-    public boolean exists(File f) throws IOException {
+    public boolean exists(Path f) throws IOException {
         return dfs.exists(getPath(f));
     }
 
-    public boolean isDirectory(File f) throws IOException {
-        if (f instanceof DFSFile) {
-          return ((DFSFile)f).isDirectory();
+    public boolean isDirectory(Path f) throws IOException {
+        if (f instanceof DfsPath) {
+          return ((DfsPath)f).isDirectory();
         }
         return dfs.isDirectory(getPath(f));
     }
 
-    public boolean isAbsolute(File f) {
-      return f.isAbsolute() ||
-        f.getPath().startsWith("/") ||
-        f.getPath().startsWith("\\");
-    }
-
-    public long getLength(File f) throws IOException {
-        if (f instanceof DFSFile) {
-          return ((DFSFile)f).length();
+    public long getLength(Path f) throws IOException {
+        if (f instanceof DfsPath) {
+          return ((DfsPath)f).length();
         }
 
-        DFSFileInfo info[] = dfs.listFiles(getPath(f));
+        DFSFileInfo info[] = dfs.listPaths(getPath(f));
         return info[0].getLen();
     }
 
-    public File[] listFilesRaw(File f) throws IOException {
-        DFSFileInfo info[] = dfs.listFiles(getPath(f));
+    public Path[] listPathsRaw(Path f) throws IOException {
+        DFSFileInfo info[] = dfs.listPaths(getPath(f));
         if (info == null) {
-            return new File[0];
+            return new Path[0];
         } else {
-            File results[] = new DFSFile[info.length];
+            Path results[] = new DfsPath[info.length];
             for (int i = 0; i < info.length; i++) {
-                results[i] = new DFSFile(info[i]);
+                results[i] = new DfsPath(info[i]);
             }
             return results;
         }
     }
 
-    public void mkdirs(File f) throws IOException {
-        dfs.mkdirs(getPath(f));
+    public boolean mkdirs(Path f) throws IOException {
+        return dfs.mkdirs(getPath(f));
     }
 
-    public void lock(File f, boolean shared) throws IOException {
+    public void lock(Path f, boolean shared) throws IOException {
         dfs.lock(getPath(f), ! shared);
     }
 
-    public void release(File f) throws IOException {
+    public void release(Path f) throws IOException {
         dfs.release(getPath(f));
     }
 
-    public void moveFromLocalFile(File src, File dst) throws IOException {
-        doFromLocalFile(src, dst, true);
-    }
-
-    public void copyFromLocalFile(File src, File dst) throws IOException {
-        doFromLocalFile(src, dst, false);
+    public void moveFromLocalFile(Path src, Path dst) throws IOException {
+      FileUtil.copy(localFs, src, this, dst, true, getConf());
     }
 
-    private void doFromLocalFile(File src, File dst, boolean deleteSource) throws IOException {
-        FileSystem localFs = getNamed("local", getConf());
-        doCopy( localFs, src, this, dst, deleteSource, getConf() );
-    }
-
-    public static void doCopy(FileSystem srcFS, 
-                        File src, 
-                        FileSystem dstFS, 
-                        File dst, 
-                        boolean deleteSource,
-                        Configuration conf
-                       ) throws IOException {
-        if (dstFS.exists(dst)) {
-            if (! dstFS.isDirectory(dst)) {
-                throw new IOException("Target " + dst + " already exists");
-            } else {
-                dst = new File(dst, src.getName());
-                if (dstFS.exists(dst)) {
-                    throw new IOException("Target " + dst + " already exists");
-                }
-            }
-        }
-
-        if (srcFS.isDirectory(src)) {
-            dstFS.mkdirs(dst);
-            File contents[] = srcFS.listFiles(src);
-            for (int i = 0; i < contents.length; i++) {
-                doCopy( srcFS, contents[i], dstFS, new File(dst, contents[i].getName()), deleteSource, conf);
-            }
-        } else {
-            byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
-            InputStream in = srcFS.open(src);
-            try {
-                OutputStream out = dstFS.create(dst);
-                try {
-                    int bytesRead = in.read(buf);
-                    while (bytesRead >= 0) {
-                        out.write(buf, 0, bytesRead);
-                        bytesRead = in.read(buf);
-                    }
-                } finally {
-                    out.close();
-                }
-            } finally {
-                in.close();
-            } 
-        }
-        if (deleteSource)
-          srcFS.delete(src);
+    public void copyFromLocalFile(Path src, Path dst) throws IOException {
+      FileUtil.copy(localFs, src, this, dst, false, getConf());
     }
 
-    public void copyToLocalFile(File src, File dst) throws IOException {
-        dst = dst.getCanonicalFile();
-        FileSystem localFs = getNamed("local", getConf());
-        doCopy( this, src, localFs, dst, false, getConf() );
+    public void copyToLocalFile(Path src, Path dst) throws IOException {
+      FileUtil.copy(this, src, localFs, dst, false, getConf());
     }
 
-    public File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
-        if (exists(fsOutputFile)) {
-            copyToLocalFile(fsOutputFile, tmpLocalFile);
-        }
-        return tmpLocalFile;
+    public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+      throws IOException {
+      return tmpLocalFile;
     }
 
     /**
      * Move completed local data to DFS destination
      */
-    public void completeLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
-        moveFromLocalFile(tmpLocalFile, fsOutputFile);
-    }
-
-    /**
-     * Fetch remote DFS file, place at tmpLocalFile
-     */
-    public File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException {
-        copyToLocalFile(fsInputFile, tmpLocalFile);
-        return tmpLocalFile;
-    }
-
-    /**
-     * We're done with the local stuff, so delete it
-     */
-    public void completeLocalInput(File localFile) throws IOException {
-        // Get rid of the local copy - we don't need it anymore.
-        FileUtil.fullyDelete(localFile, getConf());
+    public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+      throws IOException {
+      moveFromLocalFile(tmpLocalFile, fsOutputFile);
     }
 
     public void close() throws IOException {
@@ -255,27 +180,7 @@ public class DistributedFileSystem extends FileSystem {
         return dfs;
     }
     
-    private String getDFSPath(File f) {
-      List l = new ArrayList();
-      l.add(f.getName());
-      File parent = f.getParentFile();
-      while (parent != null) {
-        l.add(parent.getName());
-        parent = parent.getParentFile();
-      }
-      StringBuffer path = new StringBuffer();
-      path.append(l.get(l.size() - 1));
-      for (int i = l.size() - 2; i >= 0; i--) {
-        path.append(DFSFile.DFS_FILE_SEPARATOR);
-        path.append(l.get(i));
-      }
-      if (isAbsolute(f) && path.length() == 0) {
-        path.append(DFSFile.DFS_FILE_SEPARATOR);
-      }
-      return path.toString();
-    }
-
-    public void reportChecksumFailure(File f, FSInputStream in,
+    public void reportChecksumFailure(Path f, FSInputStream in,
                                       long start, long length, int crc) {
       
       // ignore for now, causing task to fail, and hope that when task is
@@ -305,7 +210,7 @@ public class DistributedFileSystem extends FileSystem {
     /** Return the total size of all files in the filesystem.*/
     public long getUsed()throws IOException{
         long used = 0;
-        DFSFileInfo dfsFiles[] = dfs.listFiles(getPath(new File("/")));
+        DFSFileInfo dfsFiles[] = dfs.listPaths(getPath(new Path("/")));
         for(int i=0;i<dfsFiles.length;i++){
             used += dfsFiles[i].getContentsLen();
         }

+ 1 - 1
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -199,7 +199,7 @@ class FSDataset implements FSConstants {
         }
         this.tmp = new File(dir, "tmp");
         if (tmp.exists()) {
-            FileUtil.fullyDelete(tmp, conf);
+            FileUtil.fullyDelete(tmp);
         }
         this.tmp.mkdirs();
         this.dirTree = new FSDir(data);

+ 9 - 8
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -22,6 +22,7 @@ import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 
 /*************************************************
  * FSDirectory stores the filesystem directory state.
@@ -150,10 +151,10 @@ class FSDirectory implements FSConstants {
         INode addNode(String path, INode newNode) {
           File target = new File( path );
           // find parent
-          String parentName = DFSFile.getDFSParent(path);
-          if (parentName == null)
+          Path parent = new Path(path).getParent();
+          if (parent == null)
             return null;
-          INode parentNode = getNode(parentName);
+          INode parentNode = getNode(parent.toString());
           if (parentNode == null)
             return null;
           // check whether the parent already has a node with that name
@@ -308,7 +309,7 @@ class FSDirectory implements FSConstants {
         File image = new File(dir, "image");
         File edits = new File(dir, "edits");
 
-        if (!((!image.exists() || FileUtil.fullyDelete(image, conf)) &&
+        if (!((!image.exists() || FileUtil.fullyDelete(image)) &&
               (!edits.exists() || edits.delete()) &&
               image.mkdirs())) {
           
@@ -591,7 +592,7 @@ class FSDirectory implements FSConstants {
 
         // Always do an implicit mkdirs for parent directory tree
         String pathString = path.toString();
-        mkdirs(DFSFile.getDFSParent(pathString));
+        mkdirs(new Path(pathString).getParent().toString());
         INode newNode = new INode( new File(pathString).getName(), blocks, replication);
         if( ! unprotectedAddFile(path, newNode) )
           return false;
@@ -808,10 +809,10 @@ class FSDirectory implements FSConstants {
         v.add(src);
 
         // All its parents
-        String parent = DFSFile.getDFSParent(src);
+        Path parent = new Path(src).getParent();
         while (parent != null) {
-            v.add(parent);
-            parent = DFSFile.getDFSParent(parent);
+            v.add(parent.toString());
+            parent = parent.getParent();
         }
 
         // Now go backwards through list of dirs, creating along

+ 5 - 5
src/java/org/apache/hadoop/fs/FSDataInputStream.java

@@ -36,18 +36,18 @@ public class FSDataInputStream extends DataInputStream {
   /** Verify that data matches checksums. */
   private class Checker extends FilterInputStream implements Seekable {
     private FileSystem fs;
-    private File file;
+    private Path file;
     private FSDataInputStream sums;
     private Checksum sum = new CRC32();
     private int inSum;
 
-    public Checker(FileSystem fs, File file, Configuration conf)
+    public Checker(FileSystem fs, Path file, Configuration conf)
       throws IOException {
       super(fs.openRaw(file));
       
       this.fs = fs;
       this.file = file;
-      File sumFile = fs.getChecksumFile(file);
+      Path sumFile = fs.getChecksumFile(file);
       try {
         this.sums = new FSDataInputStream(fs.openRaw(sumFile), conf);
         byte[] version = new byte[VERSION.length];
@@ -214,14 +214,14 @@ public class FSDataInputStream extends DataInputStream {
 }
   
   
-  public FSDataInputStream(FileSystem fs, File file, int bufferSize, Configuration conf)
+  public FSDataInputStream(FileSystem fs, Path file, int bufferSize, Configuration conf)
       throws IOException {
     super(null);
     this.in = new Buffer(new PositionCache(new Checker(fs, file, conf)), bufferSize);
   }
   
   
-  public FSDataInputStream(FileSystem fs, File file, Configuration conf)
+  public FSDataInputStream(FileSystem fs, Path file, Configuration conf)
     throws IOException {
     super(null);
     int bufferSize = conf.getInt("io.file.buffer.size", 4096);

+ 2 - 2
src/java/org/apache/hadoop/fs/FSDataOutputStream.java

@@ -35,7 +35,7 @@ public class FSDataOutputStream extends DataOutputStream {
     private int bytesPerSum;
 
     public Summer(FileSystem fs, 
-                  File file, 
+                  Path file, 
                   boolean overwrite, 
                   short replication,
                   Configuration conf)
@@ -125,7 +125,7 @@ public class FSDataOutputStream extends DataOutputStream {
 
   }
 
-  public FSDataOutputStream(FileSystem fs, File file,
+  public FSDataOutputStream(FileSystem fs, Path file,
                             boolean overwrite, Configuration conf,
                             int bufferSize, short replication )
   throws IOException {

+ 153 - 78
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -51,8 +51,6 @@ public abstract class FileSystem extends Configured {
      * Parse the cmd-line args, starting at i.  Remove consumed args
      * from array.  We expect param in the form:
      * '-local | -dfs <namenode:port>'
-     *
-     * @deprecated use fs.default.name config option instead
      */
     public static FileSystem parseArgs(String argv[], int i, Configuration conf) throws IOException {
         /**
@@ -107,12 +105,12 @@ public abstract class FileSystem extends Configured {
     }
 
     /** Return the name of the checksum file associated with a file.*/
-    public static File getChecksumFile(File file) {
-      return new File(file.getParentFile(), "."+file.getName()+".crc");
+    public static Path getChecksumFile(Path file) {
+      return new Path(file.getParent(), "."+file.getName()+".crc");
     }
 
     /** Return true iff file is a checksum file name.*/
-    public static boolean isChecksumFile(File file) {
+    public static boolean isChecksumFile(Path file) {
       String name = file.getName();
       return name.startsWith(".") && name.endsWith(".crc");
     }
@@ -135,49 +133,59 @@ public abstract class FileSystem extends Configured {
      *
      * The FileSystem will simply return an elt containing 'localhost'.
      */
-    public abstract String[][] getFileCacheHints(File f, long start, long len) throws IOException;
+    public abstract String[][] getFileCacheHints(Path f, long start, long len) throws IOException;
+
+    /** @deprecated Call {@link #open(Path)} instead. */
+    public FSDataInputStream open(File f) throws IOException {
+      return open(new Path(f.toString()));
+    }
 
     /**
-     * Opens an FSDataInputStream at the indicated File.
+     * Opens an FSDataInputStream at the indicated Path.
      * @param f the file name to open
      * @param bufferSize the size of the buffer to be used.
      */
-    public FSDataInputStream open(File f, int bufferSize) throws IOException {
+    public FSDataInputStream open(Path f, int bufferSize) throws IOException {
       return new FSDataInputStream(this, f, bufferSize, getConf());
     }
     
     /**
-     * Opens an FSDataInputStream at the indicated File.
+     * Opens an FSDataInputStream at the indicated Path.
      * @param f the file to open
      */
-    public FSDataInputStream open(File f) throws IOException {
+    public FSDataInputStream open(Path f) throws IOException {
       return new FSDataInputStream(this, f, getConf());
     }
 
     /**
-     * Opens an InputStream for the indicated File, whether local
+     * Opens an InputStream for the indicated Path, whether local
      * or via DFS.
      */
-    public abstract FSInputStream openRaw(File f) throws IOException;
+    public abstract FSInputStream openRaw(Path f) throws IOException;
+
+    /** @deprecated Call {@link #create(Path)} instead. */
+    public FSDataOutputStream create(File f) throws IOException {
+      return create(new Path(f.toString()));
+    }
 
     /**
-     * Opens an FSDataOutputStream at the indicated File.
+     * Opens an FSDataOutputStream at the indicated Path.
      * Files are overwritten by default.
      */
-    public FSDataOutputStream create(File f) throws IOException {
+    public FSDataOutputStream create(Path f) throws IOException {
       return create(f, true, 
                     getConf().getInt("io.file.buffer.size", 4096),
                     (short)getConf().getInt("dfs.replication", 3));
     }
 
     /**
-     * Opens an FSDataOutputStream at the indicated File.
+     * Opens an FSDataOutputStream at the indicated Path.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param bufferSize the size of the buffer to be used.
      */
-    public FSDataOutputStream create( File f, 
+    public FSDataOutputStream create( Path f, 
                                       boolean overwrite,
                                       int bufferSize
                                     ) throws IOException {
@@ -186,14 +194,14 @@ public abstract class FileSystem extends Configured {
     }
     
     /**
-     * Opens an FSDataOutputStream at the indicated File.
+     * Opens an FSDataOutputStream at the indicated Path.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param bufferSize the size of the buffer to be used.
      * @param replication required block replication for the file. 
      */
-    public FSDataOutputStream create( File f, 
+    public FSDataOutputStream create( Path f, 
                                       boolean overwrite,
                                       int bufferSize,
                                       short replication
@@ -202,20 +210,25 @@ public abstract class FileSystem extends Configured {
                                     bufferSize, replication );
     }
 
-    /** Opens an OutputStream at the indicated File.
+    /** Opens an OutputStream at the indicated Path.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param replication required block replication for the file. 
      */
-    public abstract FSOutputStream createRaw(File f, boolean overwrite, short replication)
+    public abstract FSOutputStream createRaw(Path f, boolean overwrite, short replication)
       throws IOException;
 
+    /** @deprecated Call {@link #createNewFile(Path)} instead. */
+    public boolean createNewFile(File f) throws IOException {
+      return createNewFile(new Path(f.toString()));
+    }
+
     /**
-     * Creates the given File as a brand-new zero-length file.  If
+     * Creates the given Path as a brand-new zero-length file.  If
      * create fails, or if it already existed, return false.
      */
-    public boolean createNewFile(File f) throws IOException {
+    public boolean createNewFile(Path f) throws IOException {
         if (exists(f)) {
             return false;
         } else {
@@ -229,18 +242,23 @@ public abstract class FileSystem extends Configured {
         }
     }
 
+    /** @deprecated Call {@link #rename(Path, Path)} instead. */
+    public boolean rename(File src, File dst) throws IOException {
+      return rename(new Path(src.toString()), new Path(dst.toString()));
+    }
+
     /**
-     * Renames File src to File dst.  Can take place on local fs
+     * Renames Path src to Path dst.  Can take place on local fs
      * or remote DFS.
      */
-    public boolean rename(File src, File dst) throws IOException {
+    public boolean rename(Path src, Path dst) throws IOException {
       if (isDirectory(src)) {
         return renameRaw(src, dst);
       } else {
 
         boolean value = renameRaw(src, dst);
 
-        File checkFile = getChecksumFile(src);
+        Path checkFile = getChecksumFile(src);
         if (exists(checkFile))
           renameRaw(checkFile, getChecksumFile(dst)); // try to rename checksum
 
@@ -250,15 +268,18 @@ public abstract class FileSystem extends Configured {
     }
 
     /**
-     * Renames File src to File dst.  Can take place on local fs
+     * Renames Path src to Path dst.  Can take place on local fs
      * or remote DFS.
      */
-    public abstract boolean renameRaw(File src, File dst) throws IOException;
+    public abstract boolean renameRaw(Path src, Path dst) throws IOException;
 
-    /**
-     * Deletes File
-     */
+    /** @deprecated Call {@link #delete(Path)} instead. */
     public boolean delete(File f) throws IOException {
+      return delete(new Path(f.toString()));
+    }
+
+    /** Delete a file. */
+    public boolean delete(Path f) throws IOException {
       if (isDirectory(f)) {
         return deleteRaw(f);
       } else {
@@ -268,20 +289,33 @@ public abstract class FileSystem extends Configured {
     }
 
     /**
-     * Deletes File
+     * Deletes Path
      */
-    public abstract boolean deleteRaw(File f) throws IOException;
+    public abstract boolean deleteRaw(Path f) throws IOException;
 
-    /**
-     * Check if exists
-     */
-    public abstract boolean exists(File f) throws IOException;
+    /** @deprecated call {@link #exists(Path)} instead */
+    public boolean exists(File f) throws IOException {
+      return exists(new Path(f.toString()));
+    }
+
+    /** Check if exists. */
+    public abstract boolean exists(Path f) throws IOException;
+
+    /** @deprecated Call {@link #isDirectory(Path)} instead. */
+    public boolean isDirectory(File f) throws IOException {
+      return isDirectory(new Path(f.toString()));
+    }
 
     /** True iff the named path is a directory. */
-    public abstract boolean isDirectory(File f) throws IOException;
+    public abstract boolean isDirectory(Path f) throws IOException;
 
-    /** True iff the named path is a regular file. */
+    /** @deprecated Call {@link #isFile(Path)} instead. */
     public boolean isFile(File f) throws IOException {
+      return isFile(new Path(f.toString()));
+    }
+
+    /** True iff the named path is a regular file. */
+    public boolean isFile(Path f) throws IOException {
         if (exists(f) && ! isDirectory(f)) {
             return true;
         } else {
@@ -289,28 +323,59 @@ public abstract class FileSystem extends Configured {
         }
     }
     
-    /** True iff the named path is absolute. */
-    public abstract boolean isAbsolute(File f);
+    /** @deprecated Call {@link #getLength(Path)} instead. */
+    public long getLength(File f) throws IOException {
+      return getLength(new Path(f.toString()));
+    }
 
     /** The number of bytes in a file. */
-    public abstract long getLength(File f) throws IOException;
+    public abstract long getLength(Path f) throws IOException;
 
-    /** List files in a directory. */
+    /** @deprecated Call {@link #listPaths(Path)} instead. */
     public File[] listFiles(File f) throws IOException {
-      return listFiles(f, new FileFilter() {
-          public boolean accept(File file) {
+      Path[] paths = listPaths(new Path(f.toString()));
+      if (paths == null)
+        return null;
+      File[] result = new File[paths.length];
+      for (int i = 0 ; i < paths.length; i++) {
+        result[i] = new File(paths[i].toString());
+      }
+      return result;
+    }
+
+    /** List files in a directory. */
+    public Path[] listPaths(Path f) throws IOException {
+      return listPaths(f, new PathFilter() {
+          public boolean accept(Path file) {
             return !isChecksumFile(file);
           }
         });
     }
 
     /** List files in a directory. */
-    public abstract File[] listFilesRaw(File f) throws IOException;
+    public abstract Path[] listPathsRaw(Path f) throws IOException;
+
+    /** @deprecated Call {@link #listPaths(Path)} instead. */
+    public File[] listFiles(File f, final FileFilter filt) throws IOException {
+      Path[] paths = listPaths(new Path(f.toString()),
+                               new PathFilter() {
+                                 public boolean accept(Path p) {
+                                   return filt.accept(new File(p.toString()));
+                                 }
+                               });
+      if (paths == null)
+        return null;
+      File[] result = new File[paths.length];
+      for (int i = 0 ; i < paths.length; i++) {
+        result[i] = new File(paths[i].toString());
+      }
+      return result;
+    }
 
     /** Filter files in a directory. */
-    public File[] listFiles(File f, FileFilter filter) throws IOException {
+    public Path[] listPaths(Path f, PathFilter filter) throws IOException {
         Vector results = new Vector();
-        File listing[] = listFilesRaw(f);
+        Path listing[] = listPathsRaw(f);
         if (listing != null) {
           for (int i = 0; i < listing.length; i++) {
             if (filter.accept(listing[i])) {
@@ -318,7 +383,7 @@ public abstract class FileSystem extends Configured {
             }
           }
         }
-        return (File[]) results.toArray(new File[results.size()]);
+        return (Path[]) results.toArray(new Path[results.size()]);
     }
 
     /**
@@ -326,54 +391,75 @@ public abstract class FileSystem extends Configured {
      * All relative paths will be resolved relative to it.
      * @param new_dir
      */
-    public abstract void setWorkingDirectory(File new_dir);
+    public abstract void setWorkingDirectory(Path new_dir);
     
     /**
      * Get the current working directory for the given file system
      * @return the directory pathname
      */
-    public abstract File getWorkingDirectory();
+    public abstract Path getWorkingDirectory();
     
+    /** @deprecated Call {@link #mkdirs(Path)} instead. */
+    public boolean mkdirs(File f) throws IOException {
+      return mkdirs(new Path(f.toString()));
+    }
+
     /**
      * Make the given file and all non-existent parents into
      * directories.
      */
-    public abstract void mkdirs(File f) throws IOException;
+    public abstract boolean mkdirs(Path f) throws IOException;
+
+    /** @deprecated Call {@link #lock(Path,boolean)} instead. */
+    public void lock(File f, boolean shared) throws IOException {
+      lock(new Path(f.toString()), shared);
+    }
 
     /**
-     * Obtain a lock on the given File
+     * Obtain a lock on the given Path
      */
-    public abstract void lock(File f, boolean shared) throws IOException;
+    public abstract void lock(Path f, boolean shared) throws IOException;
+
+    /** @deprecated Call {@link #release(Path)} instead. */
+    public void release(File f) throws IOException {
+      release(new Path(f.toString()));
+    }
 
     /**
      * Release the lock
      */
-    public abstract void release(File f) throws IOException;
+    public abstract void release(Path f) throws IOException;
 
     /**
      * The src file is on the local disk.  Add it to FS at
      * the given dst name and the source is kept intact afterwards
      */
-    public abstract void copyFromLocalFile(File src, File dst) throws IOException;
+    public abstract void copyFromLocalFile(Path src, Path dst) throws IOException;
 
     /**
      * The src file is on the local disk.  Add it to FS at
      * the given dst name, removing the source afterwards.
      */
-    public abstract void moveFromLocalFile(File src, File dst) throws IOException;
+    public abstract void moveFromLocalFile(Path src, Path dst) throws IOException;
 
     /**
-     * The src file is under FS2, and the dst is on the local disk.
+     * The src file is under FS, and the dst is on the local disk.
      * Copy it from FS control to the local dst name.
      */
-    public abstract void copyToLocalFile(File src, File dst) throws IOException;
+    public abstract void copyToLocalFile(Path src, Path dst) throws IOException;
 
     /**
-     * the same as copyToLocalFile(File src, File dst), except that
+     * the same as copyToLocalFile(Path src, File dst), except that
      * the source is removed afterward.
      */
     // not implemented yet
-    //public abstract void moveToLocalFile(File src, File dst) throws IOException;
+    //public abstract void moveToLocalFile(Path src, File dst) throws IOException;
+
+    /** @deprecated Call {@link #startLocalOutput(Path, Path)} instead. */
+    public File startLocalOutput(File src, File dst) throws IOException {
+      return new File(startLocalOutput(new Path(src.toString()),
+                                       new Path(dst.toString())).toString());
+    }
 
     /**
      * Returns a local File that the user can write output to.  The caller
@@ -381,23 +467,12 @@ public abstract class FileSystem extends Configured {
      * file.  If the FS is local, we write directly into the target.  If
      * the FS is remote, we write into the tmp local area.
      */
-    public abstract File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException;
-
-    /**
-     * Called when we're all done writing to the target.  A local FS will
-     * do nothing, because we've written to exactly the right place.  A remote
-     * FS will copy the contents of tmpLocalFile to the correct target at
-     * fsOutputFile.
-     */
-    public abstract void completeLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException;
+    public abstract Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException;
 
-    /**
-     * Returns a local File that the user can read from.  The caller 
-     * provides both the eventual FS target name and the local working
-     * file.  If the FS is local, we read directly from the source.  If
-     * the FS is remote, we write data into the tmp local area.
-     */
-    public abstract File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException;
+    /** @deprecated Call {@link #completeLocalOutput(Path, Path)} instead. */
+    public void completeLocalOutput(File src, File dst) throws IOException {
+      completeLocalOutput(new Path(src.toString()), new Path(dst.toString()));
+    }
 
     /**
      * Called when we're all done writing to the target.  A local FS will
@@ -405,7 +480,7 @@ public abstract class FileSystem extends Configured {
      * FS will copy the contents of tmpLocalFile to the correct target at
      * fsOutputFile.
      */
-    public abstract void completeLocalInput(File localFile) throws IOException;
+    public abstract void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException;
 
     /**
      * No more filesystem operations are needed.  Will
@@ -421,7 +496,7 @@ public abstract class FileSystem extends Configured {
      * @param length the length of the bad data in the file
      * @param crc the expected CRC32 of the data
      */
-    public abstract void reportChecksumFailure(File f, FSInputStream in,
+    public abstract void reportChecksumFailure(Path f, FSInputStream in,
                                                long start, long length,
                                                int crc);
 

+ 150 - 85
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -24,100 +24,165 @@ import org.apache.hadoop.conf.Configuration;
  * A collection of file-processing util methods
  */
 public class FileUtil {
-    /**
-     * Delete a directory and all its contents.  If
-     * we return false, the directory may be partially-deleted.
-     */
-    public static boolean fullyDelete(File dir, Configuration conf) throws IOException {
-        return fullyDelete(new LocalFileSystem(conf), dir);
-    }
-    public static boolean fullyDelete(FileSystem fs, File dir) throws IOException {
-        // 20041022, xing.
-        // Currently fs.detele(File) means fully delete for both
-        // LocalFileSystem.java and DistributedFileSystem.java. So we are okay now.
-        // If implementation changes in future, it should be modified too.
-        return fs.delete(dir);
-    }
+  
+  /** @deprecated Call {@link #fullyDelete(File)}. */
+  public static boolean fullyDelete(File dir, Configuration conf)
+    throws IOException {
+    return fullyDelete(dir);
+  }
 
-    /**
-     * Copy a file's contents to a new location.
-     * Returns whether a target file was overwritten
-     */
-    public static boolean copyContents(FileSystem fs, File src, File dst, boolean overwrite, Configuration conf) throws IOException {
-        if (fs.exists(dst) && !overwrite) {
+  /**
+   * Delete a directory and all its contents.  If
+   * we return false, the directory may be partially-deleted.
+   */
+  public static boolean fullyDelete(File dir) throws IOException {
+    File contents[] = dir.listFiles();
+    if (contents != null) {
+      for (int i = 0; i < contents.length; i++) {
+        if (contents[i].isFile()) {
+          if (! contents[i].delete()) {
+            return false;
+          }
+        } else {
+          if (! fullyDelete(contents[i])) {
             return false;
+          }
         }
+      }
+    }
+    return dir.delete();
+  }
 
-        File dstParent = dst.getParentFile();
-        if ((dstParent != null) && (!fs.exists(dstParent))) {
-            fs.mkdirs(dstParent);
-        }
 
-        if (fs.isFile(src)) {
-            FSInputStream in = fs.openRaw(src);
-            try {
-                FSOutputStream out = fs.createRaw(dst, true, 
-                                      (short)conf.getInt("dfs.replication", 3));
-                byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
-                try {
-                    int readBytes = in.read(buf);
-
-                    while (readBytes >= 0) {
-                        out.write(buf, 0, readBytes);
-                        readBytes = in.read(buf);
-                    }
-                } finally {
-                    out.close();
-                }
-            } finally {
-                in.close();
-            }
-        } else {
-            fs.mkdirs(dst);
-            File contents[] = fs.listFilesRaw(src);
-            if (contents != null) {
-                for (int i = 0; i < contents.length; i++) {
-                    File newDst = new File(dst, contents[i].getName());
-                    if (! copyContents(fs, contents[i], newDst, overwrite, conf)) {
-                        return false;
-                    }
-                }
-            }
-        }
-        return true;
+  /** Copy files between FileSystems. */
+  public static boolean copy(FileSystem srcFS, Path src, 
+                             FileSystem dstFS, Path dst, 
+                             boolean deleteSource,
+                             Configuration conf ) throws IOException {
+    dst = checkDest(src.getName(), dstFS, dst);
+
+    if (srcFS.isDirectory(src)) {
+      dstFS.mkdirs(dst);
+      Path contents[] = srcFS.listPaths(src);
+      for (int i = 0; i < contents.length; i++) {
+        copy(srcFS, contents[i], dstFS, new Path(dst, contents[i].getName()),
+             deleteSource, conf);
+      }
+    } else if (srcFS.isFile(src)) {
+      InputStream in = srcFS.open(src);
+      try {
+        copyContent(in, dstFS.create(dst), conf);
+      } finally {
+        in.close();
+      } 
+    }
+    if (deleteSource) {
+      return srcFS.delete(src);
+    } else {
+      return true;
+    }
+  }
+
+  /** Copy local files to a FileSystem. */
+  public static boolean copy(File src,
+                             FileSystem dstFS, Path dst,
+                             boolean deleteSource,
+                             Configuration conf ) throws IOException {
+    dst = checkDest(src.getName(), dstFS, dst);
+
+    if (src.isDirectory()) {
+      dstFS.mkdirs(dst);
+      File contents[] = src.listFiles();
+      for (int i = 0; i < contents.length; i++) {
+        copy(contents[i], dstFS, new Path(dst, contents[i].getName()),
+             deleteSource, conf);
+      }
+    } else if (src.isFile()) {
+      InputStream in = new FileInputStream(src);
+      try {
+        copyContent(in, dstFS.create(dst), conf);
+      } finally {
+        in.close();
+      } 
+    }
+    if (deleteSource) {
+      return FileUtil.fullyDelete(src);
+    } else {
+      return true;
+    }
+  }
+
+  /** Copy FileSystem files to local files. */
+  public static boolean copy(FileSystem srcFS, Path src, 
+                             File dst, boolean deleteSource,
+                             Configuration conf ) throws IOException {
+
+    dst = checkDest(src.getName(), dst);
+
+    if (srcFS.isDirectory(src)) {
+      dst.mkdirs();
+      Path contents[] = srcFS.listPaths(src);
+      for (int i = 0; i < contents.length; i++) {
+        copy(srcFS, contents[i], new File(dst, contents[i].getName()),
+             deleteSource, conf);
+      }
+    } else if (srcFS.isFile(src)) {
+      InputStream in = srcFS.open(src);
+      try {
+        copyContent(in, new FileOutputStream(dst), conf);
+      } finally {
+        in.close();
+      } 
     }
+    if (deleteSource) {
+      return srcFS.delete(src);
+    } else {
+      return true;
+    }
+  }
+
+  private static void copyContent(InputStream in, OutputStream out,
+                                  Configuration conf) throws IOException {
+    byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
+    try {
+      int bytesRead = in.read(buf);
+      while (bytesRead >= 0) {
+        out.write(buf, 0, bytesRead);
+        bytesRead = in.read(buf);
+      }
+    } finally {
+      out.close();
+    }
+  }
 
-    /**
-     * Copy a file and/or directory and all its contents (whether
-     * data or other files/dirs)
-     */
-    public static void recursiveCopy(FileSystem fs, File src, File dst, Configuration conf) throws IOException {
-        //
-        // Resolve the real target.
-        //
-        if (fs.exists(dst) && fs.isDirectory(dst)) {
-            dst = new File(dst, src.getName());
-        } else if (fs.exists(dst)) {
-            throw new IOException("Destination " + dst + " already exists");
+  private static Path checkDest(String srcName, FileSystem dstFS, Path dst)
+    throws IOException {
+    if (dstFS.exists(dst)) {
+      if (!dstFS.isDirectory(dst)) {
+        throw new IOException("Target " + dst + " already exists");
+      } else {
+        dst = new Path(dst, srcName);
+        if (dstFS.exists(dst)) {
+          throw new IOException("Target " + dst + " already exists");
         }
+      }
+    }
+    return dst;
+  }
 
-        //
-        // Copy the items
-        //
-        if (! fs.isDirectory(src)) {
-            //
-            // If the source is a file, then just copy the contents
-            //
-            copyContents(fs, src, dst, true, conf);
-        } else {
-            //
-            // If the source is a dir, then we need to copy all the subfiles.
-            //
-            fs.mkdirs(dst);
-            File contents[] = fs.listFiles(src);
-            for (int i = 0; i < contents.length; i++) {
-                recursiveCopy(fs, contents[i], new File(dst, contents[i].getName()), conf);
-            }
+  private static File checkDest(String srcName, File dst)
+    throws IOException {
+    if (dst.exists()) {
+      if (!dst.isDirectory()) {
+        throw new IOException("Target " + dst + " already exists");
+      } else {
+        dst = new File(dst, srcName);
+        if (dst.exists()) {
+          throw new IOException("Target " + dst + " already exists");
         }
+      }
     }
+    return dst;
+  }
+
 }

+ 69 - 128
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -29,8 +29,8 @@ import org.apache.hadoop.conf.Configuration;
  * @author Mike Cafarella
  *****************************************************************/
 public class LocalFileSystem extends FileSystem {
-    private File workingDir
-      = new File(System.getProperty("user.dir")).getAbsoluteFile();
+    private Path workingDir =
+      new Path(System.getProperty("user.dir"));
     TreeMap sharedLockDataSet = new TreeMap();
     TreeMap nonsharedLockDataSet = new TreeMap();
     TreeMap lockObjSet = new TreeMap();
@@ -52,8 +52,8 @@ public class LocalFileSystem extends FileSystem {
      * Return 1x1 'localhost' cell if the file exists.
      * Return null if otherwise.
      */
-    public String[][] getFileCacheHints(File f, long start, long len) throws IOException {
-        if (! f.exists()) {
+    public String[][] getFileCacheHints(Path f, long start, long len) throws IOException {
+        if (! exists(f)) {
             return null;
         } else {
             String result[][] = new String[1][];
@@ -65,14 +65,22 @@ public class LocalFileSystem extends FileSystem {
 
     public String getName() { return "local"; }
 
+    /** Convert a path to a File. */
+    public File pathToFile(Path path) {
+      if (!path.isAbsolute()) {
+        path = new Path(workingDir, path);
+      }
+      return new File(path.toString());
+    }
+
     /*******************************************************
      * For open()'s FSInputStream
      *******************************************************/
     class LocalFSFileInputStream extends FSInputStream {
         FileInputStream fis;
 
-        public LocalFSFileInputStream(File f) throws IOException {
-          this.fis = new FileInputStream(f);
+        public LocalFSFileInputStream(Path f) throws IOException {
+          this.fis = new FileInputStream(pathToFile(f));
         }
 
         public void seek(long pos) throws IOException {
@@ -109,10 +117,9 @@ public class LocalFileSystem extends FileSystem {
         public long skip(long n) throws IOException { return fis.skip(n); }
     }
     
-    public FSInputStream openRaw(File f) throws IOException {
-        f = makeAbsolute(f);
-        if (! f.exists()) {
-            throw new FileNotFoundException(f.toString());
+    public FSInputStream openRaw(Path f) throws IOException {
+        if (! exists(f)) {
+            throw new FileNotFoundException(toString());
         }
         return new LocalFSFileInputStream(f);
     }
@@ -123,8 +130,8 @@ public class LocalFileSystem extends FileSystem {
     class LocalFSFileOutputStream extends FSOutputStream {
       FileOutputStream fos;
 
-      public LocalFSFileOutputStream(File f) throws IOException {
-        this.fos = new FileOutputStream(f);
+      public LocalFSFileOutputStream(Path f) throws IOException {
+        this.fos = new FileOutputStream(pathToFile(f));
       }
 
       public long getPos() throws IOException {
@@ -153,90 +160,72 @@ public class LocalFileSystem extends FileSystem {
       }
     }
 
-    private File makeAbsolute(File f) {
-      if (isAbsolute(f)) {
-        return f;
-      } else {
-        return new File(workingDir, f.toString()).getAbsoluteFile();
-      }
-    }
-    
-    public FSOutputStream createRaw(File f, boolean overwrite, short replication)
+    public FSOutputStream createRaw(Path f, boolean overwrite, short replication)
       throws IOException {
-        f = makeAbsolute(f);
-        if (f.exists() && ! overwrite) {
+        if (exists(f) && ! overwrite) {
             throw new IOException("File already exists:"+f);
         }
-        File parent = f.getParentFile();
+        Path parent = f.getParent();
         if (parent != null)
-          parent.mkdirs();
+          mkdirs(parent);
 
         return new LocalFSFileOutputStream(f);
     }
 
-    public boolean renameRaw(File src, File dst) throws IOException {
-        src = makeAbsolute(src);
-        dst = makeAbsolute(dst);
+    public boolean renameRaw(Path src, Path dst) throws IOException {
         if (useCopyForRename) {
-            FileUtil.copyContents(this, src, dst, true, getConf());
-            return fullyDelete(src);
-        } else return src.renameTo(dst);
+          return FileUtil.copy(this, src, this, dst, true, getConf());
+        } else return pathToFile(src).renameTo(pathToFile(dst));
     }
 
-    public boolean deleteRaw(File f) throws IOException {
-        f = makeAbsolute(f);
+    public boolean deleteRaw(Path p) throws IOException {
+        File f = pathToFile(p);
         if (f.isFile()) {
             return f.delete();
-        } else return fullyDelete(f);
-    }
-
-    public boolean exists(File f) throws IOException {
-        f = makeAbsolute(f);
-        return f.exists();
+        } else return FileUtil.fullyDelete(f);
     }
 
-    public boolean isDirectory(File f) throws IOException {
-        f = makeAbsolute(f);
-        return f.isDirectory();
+    public boolean exists(Path f) throws IOException {
+        return pathToFile(f).exists();
     }
 
-    public boolean isAbsolute(File f) {
-      return f.isAbsolute() ||
-        f.getPath().startsWith("/") ||
-        f.getPath().startsWith("\\");
+    public boolean isDirectory(Path f) throws IOException {
+        return pathToFile(f).isDirectory();
     }
 
-    public long getLength(File f) throws IOException {
-        f = makeAbsolute(f);
-        return f.length();
+    public long getLength(Path f) throws IOException {
+        return pathToFile(f).length();
     }
 
-    public File[] listFilesRaw(File f) throws IOException {
-        f = makeAbsolute(f);
-        return f.listFiles();
+    public Path[] listPathsRaw(Path f) throws IOException {
+        String[] names = pathToFile(f).list();
+        if (names == null) {
+          return null;
+        }
+        Path[] results = new Path[names.length];
+        for (int i = 0; i < names.length; i++) {
+          results[i] = new Path(f, names[i]);
+        }
+        return results;
     }
 
-    public void mkdirs(File f) throws IOException {
-        f = makeAbsolute(f);
-        f.mkdirs();
+    public boolean mkdirs(Path f) throws IOException {
+      return pathToFile(f).mkdirs();
     }
 
     /**
      * Set the working directory to the given directory.
-     * Sets both a local variable and the system property.
-     * Note that the system property is only used if the application explictly
-     * calls java.io.File.getAbsolutePath().
      */
-    public void setWorkingDirectory(File new_dir) {
-      workingDir = makeAbsolute(new_dir);
+    public void setWorkingDirectory(Path newDir) {
+      workingDir = newDir;
     }
     
-    public File getWorkingDirectory() {
+    public Path getWorkingDirectory() {
       return workingDir;
     }
     
-    public synchronized void lock(File f, boolean shared) throws IOException {
-        f = makeAbsolute(f);
+    public synchronized void lock(Path p, boolean shared) throws IOException {
+        File f = pathToFile(p);
         f.createNewFile();
 
         FileLock lockObj = null;
@@ -252,8 +241,8 @@ public class LocalFileSystem extends FileSystem {
         lockObjSet.put(f, lockObj);
     }
 
-    public synchronized void release(File f) throws IOException {
-        f = makeAbsolute(f);
+    public synchronized void release(Path p) throws IOException {
+        File f = pathToFile(p);
         FileLock lockObj = (FileLock) lockObjSet.get(f);
         FileInputStream sharedLockData = (FileInputStream) sharedLockDataSet.get(f);
         FileOutputStream nonsharedLockData = (FileOutputStream) nonsharedLockDataSet.get(f);
@@ -277,52 +266,29 @@ public class LocalFileSystem extends FileSystem {
     }
 
     // In the case of the local filesystem, we can just rename the file.
-    public void moveFromLocalFile(File src, File dst) throws IOException {
-        if (! src.equals(dst)) {
-            src = makeAbsolute(src);
-            dst = makeAbsolute(dst);
-            if (useCopyForRename) {
-                FileUtil.copyContents(this, src, dst, true, getConf());
-                fullyDelete(src);
-            } else src.renameTo(dst);
-        }
+    public void moveFromLocalFile(Path src, Path dst) throws IOException {
+      rename(src, dst);
     }
 
     // Similar to moveFromLocalFile(), except the source is kept intact.
-    public void copyFromLocalFile(File src, File dst) throws IOException {
-        if (! src.equals(dst)) {
-            src = makeAbsolute(src);
-            dst = makeAbsolute(dst);
-            FileUtil.copyContents(this, src, dst, true, getConf());
-        }
+    public void copyFromLocalFile(Path src, Path dst) throws IOException {
+      FileUtil.copy(this, src, this, dst, false, getConf());
     }
 
     // We can't delete the src file in this case.  Too bad.
-    public void copyToLocalFile(File src, File dst) throws IOException {
-        if (! src.equals(dst)) {
-            src = makeAbsolute(src);
-            dst = makeAbsolute(dst);
-            FileUtil.copyContents(this, src, dst, true, getConf());
-        }
+    public void copyToLocalFile(Path src, Path dst) throws IOException {
+      FileUtil.copy(this, src, this, dst, false, getConf());
     }
 
     // We can write output directly to the final location
-    public File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
-        return makeAbsolute(fsOutputFile);
+    public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+      throws IOException {
+      return fsOutputFile;
     }
 
     // It's in the right place - nothing to do.
-    public void completeLocalOutput(File fsWorkingFile, File tmpLocalFile) throws IOException {
-    }
-
-    // We can read directly from the real local fs.
-    public File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException {
-        return makeAbsolute(fsInputFile);
-    }
-
-    // We're done reading.  Nothing to clean up.
-    public void completeLocalInput(File localFile) throws IOException {
-        // Ignore the file, it's at the right destination!
+    public void completeLocalOutput(Path fsWorkingFile, Path tmpLocalFile)
+      throws IOException {
     }
 
     public void close() throws IOException {}
@@ -331,39 +297,14 @@ public class LocalFileSystem extends FileSystem {
         return "LocalFS";
     }
     
-    /**
-     * Implement our own version instead of using the one in FileUtil,
-     * to avoid infinite recursion.
-     * @param dir
-     * @return
-     * @throws IOException
-     */
-    private boolean fullyDelete(File dir) throws IOException {
-        dir = makeAbsolute(dir);
-        File contents[] = dir.listFiles();
-        if (contents != null) {
-            for (int i = 0; i < contents.length; i++) {
-                if (contents[i].isFile()) {
-                    if (! contents[i].delete()) {
-                        return false;
-                    }
-                } else {
-                    if (! fullyDelete(contents[i])) {
-                        return false;
-                    }
-                }
-            }
-        }
-        return dir.delete();
-    }
 
     /** Moves files to a bad file directory on the same device, so that their
      * storage will not be reused. */
-    public void reportChecksumFailure(File f, FSInputStream in,
+    public void reportChecksumFailure(Path p, FSInputStream in,
                                       long start, long length, int crc) {
       try {
         // canonicalize f   
-        f = makeAbsolute(f).getCanonicalFile();
+        File f = pathToFile(p).getCanonicalFile();
       
         // find highest writable parent dir of f on the same device
         String device = new DF(f.toString(), getConf()).getMount();
@@ -384,11 +325,11 @@ public class LocalFileSystem extends FileSystem {
         f.renameTo(badFile);                      // rename it
 
         // move checksum file too
-        File checkFile = getChecksumFile(f);
+        File checkFile = pathToFile(getChecksumFile(p));
         checkFile.renameTo(new File(badDir, checkFile.getName()+suffix));
 
       } catch (IOException e) {
-        LOG.warning("Error moving bad file " + f + ": " + e);
+        LOG.warning("Error moving bad file " + p + ": " + e);
       }
     }
 

+ 178 - 0
src/java/org/apache/hadoop/fs/Path.java

@@ -0,0 +1,178 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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;
+
+import java.util.*;
+
+/** Names a file or directory in a {@link FileSystem}.
+ * Path strings use slash as the directory separator.  A path string is
+ * absolute if it begins with a slash.
+ */
+public class Path implements Comparable {
+
+  /** The directory separator, a slash. */
+  public static final String SEPARATOR = "/";
+  
+  static final boolean WINDOWS
+    = System.getProperty("os.name").startsWith("Windows");
+
+  private boolean isAbsolute;                     // if path starts with sepr
+  private String[] elements;                      // tokenized path elements
+  private String drive;                           // Windows drive letter
+  private String asString;                        // cached toString() value
+
+  /** Resolve a child path against a parent path. */
+  public Path(String parent, String child) {
+    this(new Path(parent), new Path(child));
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(Path parent, String child) {
+    this(parent, new Path(child));
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(String parent, Path child) {
+    this(new Path(parent), child);
+  }
+
+  /** Resolve a child path against a parent path. */
+  public Path(Path parent, Path child) {
+    if (child.isAbsolute()) {
+      this.isAbsolute = child.isAbsolute;
+      this.elements = child.elements;
+    } else {
+      this.isAbsolute = parent.isAbsolute;
+      this.elements = new String[parent.elements.length+child.elements.length];
+      for (int i = 0; i < parent.elements.length; i++) {
+        elements[i] = parent.elements[i];
+      }
+      for (int i = 0; i < child.elements.length; i++) {
+        elements[i+parent.elements.length] = child.elements[i];
+      }
+    }
+    this.drive = child.drive == null ? parent.drive : child.drive;
+  }
+
+  /** Construct a path from a String. */
+  public Path(String pathString) {
+    if (WINDOWS) {                                // parse Windows path
+      int colon = pathString.indexOf(':');
+      if (colon == 1) {                           // parse Windows drive letter
+        this.drive = pathString.substring(0, 1);
+        pathString = pathString.substring(2);
+      }
+      pathString = pathString.replace('\\','/');  // convert backslash to slash
+    }
+
+    // determine whether the path is absolute
+    this.isAbsolute = pathString.startsWith(SEPARATOR);
+
+
+    // tokenize the path into elements
+    Enumeration tokens = new StringTokenizer(pathString, SEPARATOR);
+    ArrayList list = Collections.list(tokens);
+    this.elements = (String[])list.toArray(new String[list.size()]);
+  }
+
+  private Path(boolean isAbsolute, String[] elements, String drive) {
+    this.isAbsolute = isAbsolute;
+    this.elements = elements;
+    this.drive = drive;
+  }
+
+  /** True if this path is absolute. */
+  public boolean isAbsolute() { return isAbsolute; }
+
+  /** Returns the final component of this path.*/
+  public String getName() {
+    if (elements.length == 0) {
+      return "";
+    } else {
+      return elements[elements.length-1];
+    }
+  }
+
+  /** Returns the parent of a path. */
+  public Path getParent() {
+    if (elements.length  == 0) {
+      return null;
+    }
+    String[] newElements = new String[elements.length-1];
+    for (int i = 0; i < newElements.length; i++) {
+      newElements[i] = elements[i];
+    }
+    return new Path(isAbsolute, newElements, drive);
+  }
+
+  /** Adds a suffix to a the final name in the path.*/
+  public Path suffix(String suffix) {
+    return new Path(getParent(), getName()+suffix);
+  }
+
+  public String toString() {
+    if (asString == null) {
+      StringBuffer buffer = new StringBuffer();
+
+      if (drive != null) {
+        buffer.append(drive);
+        buffer.append(':');
+      }
+
+      if (elements.length == 0 && isAbsolute) {
+        buffer.append(SEPARATOR);
+      }
+
+      for (int i = 0; i < elements.length; i++) {
+        if (i !=0 || isAbsolute) {
+          buffer.append(SEPARATOR);
+        }
+        buffer.append(elements[i]);
+      }
+      asString = buffer.toString();
+    }
+    return asString;
+  }
+
+  public boolean equals(Object o) {
+    if (!(o instanceof Path)) {
+      return false;
+    }
+    Path that = (Path)o;
+    return
+      this.isAbsolute == that.isAbsolute &&
+      Arrays.equals(this.elements, that.elements) &&
+      this.drive == null ? true : this.drive.equals(that.drive);
+  }
+
+  public int hashCode() {
+    int hashCode = isAbsolute ? 1 : -1;
+    for (int i = 0; i < elements.length; i++) {
+      hashCode ^= elements[i].hashCode();
+    }
+    if (drive != null) {
+      hashCode ^= drive.hashCode();
+    }
+    return hashCode;
+  }
+
+  public int compareTo(Object o) {
+    Path that = (Path)o;
+    return this.toString().compareTo(that.toString());
+  }
+
+}

+ 30 - 0
src/java/org/apache/hadoop/fs/PathFilter.java

@@ -0,0 +1,30 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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;
+
+public interface PathFilter {
+  /**
+   * Tests whether or not the specified abstract pathname should be
+   * included in a pathname list.
+   *
+   * @param  path  The abstract pathname to be tested
+   * @return  <code>true</code> if and only if <code>pathname</code>
+   *          should be included
+   */
+  boolean accept(Path path);
+}
+
+

+ 20 - 22
src/java/org/apache/hadoop/io/MapFile.java

@@ -91,19 +91,17 @@ public class MapFile {
       this.comparator = comparator;
       this.lastKey = comparator.newKey();
 
-      File dir = new File(dirName);
+      Path dir = new Path(dirName);
       fs.mkdirs(dir);
 
-      File dataFile = new File(dir, DATA_FILE_NAME);
-      File indexFile = new File(dir, INDEX_FILE_NAME);
+      Path dataFile = new Path(dir, DATA_FILE_NAME);
+      Path indexFile = new Path(dir, INDEX_FILE_NAME);
 
       Class keyClass = comparator.getKeyClass();
       this.data =
-        new SequenceFile.Writer(fs, dataFile.getPath(), keyClass, valClass,
-                                compress);
+        new SequenceFile.Writer(fs, dataFile, keyClass, valClass, compress);
       this.index =
-        new SequenceFile.Writer(fs, indexFile.getPath(),
-                                keyClass, LongWritable.class);
+        new SequenceFile.Writer(fs, indexFile, keyClass, LongWritable.class);
     }
     
     /** The number of entries that are added before an index entry is added.*/
@@ -197,12 +195,12 @@ public class MapFile {
     /** Construct a map reader for the named map using the named comparator.*/
     public Reader(FileSystem fs, String dirName, WritableComparator comparator, Configuration conf)
       throws IOException {
-      File dir = new File(dirName);
-      File dataFile = new File(dir, DATA_FILE_NAME);
-      File indexFile = new File(dir, INDEX_FILE_NAME);
+      Path dir = new Path(dirName);
+      Path dataFile = new Path(dir, DATA_FILE_NAME);
+      Path indexFile = new Path(dir, INDEX_FILE_NAME);
 
       // open the data
-      this.data = new SequenceFile.Reader(fs, dataFile.getPath(),  conf);
+      this.data = new SequenceFile.Reader(fs, dataFile,  conf);
       this.firstPosition = data.getPosition();
 
       if (comparator == null)
@@ -213,7 +211,7 @@ public class MapFile {
       this.getKey = this.comparator.newKey();
 
       // open the index
-      this.index = new SequenceFile.Reader(fs, indexFile.getPath(), conf);
+      this.index = new SequenceFile.Reader(fs, indexFile, conf);
     }
 
     private void readIndex() throws IOException {
@@ -387,8 +385,8 @@ public class MapFile {
   /** Renames an existing map directory. */
   public static void rename(FileSystem fs, String oldName, String newName)
     throws IOException {
-    File oldDir = new File(oldName);
-    File newDir = new File(newName);
+    Path oldDir = new Path(oldName);
+    Path newDir = new Path(newName);
     if (!fs.rename(oldDir, newDir)) {
       throw new IOException("Could not rename " + oldDir + " to " + newDir);
     }
@@ -396,9 +394,9 @@ public class MapFile {
 
   /** Deletes the named map file. */
   public static void delete(FileSystem fs, String name) throws IOException {
-    File dir = new File(name);
-    File data = new File(dir, DATA_FILE_NAME);
-    File index = new File(dir, INDEX_FILE_NAME);
+    Path dir = new Path(name);
+    Path data = new Path(dir, DATA_FILE_NAME);
+    Path index = new Path(dir, INDEX_FILE_NAME);
 
     fs.delete(data);
     fs.delete(index);
@@ -415,11 +413,11 @@ public class MapFile {
    * @return number of valid entries in this MapFile, or -1 if no fixing was needed
    * @throws Exception
    */
-  public static long fix(FileSystem fs, File dir,
+  public static long fix(FileSystem fs, Path dir,
           Class keyClass, Class valueClass, boolean dryrun, Configuration conf) throws Exception {
     String dr = (dryrun ? "[DRY RUN ] " : "");
-    File data = new File(dir, DATA_FILE_NAME);
-    File index = new File(dir, INDEX_FILE_NAME);
+    Path data = new Path(dir, DATA_FILE_NAME);
+    Path index = new Path(dir, INDEX_FILE_NAME);
     int indexInterval = 128;
     if (!fs.exists(data)) {
       // there's nothing we can do to fix this!
@@ -429,7 +427,7 @@ public class MapFile {
       // no fixing needed
       return -1;
     }
-    SequenceFile.Reader dataReader = new SequenceFile.Reader(fs, data.toString(), conf);
+    SequenceFile.Reader dataReader = new SequenceFile.Reader(fs, data, conf);
     if (!dataReader.getKeyClass().equals(keyClass)) {
       throw new Exception(dr + "Wrong key class in " + dir + ", expected" + keyClass.getName() +
               ", got " + dataReader.getKeyClass().getName());
@@ -442,7 +440,7 @@ public class MapFile {
     Writable key = (Writable)keyClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     Writable value = (Writable)valueClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     SequenceFile.Writer indexWriter = null;
-    if (!dryrun) indexWriter = new SequenceFile.Writer(fs, index.toString(), keyClass, LongWritable.class);
+    if (!dryrun) indexWriter = new SequenceFile.Writer(fs, index, keyClass, LongWritable.class);
     try {
       long pos = 0L;
       LongWritable position = new LongWritable();

+ 42 - 31
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -51,7 +51,7 @@ public class SequenceFile {
     private FSDataOutputStream out;
     private DataOutputBuffer buffer = new DataOutputBuffer();
     private FileSystem fs = null;
-    private File target = null;
+    private Path target = null;
 
     private Class keyClass;
     private Class valClass;
@@ -78,9 +78,14 @@ public class SequenceFile {
       }
     }
 
+    /** @deprecated Call {@link #Writer(FileSystem,Path,Class,Class)}. */
+    public Writer(FileSystem fs, String name, Class keyClass, Class valClass)
+      throws IOException {
+      this(fs, new Path(name), keyClass, valClass, false);
+    }
+
     /** Create the named file. */
-    public Writer(FileSystem fs, String name,
-                  Class keyClass, Class valClass)
+    public Writer(FileSystem fs, Path name, Class keyClass, Class valClass)
       throws IOException {
       this(fs, name, keyClass, valClass, false);
     }
@@ -88,11 +93,11 @@ public class SequenceFile {
     /** Create the named file.
      * @param compress if true, values are compressed.
      */
-    public Writer(FileSystem fs, String name,
+    public Writer(FileSystem fs, Path name,
                   Class keyClass, Class valClass, boolean compress)
       throws IOException {
       this.fs = fs;
-      this.target = new File(name);
+      this.target = name;
       init(fs.create(target), keyClass, valClass, compress);
     }
     
@@ -197,7 +202,7 @@ public class SequenceFile {
 
   /** Writes key/value pairs from a sequence-format file. */
   public static class Reader {
-    private String file;
+    private Path file;
     private FSDataInputStream in;
     private DataOutputBuffer outBuf = new DataOutputBuffer();
     private DataInputBuffer inBuf = new DataInputBuffer();
@@ -221,26 +226,32 @@ public class SequenceFile {
     private Inflater inflater = new Inflater();
     private Configuration conf;
 
+    /** @deprecated Call {@link #Reader(FileSystem,Path,Configuration)}.*/
+    public Reader(FileSystem fs, String file, Configuration conf)
+      throws IOException {
+      this(fs, new Path(file), conf);
+    }
+
     /** Open the named file. */
-    public Reader(FileSystem fs, String file, Configuration conf) throws IOException {
+    public Reader(FileSystem fs, Path file, Configuration conf)
+      throws IOException {
       this(fs, file, conf.getInt("io.file.buffer.size", 4096));
       this.conf = conf;
     }
 
-    private Reader(FileSystem fs, String name, int bufferSize) throws IOException {
+    private Reader(FileSystem fs, Path name, int bufferSize) throws IOException {
       this.fs = fs;
       this.file = name;
-      File file = new File(name);
       this.in = fs.open(file, bufferSize);
       this.end = fs.getLength(file);
       init();
     }
     
-    private Reader(FileSystem fs, String file, int bufferSize, long start, long length)
+    private Reader(FileSystem fs, Path file, int bufferSize, long start, long length)
       throws IOException {
       this.fs = fs;
       this.file = file;
-      this.in = fs.open(new File(file), bufferSize);
+      this.in = fs.open(file, bufferSize);
       seek(start);
       init();
 
@@ -438,7 +449,7 @@ public class SequenceFile {
 
     /** Returns the name of the file. */
     public String toString() {
-      return file;
+      return file.toString();
     }
 
   }
@@ -453,10 +464,10 @@ public class SequenceFile {
 
     private WritableComparator comparator;
 
-    private String inFile;                        // when sorting
-    private String[] inFiles;                     // when merging
+    private Path inFile;                        // when sorting
+    private Path[] inFiles;                     // when merging
 
-    private String outFile;
+    private Path outFile;
 
     private int memory; // bytes
     private int factor; // merged per pass
@@ -497,8 +508,8 @@ public class SequenceFile {
     public int getMemory() { return memory; }
 
     /** Perform a file sort.*/
-    public void sort(String inFile, String outFile) throws IOException {
-      if (fs.exists(new File(outFile))) {
+    public void sort(Path inFile, Path outFile) throws IOException {
+      if (fs.exists(outFile)) {
         throw new IOException("already exists: " + outFile);
       }
 
@@ -536,7 +547,7 @@ public class SequenceFile {
       
       private Reader in;
       private FSDataOutputStream out;
-        private String outName;
+      private Path outName;
 
       public SortPass(Configuration conf) throws IOException {
         in = new Reader(fs, inFile, conf);
@@ -605,8 +616,8 @@ public class SequenceFile {
 
       private void flush(int count, boolean done) throws IOException {
         if (out == null) {
-          outName = done ? outFile : outFile+".0";
-          out = fs.create(new File(outName));
+          outName = done ? outFile : outFile.suffix(".0");
+          out = fs.create(outName);
         }
 
         if (!done) {                              // an intermediate file
@@ -694,29 +705,29 @@ public class SequenceFile {
 
       private MergeQueue queue;
       private FSDataInputStream in;
-      private String inName;
+      private Path inName;
 
       public MergePass(int pass, boolean last) throws IOException {
         this.pass = pass;
         this.last = last;
 
         this.queue =
-          new MergeQueue(factor, last ? outFile : outFile+"."+pass, last);
+          new MergeQueue(factor, last?outFile:outFile.suffix("."+pass), last);
 
-        this.inName = outFile+"."+(pass-1);
-        this.in = fs.open(new File(inName));
+        this.inName = outFile.suffix("."+(pass-1));
+        this.in = fs.open(inName);
       }
 
       public void close() throws IOException {
         in.close();                               // close and delete input
-        fs.delete(new File(inName));
+        fs.delete(inName);
 
         queue.close();                            // close queue
       }
 
       public int run() throws IOException {
         int segments = 0;
-        long end = fs.getLength(new File(inName));
+        long end = fs.getLength(inName);
 
         while (in.getPos() < end) {
           LOG.finer("merging segment " + segments);
@@ -756,12 +767,12 @@ public class SequenceFile {
     }
 
     /** Merge the provided files.*/
-    public void merge(String[] inFiles, String outFile) throws IOException {
+    public void merge(Path[] inFiles, Path outFile) throws IOException {
       this.inFiles = inFiles;
       this.outFile = outFile;
       this.factor = inFiles.length;
 
-      if (new File(outFile).exists()) {
+      if (fs.exists(outFile)) {
         throw new IOException("already exists: " + outFile);
       }
 
@@ -788,7 +799,7 @@ public class SequenceFile {
       public void run() throws IOException {
         LOG.finer("merging files=" + inFiles.length);
         for (int i = 0; i < inFiles.length; i++) {
-          String inFile = inFiles[i];
+          Path inFile = inFiles[i];
           MergeStream ms =
             new MergeStream(new Reader(fs, inFile, memory/(factor+1)));
           if (ms.next())
@@ -836,10 +847,10 @@ public class SequenceFile {
         put(stream);
       }
 
-      public MergeQueue(int size, String outName, boolean done)
+      public MergeQueue(int size, Path outName, boolean done)
         throws IOException {
         initialize(size);
-        this.out = fs.create(new File(outName), true, memory/(factor+1));
+        this.out = fs.create(outName, true, memory/(factor+1));
         this.done = done;
       }
 

+ 14 - 5
src/java/org/apache/hadoop/mapred/FileSplit.java

@@ -17,38 +17,47 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.File;                              // deprecated
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 /** A section of an input file.  Returned by {@link
  * InputFormat#getSplits(FileSystem, JobConf, int)} and passed to
  * InputFormat#getRecordReader(FileSystem,FileSplit,JobConf,Reporter). */
 public class FileSplit implements Writable {
-  private File file;
+  private Path file;
   private long start;
   private long length;
   
   FileSplit() {}
 
+  /** @deprecated Call {@link #FileSplit(Path,long,long)} instead. */
+  public FileSplit(File file, long start, long length) {
+    this(new Path(file.toString()), start, length);
+  }
+
   /** Constructs a split.
    *
    * @param file the file name
    * @param start the position of the first byte in the file to process
    * @param length the number of bytes in the file to process
    */
-  public FileSplit(File file, long start, long length) {
+  public FileSplit(Path file, long start, long length) {
     this.file = file;
     this.start = start;
     this.length = length;
   }
   
+  /** @deprecated Call {@link #getPath()} instead. */
+  public File getFile() { return new File(file.toString()); }
+  
   /** The file containing this split's data. */
-  public File getFile() { return file; }
+  public Path getPath() { return file; }
   
   /** The position of the first byte in the file to process. */
   public long getStart() { return start; }
@@ -68,7 +77,7 @@ public class FileSplit implements Writable {
     out.writeLong(length);
   }
   public void readFields(DataInput in) throws IOException {
-    file = new File(UTF8.readString(in));
+    file = new Path(UTF8.readString(in));
     start = in.readLong();
     length = in.readLong();
   }

+ 25 - 19
src/java/org/apache/hadoop/mapred/InputFormatBase.java

@@ -17,12 +17,13 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
+import java.io.File;                              // deprecated
 
 import java.util.ArrayList;
 import java.util.logging.Logger;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.LogFormatter;
 
 /** A base class for {@link InputFormat}. */
@@ -45,35 +46,40 @@ public abstract class InputFormatBase implements InputFormat {
                                                Reporter reporter)
     throws IOException;
 
+  /** @deprecated Call {@link #listFiles(FileSystem,JobConf)} instead. */
+  protected File[] listFiles(FileSystem fs, JobConf job)
+    throws IOException {
+    Path[] paths = listPaths(fs, job);
+    File[] result = new File[paths.length];
+    for (int i = 0 ; i < paths.length; i++) {
+      result[i] = new File(paths[i].toString());
+    }
+    return result;
+  }
+
   /** List input directories.
    * Subclasses may override to, e.g., select only files matching a regular
    * expression.
    * Property mapred.input.subdir, if set, names a subdirectory that
    * is appended to all input dirs specified by job, and if the given fs
-   * lists those too, each is added to the returned array of File.
+   * lists those too, each is added to the returned array of Path.
    * @param fs
    * @param job
-   * @return array of File objects, never zero length.
+   * @return array of Path objects, never zero length.
    * @throws IOException if zero items.
    */
-  protected File[] listFiles(FileSystem fs, JobConf job)
+  protected Path[] listPaths(FileSystem fs, JobConf job)
     throws IOException {
-    File[] dirs = job.getInputDirs();
-    String workDir = job.getWorkingDirectory();
+    Path[] dirs = job.getInputPaths();
     String subdir = job.get("mapred.input.subdir");
     ArrayList result = new ArrayList();
     for (int i = 0; i < dirs.length; i++) {
-      // if it is relative, make it absolute using the directory from the 
-      // JobConf
-      if (workDir != null && !fs.isAbsolute(dirs[i])) {
-        dirs[i] = new File(workDir, dirs[i].toString());
-      }
-      File[] dir = fs.listFiles(dirs[i]);
+      Path[] dir = fs.listPaths(dirs[i]);
       if (dir != null) {
         for (int j = 0; j < dir.length; j++) {
-          File file = dir[j];
+          Path file = dir[j];
           if (subdir != null) {
-            File[] subFiles = fs.listFiles(new File(file, subdir));
+            Path[] subFiles = fs.listPaths(new Path(file, subdir));
             if (subFiles != null) {
               for (int k = 0; k < subFiles.length; k++) {
                 result.add(subFiles[k]);
@@ -89,18 +95,18 @@ public abstract class InputFormatBase implements InputFormat {
     if (result.size() == 0) {
       throw new IOException("No input directories specified in: "+job);
     }
-    return (File[])result.toArray(new File[result.size()]);
+    return (Path[])result.toArray(new Path[result.size()]);
   }
 
-  /** Splits files returned by {#listFiles(FileSystem,JobConf) when
+  /** Splits files returned by {#listPaths(FileSystem,JobConf) when
    * they're too big.*/ 
   public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits)
     throws IOException {
 
-    File[] files = listFiles(fs, job);
+    Path[] files = listPaths(fs, job);
 
     for (int i = 0; i < files.length; i++) {      // check we have valid files
-      File file = files[i];
+      Path file = files[i];
       if (fs.isDirectory(file) || !fs.exists(file)) {
         throw new IOException("Not a file: "+files[i]);
       }
@@ -132,7 +138,7 @@ public abstract class InputFormatBase implements InputFormat {
 
     ArrayList splits = new ArrayList(numSplits);  // generate splits
     for (int i = 0; i < files.length; i++) {
-      File file = files[i];
+      Path file = files[i];
       long length = fs.getLength(file);
 
       long bytesRemaining = length;

+ 11 - 10
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -236,31 +236,32 @@ public class JobClient implements MRConstants {
         //
 
         // Create a number of filenames in the JobTracker's fs namespace
-        File submitJobDir = new File(job.getSystemDir(), "submit_" + Integer.toString(Math.abs(r.nextInt()), 36));
-        File submitJobFile = new File(submitJobDir, "job.xml");
-        File submitJarFile = new File(submitJobDir, "job.jar");
+        Path submitJobDir = new Path(job.getSystemDir(), "submit_" + Integer.toString(Math.abs(r.nextInt()), 36));
+        Path submitJobFile = new Path(submitJobDir, "job.xml");
+        Path submitJarFile = new Path(submitJobDir, "job.jar");
 
         String originalJarPath = job.getJar();
 
-        if (originalJarPath != null) {           // Copy jar to JobTracker's fs
+        FileSystem localFs = FileSystem.getNamed("local", job);
+        FileSystem fs = getFs();
+
+        if (originalJarPath != null) {           // copy jar to JobTracker's fs
           job.setJar(submitJarFile.toString());
-          getFs().copyFromLocalFile(new File(originalJarPath), submitJarFile);
+          fs.copyFromLocalFile(new Path(originalJarPath), submitJarFile);
         }
 
-        FileSystem fileSys = getFs();
-
         // Set the user's name and working directory
         String user = System.getProperty("user.name");
         job.setUser(user != null ? user : "Dr Who");
         if (job.getWorkingDirectory() == null) {
-          job.setWorkingDirectory(fileSys.getWorkingDirectory().toString());          
+          job.setWorkingDirectory(fs.getWorkingDirectory());          
         }
 
         // Check the output specification
         job.getOutputFormat().checkOutputSpecs(fs, job);
 
         // Write job file to JobTracker's fs        
-        FSDataOutputStream out = fileSys.create(submitJobFile);
+        FSDataOutputStream out = fs.create(submitJobFile);
         try {
           job.write(out);
         } finally {
@@ -270,7 +271,7 @@ public class JobClient implements MRConstants {
         //
         // Now, actually submit the job (using the submit name)
         //
-        JobStatus status = jobSubmitClient.submitJob(submitJobFile.getPath());
+        JobStatus status = jobSubmitClient.submitJob(submitJobFile.toString());
         if (status != null) {
             return new NetworkedJob(status);
         } else {

+ 60 - 21
src/java/org/apache/hadoop/mapred/JobConf.java

@@ -28,7 +28,9 @@ import java.util.Enumeration;
 import java.net.URL;
 import java.net.URLDecoder;
 
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.io.Writable;
@@ -82,14 +84,14 @@ public class JobConf extends Configuration {
    * @param config a Configuration-format XML job description file
    */
   public JobConf(String config) {
-    this(new File(config));
+    this(new Path(config));
   }
 
   /** Construct a map/reduce configuration.
    *
    * @param config a Configuration-format XML job description file
    */
-  public JobConf(File config) {
+  public JobConf(Path config) {
     super();
     addDefaultResource("mapred-default.xml");
     addDefaultResource(config);
@@ -98,9 +100,8 @@ public class JobConf extends Configuration {
   public String getJar() { return get("mapred.jar"); }
   public void setJar(String jar) { set("mapred.jar", jar); }
 
-  public File getSystemDir() {
-    return new File(get("mapred.system.dir", "/tmp/hadoop/mapred/system"))
-      .getAbsoluteFile();
+  public Path getSystemDir() {
+    return new Path(get("mapred.system.dir", "/tmp/hadoop/mapred/system"));
   }
 
   public String[] getLocalDirs() throws IOException {
@@ -110,35 +111,50 @@ public class JobConf extends Configuration {
   public void deleteLocalFiles() throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileUtil.fullyDelete(new File(localDirs[i]), this);
+      FileSystem.getNamed("local", this).delete(new Path(localDirs[i]));
     }
   }
 
   public void deleteLocalFiles(String subdir) throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileUtil.fullyDelete(new File(localDirs[i], subdir), this);
+      FileSystem.getNamed("local", this).delete(new Path(localDirs[i], subdir));
     }
   }
 
+  /** @deprecated Call {@link #getLocalPath(String)} instead. */
+  public File getLocalFile(String subdir, String name) throws IOException {
+    return new File(getLocalPath(subdir+Path.SEPARATOR+name).toString());
+  }
+
   /** Constructs a local file name.  Files are distributed among configured
    * local directories.*/
-  public File getLocalFile(String subdir, String name) throws IOException {
-    return getFile("mapred.local.dir", subdir + File.separator + name);
+  public Path getLocalPath(String pathString) throws IOException {
+    return getLocalPath("mapred.local.dir", pathString);
+  }
+
+  /** @deprecated Call {@link #setInputPath(Path)} instead.*/
+  public void setInputDir(File dir) { setInputPath(new Path(dir.toString())); }
+
+  public void setInputPath(Path dir) {
+    dir = new Path(getWorkingDirectory(), dir);
+    set("mapred.input.dir", dir);
   }
 
-  public void setInputDir(File dir) { set("mapred.input.dir", dir); }
+  /** @deprecated Call {@link #addInputPath(Path)} instead.*/
+  public void addInputDir(File dir) { addInputPath(new Path(dir.toString())); }
 
-  public void addInputDir(File dir) {
+  public void addInputPath(Path dir) {
+    dir = new Path(getWorkingDirectory(), dir);
     String dirs = get("mapred.input.dir");
     set("mapred.input.dir", dirs == null ? dir.toString() : dirs + "," + dir);
   }
-  public File[] getInputDirs() {
+  public Path[] getInputPaths() {
     String dirs = get("mapred.input.dir", "");
     ArrayList list = Collections.list(new StringTokenizer(dirs, ","));
-    File[] result = new File[list.size()];
+    Path[] result = new Path[list.size()];
     for (int i = 0; i < list.size(); i++) {
-      result[i] = new File((String)list.get(i));
+      result[i] = new Path((String)list.get(i));
     }
     return result;
   }
@@ -163,24 +179,47 @@ public class JobConf extends Configuration {
    * Set the current working directory for the default file system
    * @param dir the new current working directory
    */
-  public void setWorkingDirectory(String dir) {
-    set("mapred.working.dir", dir);
+  public void setWorkingDirectory(Path dir) {
+    dir = new Path(getWorkingDirectory(), dir);
+    set("mapred.working.dir", dir.toString());
   }
   
   /**
    * Get the current working directory for the default file system.
    * @return the directory name
    */
-  public String getWorkingDirectory() {
-    return get("mapred.working.dir"); 
+  public Path getWorkingDirectory() {
+    String name = get("mapred.working.dir");
+    if (name != null) {
+      return new Path(name);
+    } else {
+      try {
+        Path dir = FileSystem.get(this).getWorkingDirectory();
+        set("mapred.working.dir", dir.toString());
+        return dir;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
   }
   
-  public File getOutputDir() { 
+  /** @deprecated Call {@link #getOutputPath()} instead.*/
+  public File getOutputDir() { return new File(getOutputPath().toString()); }
+
+  public Path getOutputPath() { 
     String name = get("mapred.output.dir");
-    return name == null ? null: new File(name);
+    return name == null ? null: new Path(name);
   }
 
-  public void setOutputDir(File dir) { set("mapred.output.dir", dir); }
+  /** @deprecated Call {@link #setOutputPath(Path)} instead.*/
+  public void setOutputDir(File dir) {
+    setOutputPath(new Path(dir.toString()));
+  }
+
+  public void setOutputPath(Path dir) {
+    dir = new Path(getWorkingDirectory(), dir);
+    set("mapred.output.dir", dir);
+  }
 
   public InputFormat getInputFormat() {
     return (InputFormat)newInstance(getClass("mapred.input.format.class",

+ 16 - 15
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -35,8 +35,8 @@ class JobInProgress {
 
     JobProfile profile;
     JobStatus status;
-    File localJobFile = null;
-    File localJarFile = null;
+    Path localJobFile = null;
+    Path localJarFile = null;
 
     TaskInProgress maps[] = new TaskInProgress[0];
     TaskInProgress reduces[] = new TaskInProgress[0];
@@ -52,6 +52,8 @@ class JobInProgress {
     private JobConf conf;
     boolean tasksInited = false;
 
+    private LocalFileSystem localFs;
+  
     /**
      * Create a JobInProgress with the given job file, plus a handle
      * to the tracker.
@@ -63,22 +65,22 @@ class JobInProgress {
         this.jobtracker = jobtracker;
         this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.PREP);
         this.startTime = System.currentTimeMillis();
+        this.localFs = (LocalFileSystem)FileSystem.getNamed("local", default_conf);
 
         JobConf default_job_conf = new JobConf(default_conf);
-        this.localJobFile = default_job_conf.getLocalFile(JobTracker.SUBDIR, 
-            jobid + ".xml");
-        this.localJarFile = default_job_conf.getLocalFile(JobTracker.SUBDIR, 
-            jobid + ".jar");
+        this.localJobFile = default_job_conf.getLocalPath(JobTracker.SUBDIR 
+                                                          +"/"+jobid + ".xml");
+        this.localJarFile = default_job_conf.getLocalPath(JobTracker.SUBDIR
+                                                          +"/"+ jobid + ".jar");
         FileSystem fs = FileSystem.get(default_conf);
-        fs.copyToLocalFile(new File(jobFile), localJobFile);
-
+        fs.copyToLocalFile(new Path(jobFile), localJobFile);
         conf = new JobConf(localJobFile);
         this.profile = new JobProfile(conf.getUser(), jobid, jobFile, url,
                                       conf.getJobName());
         String jarFile = conf.getJar();
         if (jarFile != null) {
-          fs.copyToLocalFile(new File(jarFile), localJarFile);
-          conf.setJar(localJarFile.getCanonicalPath());
+          fs.copyToLocalFile(new Path(jarFile), localJarFile);
+          conf.setJar(localJarFile.toString());
         }
 
         this.numMapTasks = conf.getNumMapTasks();
@@ -107,7 +109,7 @@ class JobInProgress {
         if (ifClassName != null && localJarFile != null) {
           try {
             ClassLoader loader =
-              new URLClassLoader(new URL[]{ localJarFile.toURL() });
+              new URLClassLoader(new URL[]{ localFs.pathToFile(localJarFile).toURL() });
             Class inputFormatClass = loader.loadClass(ifClassName);
             inputFormat = (InputFormat)inputFormatClass.newInstance();
           } catch (Exception e) {
@@ -152,7 +154,7 @@ class JobInProgress {
         // Obtain some tasktracker-cache information for the map task splits.
         //
         for (int i = 0; i < maps.length; i++) {
-            String hints[][] = fs.getFileCacheHints(splits[i].getFile(), splits[i].getStart(), splits[i].getLength());
+            String hints[][] = fs.getFileCacheHints(splits[i].getPath(), splits[i].getStart(), splits[i].getLength());
             cachedHints.put(maps[i].getTIPId(), hints);
         }
 
@@ -165,7 +167,7 @@ class JobInProgress {
      * prefetches and caches a lot of these hints.  If the hint is
      * not available, then we pass it through to the filesystem.
      */
-    String[][] getFileCacheHints(String tipID, File f, long start, long len) throws IOException {
+    String[][] getFileCacheHints(String tipID, Path f, long start, long len) throws IOException {
         String results[][] = (String[][]) cachedHints.get(tipID);
         if (tipID == null) {
             FileSystem fs = FileSystem.get(conf);
@@ -471,7 +473,6 @@ class JobInProgress {
     synchronized void garbageCollect() {
       try {
         // Definitely remove the local-disk copy of the job file
-        FileSystem localFs = FileSystem.getNamed("local", conf);
         if (localJobFile != null) {
             localFs.delete(localJobFile);
             localJobFile = null;
@@ -484,7 +485,7 @@ class JobInProgress {
         // JobClient always creates a new directory with job files
         // so we remove that directory to cleanup
         FileSystem fs = FileSystem.get(conf);
-        fs.delete(new File(profile.getJobFile()).getParentFile());
+        fs.delete(new Path(profile.getJobFile()).getParent());
 
       } catch (IOException e) {
         LOG.warning("Error cleaning up "+profile.getJobId()+": "+e);

+ 2 - 2
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -300,7 +300,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     // the files when we're done with the job.
     static final String SUBDIR = "jobTracker";
     FileSystem fs;
-    File systemDir;
+    Path systemDir;
     private Configuration conf;
 
     /**
@@ -323,7 +323,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         JobConf jobConf = new JobConf(conf);
         this.systemDir = jobConf.getSystemDir();
         this.fs = FileSystem.get(conf);
-        FileUtil.fullyDelete(fs, systemDir);
+        fs.delete(systemDir);
         fs.mkdirs(systemDir);
 
         // Same with 'localDir' except it's always on the local disk.

+ 8 - 18
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -45,7 +45,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
     private ArrayList mapIds = new ArrayList();
     private MapOutputFile mapoutputFile;
     private JobProfile profile;
-    private File localFile;
+    private Path localFile;
     private FileSystem localFs;
 
     public Job(String file, Configuration conf) throws IOException {
@@ -54,10 +54,10 @@ class LocalJobRunner implements JobSubmissionProtocol {
       this.mapoutputFile = new MapOutputFile();
       this.mapoutputFile.setConf(conf);
 
-      this.localFile = new JobConf(conf).getLocalFile("localRunner", id+".xml");
+      this.localFile = new JobConf(conf).getLocalPath("localRunner/"+id+".xml");
       this.localFs = FileSystem.getNamed("local", conf);
 
-      fs.copyToLocalFile(new File(file), localFile);
+      fs.copyToLocalFile(new Path(file), localFile);
       this.job = new JobConf(localFile);
       profile = new JobProfile(job.getUser(), id, file, 
                                "http://localhost:8080/", job.getJobName());
@@ -73,18 +73,10 @@ class LocalJobRunner implements JobSubmissionProtocol {
       return profile;
     }
     
-    private void setWorkingDirectory(JobConf conf, FileSystem fs) {
-      String dir = conf.getWorkingDirectory();
-      if (dir != null) {
-        fs.setWorkingDirectory(new File(dir));
-      }
-    }
-    
     public void run() {
       try {
         // split input into minimum number of splits
         FileSplit[] splits;
-        setWorkingDirectory(job, fs);
         splits = job.getInputFormat().getSplits(fs, job, 1);
 
         
@@ -103,9 +95,9 @@ class LocalJobRunner implements JobSubmissionProtocol {
         String reduceId = "reduce_" + newId();
         for (int i = 0; i < mapIds.size(); i++) {
           String mapId = (String)mapIds.get(i);
-          File mapOut = this.mapoutputFile.getOutputFile(mapId, 0);
-          File reduceIn = this.mapoutputFile.getInputFile(mapId, reduceId);
-          reduceIn.getParentFile().mkdirs();
+          Path mapOut = this.mapoutputFile.getOutputFile(mapId, 0);
+          Path reduceIn = this.mapoutputFile.getInputFile(mapId, reduceId);
+          localFs.mkdirs(reduceIn.getParent());
           if (!localFs.rename(mapOut, reduceIn))
             throw new IOException("Couldn't rename " + mapOut);
           this.mapoutputFile.removeAll(mapId);
@@ -116,9 +108,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
         for (int i = 0; i < mapIds.size(); i++) {
             mapDependencies[i][0] = (String) mapIds.get(i);
         }
-        setWorkingDirectory(job, fs);
-        ReduceTask reduce = new ReduceTask(file, reduceId,
-            mapDependencies,0);
+        ReduceTask reduce = new ReduceTask(file, reduceId, mapDependencies,0);
         reduce.setConf(job);
         reduce_tasks += 1;
         reduce.run(job, this);
@@ -133,7 +123,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
 
       } finally {
         try {
-          fs.delete(new File(file).getParentFile()); // delete submit dir
+          fs.delete(new Path(file).getParent());  // delete submit dir
           localFs.delete(localFile);              // delete local copy
         } catch (IOException e) {
           LOG.warning("Error cleaning up "+id+": "+e);

+ 13 - 4
src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java

@@ -17,10 +17,11 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
+import java.io.File;                              // deprecated
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.WritableComparable;
@@ -33,7 +34,7 @@ public class MapFileOutputFormat extends OutputFormatBase {
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
-    File file = new File(job.getOutputDir(), name);
+    Path file = new Path(job.getOutputPath(), name);
 
     final MapFile.Writer out =
       new MapFile.Writer(fs, file.toString(),
@@ -53,10 +54,18 @@ public class MapFileOutputFormat extends OutputFormatBase {
       };
   }
 
+  /** @deprecated Call {@link #getReaders(FileSystem, Path, Configuration)}. */
+  public static MapFile.Reader[] getReaders(FileSystem fs, File dir,
+                                            Configuration conf)
+    throws IOException {
+    return getReaders(fs, new Path(dir.toString()), conf);
+  }
+
   /** Open the output generated by this format. */
-  public static MapFile.Reader[] getReaders(FileSystem fs, File dir, Configuration conf)
+  public static MapFile.Reader[] getReaders(FileSystem fs, Path dir,
+                                            Configuration conf)
     throws IOException {
-    File[] names = fs.listFiles(dir);
+    Path[] names = fs.listPaths(dir);
     
     // sort names, so that hash partitioning works
     Arrays.sort(names);

+ 16 - 12
src/java/org/apache/hadoop/mapred/MapOutputFile.java

@@ -57,24 +57,24 @@ class MapOutputFile implements Writable, Configurable {
    * @param mapTaskId a map task id
    * @param partition a reduce partition
    */
-  public File getOutputFile(String mapTaskId, int partition)
+  public Path getOutputFile(String mapTaskId, int partition)
     throws IOException {
-    return this.jobConf.getLocalFile(mapTaskId, "part-"+partition+".out");
+    return this.jobConf.getLocalPath(mapTaskId+"/part-"+partition+".out");
   }
 
   /** Create a local reduce input file name.
    * @param mapTaskId a map task id
    * @param reduceTaskId a reduce task id
    */
-  public File getInputFile(String mapTaskId, String reduceTaskId)
+  public Path getInputFile(String mapTaskId, String reduceTaskId)
     throws IOException {
-    return this.jobConf.getLocalFile(reduceTaskId, mapTaskId+".out");
+    return this.jobConf.getLocalPath(reduceTaskId+"/"+mapTaskId+".out");
   }
-  public File getInputFile(String mapTaskIds[], String reduceTaskId)
+  public Path getInputFile(String mapTaskIds[], String reduceTaskId)
     throws IOException {
     for (int i = 0; i < mapTaskIds.length; i++) {
-      File file = jobConf.getLocalFile(reduceTaskId, mapTaskIds[i]+".out");
-      if (file.exists())
+      Path file = jobConf.getLocalPath(reduceTaskId+"/"+mapTaskIds[i]+".out");
+      if (getLocalFs().exists(file))
         return file;
     }
     throw new IOException("Input file not found!");
@@ -103,17 +103,21 @@ class MapOutputFile implements Writable, Configurable {
     this.partition = partition;
   }
 
+  private FileSystem getLocalFs() throws IOException {
+    return FileSystem.getNamed("local", this.jobConf);
+  }
+
   public void write(DataOutput out) throws IOException {
     UTF8.writeString(out, mapTaskId);
     UTF8.writeString(out, reduceTaskId);
     out.writeInt(partition);
     
-    File file = getOutputFile(mapTaskId, partition);
+    Path file = getOutputFile(mapTaskId, partition);
     FSDataInputStream in = null;
     try {
       // write the length-prefixed file content to the wire
-      out.writeLong(file.length());
-      in = FileSystem.getNamed("local", this.jobConf).open(file);
+      out.writeLong(getLocalFs().getLength(file));
+      in = getLocalFs().open(file);
     } catch (FileNotFoundException e) {
       TaskTracker.LOG.log(Level.SEVERE, "Can't open map output:" + file, e);
       ((MapOutputServer)Server.get()).getTaskTracker().mapOutputLost(mapTaskId);
@@ -146,11 +150,11 @@ class MapOutputFile implements Writable, Configurable {
     ProgressReporter reporter = (ProgressReporter)REPORTERS.get();
 
     // read the length-prefixed file content into a local file
-    File file = getInputFile(mapTaskId, reduceTaskId);
+    Path file = getInputFile(mapTaskId, reduceTaskId);
     long length = in.readLong();
     float progPerByte = 1.0f / length;
     long unread = length;
-    FSDataOutputStream out = FileSystem.getNamed("local", this.jobConf).create(file);
+    FSDataOutputStream out = getLocalFs().create(file);
     try {
       byte[] buffer = new byte[8192];
       while (unread > 0) {

+ 1 - 1
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -76,7 +76,7 @@ class MapTask extends Task {
       for (int i = 0; i < partitions; i++) {
         outs[i] =
           new SequenceFile.Writer(FileSystem.getNamed("local", job),
-                                  this.mapOutputFile.getOutputFile(getTaskId(), i).toString(),
+                                  this.mapOutputFile.getOutputFile(getTaskId(), i),
                                   job.getOutputKeyClass(),
                                   job.getOutputValueClass());
       }

+ 2 - 2
src/java/org/apache/hadoop/mapred/OutputFormatBase.java

@@ -17,9 +17,9 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 /** A base class for {@link OutputFormat}. */
 public abstract class OutputFormatBase implements OutputFormat {
@@ -29,7 +29,7 @@ public abstract class OutputFormatBase implements OutputFormat {
 
   public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
     // Ensure that the output directory is set and not already there
-    File outDir = job.getOutputDir();
+    Path outDir = job.getOutputPath();
     if (outDir == null && job.getNumReduceTasks() != 0) {
       throw new IOException("Output directory not set in JobConf.");
     }

+ 7 - 8
src/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -184,7 +184,7 @@ class ReduceTask extends Task {
     copyPhase.complete();                         // copy is already complete
 
     // open a file to collect map output
-    String file = job.getLocalFile(getTaskId(), "all.1").toString();
+    Path file = job.getLocalPath(getTaskId()+Path.SEPARATOR+"all.1");
     SequenceFile.Writer writer =
       new SequenceFile.Writer(lfs, file, keyClass, valueClass);
     try {
@@ -196,14 +196,13 @@ class ReduceTask extends Task {
       DataOutputBuffer buffer = new DataOutputBuffer();
 
       for (int i = 0; i < mapTaskIds.length; i++) {
-        File partFile =
+        Path partFile =
           this.mapOutputFile.getInputFile(mapTaskIds[i], getTaskId());
         float progPerByte = 1.0f / lfs.getLength(partFile);
         Progress phase = appendPhase.phase();
         phase.setStatus(partFile.toString());
 
-        SequenceFile.Reader in =
-          new SequenceFile.Reader(lfs, partFile.toString(), job);
+        SequenceFile.Reader in = new SequenceFile.Reader(lfs, partFile, job);
         try {
           int keyLen;
           while((keyLen = in.next(buffer)) > 0) {
@@ -241,7 +240,7 @@ class ReduceTask extends Task {
       };
     sortProgress.setName("Sort progress reporter for task "+getTaskId());
 
-    String sortedFile = job.getLocalFile(getTaskId(), "all.2").toString();
+    Path sortedFile = job.getLocalPath(getTaskId()+Path.SEPARATOR+"all.2");
 
     WritableComparator comparator = job.getOutputKeyComparator();
     
@@ -252,7 +251,7 @@ class ReduceTask extends Task {
       SequenceFile.Sorter sorter =
         new SequenceFile.Sorter(lfs, comparator, valueClass, job);
       sorter.sort(file, sortedFile);              // sort
-      lfs.delete(new File(file));                 // remove unsorted
+      lfs.delete(file);                           // remove unsorted
 
     } finally {
       sortComplete = true;
@@ -275,7 +274,7 @@ class ReduceTask extends Task {
     // apply reduce function
     SequenceFile.Reader in = new SequenceFile.Reader(lfs, sortedFile, job);
     Reporter reporter = getReporter(umbilical, getProgress());
-    long length = lfs.getLength(new File(sortedFile));
+    long length = lfs.getLength(sortedFile);
     try {
       ValuesIterator values = new ValuesIterator(in, length, comparator,
                                                  umbilical);
@@ -287,7 +286,7 @@ class ReduceTask extends Task {
     } finally {
       reducer.close();
       in.close();
-      lfs.delete(new File(sortedFile));           // remove sorted
+      lfs.delete(sortedFile);                     // remove sorted
       out.close(reporter);
     }
     done(umbilical);

+ 6 - 6
src/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java

@@ -17,9 +17,9 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
@@ -31,14 +31,14 @@ public class SequenceFileInputFormat extends InputFormatBase {
     setMinSplitSize(SequenceFile.SYNC_INTERVAL);
   }
 
-  protected File[] listFiles(FileSystem fs, JobConf job)
+  protected Path[] listPaths(FileSystem fs, JobConf job)
     throws IOException {
 
-    File[] files = super.listFiles(fs, job);
+    Path[] files = super.listPaths(fs, job);
     for (int i = 0; i < files.length; i++) {
-      File file = files[i];
-      if (file.isDirectory()) {                   // it's a MapFile
-        files[i] = new File(file, MapFile.DATA_FILE_NAME); // use the data file
+      Path file = files[i];
+      if (fs.isDirectory(file)) {                 // it's a MapFile
+        files[i] = new Path(file, MapFile.DATA_FILE_NAME); // use the data file
       }
     }
     return files;

+ 13 - 6
src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java

@@ -17,10 +17,11 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
+import java.io.File;                              // deprecated
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.WritableComparable;
@@ -33,10 +34,10 @@ public class SequenceFileOutputFormat extends OutputFormatBase {
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
-    File file = new File(job.getOutputDir(), name);
+    Path file = new Path(job.getOutputPath(), name);
 
     final SequenceFile.Writer out =
-      new SequenceFile.Writer(fs, file.toString(),
+      new SequenceFile.Writer(fs, file,
                               job.getOutputKeyClass(),
                               job.getOutputValueClass(),
                               job.getBoolean("mapred.output.compress", false));
@@ -53,18 +54,24 @@ public class SequenceFileOutputFormat extends OutputFormatBase {
       };
   }
 
+  /** @deprecated Call {@link #getReaders(Configuration, Path)} instead. */
+  public static SequenceFile.Reader[] getReaders(Configuration conf, File dir) 
+    throws IOException {
+    return getReaders(conf, new Path(dir.toString()));
+  }
+
   /** Open the output generated by this format. */
-  public static SequenceFile.Reader[] getReaders(Configuration conf, File dir)
+  public static SequenceFile.Reader[] getReaders(Configuration conf, Path dir)
     throws IOException {
     FileSystem fs = FileSystem.get(conf);
-    File[] names = fs.listFiles(dir);
+    Path[] names = fs.listPaths(dir);
     
     // sort names, so that hash partitioning works
     Arrays.sort(names);
     
     SequenceFile.Reader[] parts = new SequenceFile.Reader[names.length];
     for (int i = 0; i < names.length; i++) {
-      parts[i] = new SequenceFile.Reader(fs, names[i].toString(), conf);
+      parts[i] = new SequenceFile.Reader(fs, names[i], conf);
     }
     return parts;
   }

+ 1 - 1
src/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java

@@ -33,7 +33,7 @@ public class SequenceFileRecordReader implements RecordReader {
   public SequenceFileRecordReader(Configuration conf, FileSplit split)
     throws IOException {
     FileSystem fs = FileSystem.get(conf);
-    this.in = new SequenceFile.Reader(fs, split.getFile().toString(), conf);
+    this.in = new SequenceFile.Reader(fs, split.getPath(), conf);
     this.end = split.getStart() + split.getLength();
 
     if (split.getStart() > in.getPosition())

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskInProgress.java

@@ -372,7 +372,7 @@ class TaskInProgress {
             try {
                 if (isMapTask()) {
                     if (hints == null) {
-                        hints = job.getFileCacheHints(getTIPId(), split.getFile(), split.getStart(), split.getLength());
+                        hints = job.getFileCacheHints(getTIPId(), split.getPath(), split.getStart(), split.getLength());
                     }
                     if (hints != null) {
                         for (int i = 0; i < hints.length; i++) {

+ 16 - 20
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -379,7 +379,7 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol, MapOutpu
             this.task = task;
             this.lastProgressReport = System.currentTimeMillis();
             this.jobConf = new JobConf(conf);
-            this.jobConf.deleteLocalFiles(SUBDIR + File.separator + task.getTaskId());
+            this.jobConf.deleteLocalFiles(SUBDIR + "/" + task.getTaskId());
             localizeTask(task);
         }
 
@@ -388,23 +388,23 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol, MapOutpu
          * So here, edit the Task's fields appropriately.
          */
         void localizeTask(Task t) throws IOException {
-            File localJobFile =
-              this.jobConf.getLocalFile(SUBDIR+File.separator+t.getTaskId(), "job.xml");
-            File localJarFile =
-              this.jobConf.getLocalFile(SUBDIR+File.separator+t.getTaskId(), "job.jar");
+            Path localJobFile =
+              this.jobConf.getLocalPath(SUBDIR+"/"+t.getTaskId()+"/"+"job.xml");
+            Path localJarFile =
+              this.jobConf.getLocalPath(SUBDIR+"/"+t.getTaskId()+"/"+"job.jar");
 
             String jobFile = t.getJobFile();
-            fs.copyToLocalFile(new File(jobFile), localJobFile);
-            t.setJobFile(localJobFile.getCanonicalPath());
+            fs.copyToLocalFile(new Path(jobFile), localJobFile);
+            t.setJobFile(localJobFile.toString());
 
             JobConf jc = new JobConf(localJobFile);
             String jarFile = jc.getJar();
             if (jarFile != null) {
-              fs.copyToLocalFile(new File(jarFile), localJarFile);
-              jc.setJar(localJarFile.getCanonicalPath());
+              fs.copyToLocalFile(new Path(jarFile), localJarFile);
+              jc.setJar(localJarFile.toString());
 
-              BufferedOutputStream out =
-                new BufferedOutputStream(new FileOutputStream(localJobFile));
+              FileSystem localFs = FileSystem.getNamed("local", fConf);
+              OutputStream out = localFs.create(localJobFile);
               try {
                 jc.write(out);
               } finally {
@@ -569,7 +569,7 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol, MapOutpu
                 runner.close();
             } catch (IOException ie) {
             }
-            this.jobConf.deleteLocalFiles(SUBDIR + File.separator + task.getTaskId());
+            this.jobConf.deleteLocalFiles(SUBDIR + "/" + task.getTaskId());
         }
     }
 
@@ -695,18 +695,14 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol, MapOutpu
           Task task = umbilical.getTask(taskid);
           JobConf job = new JobConf(task.getJobFile());
 
-          conf.addFinalResource(new File(task.getJobFile()));
+          conf.addFinalResource(new Path(task.getJobFile()));
 
           startPinging(umbilical, taskid);        // start pinging parent
 
           try {
-              // If the user set a working directory, use it
-              String workDir = job.getWorkingDirectory();
-              if (workDir != null) {
-                FileSystem file_sys = FileSystem.get(job);
-                file_sys.setWorkingDirectory(new File(workDir));
-              }
-              task.run(job, umbilical);           // run the task
+            // use job-specified working directory
+            FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory());
+            task.run(job, umbilical);             // run the task
           } catch (FSError e) {
             LOG.log(Level.SEVERE, "FSError from child", e);
             umbilical.fsError(e.getMessage());

+ 1 - 1
src/java/org/apache/hadoop/mapred/TextInputFormat.java

@@ -40,7 +40,7 @@ public class TextInputFormat extends InputFormatBase {
     final long end = start + split.getLength();
 
     // open the file and seek to the start of the split
-    final FSDataInputStream in = fs.open(split.getFile());
+    final FSDataInputStream in = fs.open(split.getPath());
     
     if (start != 0) {
       in.seek(start-1);

+ 2 - 2
src/java/org/apache/hadoop/mapred/TextOutputFormat.java

@@ -17,9 +17,9 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
 
 import org.apache.hadoop.io.WritableComparable;
@@ -31,7 +31,7 @@ public class TextOutputFormat extends OutputFormatBase {
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
-    File file = new File(job.getOutputDir(), name);
+    Path file = new Path(job.getOutputPath(), name);
 
     final FSDataOutputStream out = fs.create(file);
 

+ 1 - 1
src/test/org/apache/hadoop/dfs/ClusterTestDFS.java

@@ -135,7 +135,7 @@ public class ClusterTestDFS extends TestCase implements FSConstants {
   protected void prepareTempFileSpace() {
     if (baseDir.exists()) {
       try { // start from a blank slate
-        FileUtil.fullyDelete(baseDir, conf);
+        FileUtil.fullyDelete(baseDir);
       } catch (Exception ignored) {
       }
     }

+ 12 - 11
src/test/org/apache/hadoop/dfs/TestLocalDFS.java

@@ -4,6 +4,7 @@ import junit.framework.TestCase;
 import java.io.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 /**
  * This class tests the DFS class via the FileSystem interface in a single node
@@ -12,13 +13,13 @@ import org.apache.hadoop.fs.FileSystem;
  */
 public class TestLocalDFS extends TestCase {
 
-  private void writeFile(FileSystem fileSys, File name) throws IOException {
+  private void writeFile(FileSystem fileSys, Path name) throws IOException {
     DataOutputStream stm = fileSys.create(name);
     stm.writeBytes("oom");
     stm.close();
   }
   
-  private void readFile(FileSystem fileSys, File name) throws IOException {
+  private void readFile(FileSystem fileSys, Path name) throws IOException {
     DataInputStream stm = fileSys.open(name);
     byte[] buffer = new byte[4];
     int bytesRead = stm.read(buffer, 0 ,4);
@@ -26,7 +27,7 @@ public class TestLocalDFS extends TestCase {
     stm.close();
   }
   
-  private void cleanupFile(FileSystem fileSys, File name) throws IOException {
+  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     fileSys.delete(name);
     assertTrue(!fileSys.exists(name));
@@ -40,22 +41,22 @@ public class TestLocalDFS extends TestCase {
     MiniDFSCluster cluster = new MiniDFSCluster(65312, conf);
     FileSystem fileSys = cluster.getFileSystem();
     try {
-      File orig_path = fileSys.getWorkingDirectory();
+      Path orig_path = fileSys.getWorkingDirectory();
       assertTrue(orig_path.isAbsolute());
-      File file1 = new File("somewhat/random.txt");
+      Path file1 = new Path("somewhat/random.txt");
       writeFile(fileSys, file1);
-      assertTrue(fileSys.exists(new File(orig_path, file1.getPath())));
+      assertTrue(fileSys.exists(new Path(orig_path, file1.toString())));
       fileSys.delete(file1);
-      File subdir1 = new File("/somewhere").getAbsoluteFile();
+      Path subdir1 = new Path("/somewhere");
       fileSys.setWorkingDirectory(subdir1);
       writeFile(fileSys, file1);
-      cleanupFile(fileSys, new File(subdir1, file1.getPath()));
-      File subdir2 = new File("else");
+      cleanupFile(fileSys, new Path(subdir1, file1.toString()));
+      Path subdir2 = new Path("else");
       fileSys.setWorkingDirectory(subdir2);
       writeFile(fileSys, file1);
       readFile(fileSys, file1);
-      cleanupFile(fileSys, new File(new File(subdir1, subdir2.getPath()),
-                                     file1.getPath()));
+      cleanupFile(fileSys, new Path(new Path(subdir1, subdir2.toString()),
+                                     file1.toString()));
     } finally {
       fileSys.close();
       cluster.shutdown();

+ 16 - 17
src/test/org/apache/hadoop/fs/TestFileSystem.java

@@ -36,10 +36,10 @@ public class TestFileSystem extends TestCase {
   private static final int SEEKS_PER_FILE = 4;
 
   private static String ROOT = System.getProperty("test.build.data","fs_test");
-  private static File CONTROL_DIR = new File(ROOT, "fs_control");
-  private static File WRITE_DIR = new File(ROOT, "fs_write");
-  private static File READ_DIR = new File(ROOT, "fs_read");
-  private static File DATA_DIR = new File(ROOT, "fs_data");
+  private static Path CONTROL_DIR = new Path(ROOT, "fs_control");
+  private static Path WRITE_DIR = new Path(ROOT, "fs_write");
+  private static Path READ_DIR = new Path(ROOT, "fs_read");
+  private static Path DATA_DIR = new Path(ROOT, "fs_data");
 
   public void testFs() throws Exception {
     testFs(10 * MEGA, 100, 0);
@@ -67,13 +67,12 @@ public class TestFileSystem extends TestCase {
 
     LOG.info("creating control file: "+megaBytes+" bytes, "+numFiles+" files");
 
-    File controlFile = new File(CONTROL_DIR, "files");
+    Path controlFile = new Path(CONTROL_DIR, "files");
     fs.delete(controlFile);
     Random random = new Random(seed);
 
     SequenceFile.Writer writer =
-      new SequenceFile.Writer(fs, controlFile.toString(),
-                              UTF8.class, LongWritable.class);
+      new SequenceFile.Writer(fs, controlFile, UTF8.class, LongWritable.class);
 
     long totalSize = 0;
     long maxSize = ((megaBytes / numFiles) * 2) + 1;
@@ -135,7 +134,7 @@ public class TestFileSystem extends TestCase {
       reporter.setStatus("creating " + name);
 
       // write to temp file initially to permit parallel execution
-      File tempFile = new File(DATA_DIR, name+suffix);
+      Path tempFile = new Path(DATA_DIR, name+suffix);
       OutputStream out = fs.create(tempFile);
 
       long written = 0;
@@ -156,7 +155,7 @@ public class TestFileSystem extends TestCase {
         out.close();
       }
       // rename to final location
-      fs.rename(tempFile, new File(DATA_DIR, name));
+      fs.rename(tempFile, new Path(DATA_DIR, name));
 
       collector.collect(new UTF8("bytes"), new LongWritable(written));
 
@@ -177,7 +176,7 @@ public class TestFileSystem extends TestCase {
     JobConf job = new JobConf(conf);
     job.setBoolean("fs.test.fastCheck", fastCheck);
 
-    job.setInputDir(CONTROL_DIR);
+    job.setInputPath(CONTROL_DIR);
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(LongWritable.class);
@@ -185,7 +184,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(WriteMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputDir(WRITE_DIR);
+    job.setOutputPath(WRITE_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);
@@ -227,7 +226,7 @@ public class TestFileSystem extends TestCase {
       reporter.setStatus("opening " + name);
 
       DataInputStream in =
-        new DataInputStream(fs.open(new File(DATA_DIR, name)));
+        new DataInputStream(fs.open(new Path(DATA_DIR, name)));
 
       long read = 0;
       try {
@@ -273,7 +272,7 @@ public class TestFileSystem extends TestCase {
     job.setBoolean("fs.test.fastCheck", fastCheck);
 
 
-    job.setInputDir(CONTROL_DIR);
+    job.setInputPath(CONTROL_DIR);
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(LongWritable.class);
@@ -281,7 +280,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(ReadMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputDir(READ_DIR);
+    job.setOutputPath(READ_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);
@@ -323,7 +322,7 @@ public class TestFileSystem extends TestCase {
 
       reporter.setStatus("opening " + name);
 
-      FSDataInputStream in = fs.open(new File(DATA_DIR, name));
+      FSDataInputStream in = fs.open(new Path(DATA_DIR, name));
         
       try {
         for (int i = 0; i < SEEKS_PER_FILE; i++) {
@@ -368,7 +367,7 @@ public class TestFileSystem extends TestCase {
     JobConf job = new JobConf(conf);
     job.setBoolean("fs.test.fastCheck", fastCheck);
 
-    job.setInputDir(CONTROL_DIR);
+    job.setInputPath(CONTROL_DIR);
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(LongWritable.class);
@@ -376,7 +375,7 @@ public class TestFileSystem extends TestCase {
     job.setMapperClass(SeekMapper.class);
     job.setReducerClass(LongSumReducer.class);
 
-    job.setOutputDir(READ_DIR);
+    job.setOutputPath(READ_DIR);
     job.setOutputKeyClass(UTF8.class);
     job.setOutputValueClass(LongWritable.class);
     job.setNumReduceTasks(1);

+ 10 - 15
src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -10,13 +10,13 @@ import junit.framework.*;
  */
 public class TestLocalFileSystem extends TestCase {
 
-  private void writeFile(FileSystem fs, File name) throws IOException {
+  private void writeFile(FileSystem fs, Path name) throws IOException {
     FSDataOutputStream stm = fs.create(name);
     stm.writeBytes("42\n");
     stm.close();
   }
   
-  private void cleanupFile(FileSystem fs, File name) throws IOException {
+  private void cleanupFile(FileSystem fs, Path name) throws IOException {
     assertTrue(fs.exists(name));
     fs.delete(name);
     assertTrue(!fs.exists(name));
@@ -28,9 +28,8 @@ public class TestLocalFileSystem extends TestCase {
   public void testWorkingDirectory() throws IOException {
     Configuration conf = new Configuration();
     FileSystem fileSys = FileSystem.getNamed("local", conf);
-    File origDir = fileSys.getWorkingDirectory();
-    File subdir = new File("build/test/data/work-dir/new subdir");
-    File subdirAbsolute = subdir.getAbsoluteFile();
+    Path origDir = fileSys.getWorkingDirectory();
+    Path subdir = new Path("build/test/data/work-dir/new subdir");
     try {
       // make sure it doesn't already exist
       assertTrue(!fileSys.exists(subdir));
@@ -41,33 +40,29 @@ public class TestLocalFileSystem extends TestCase {
       fileSys.setWorkingDirectory(subdir);
       
       // create a directory and check for it
-      File dir1 = new File("dir1");
-      File dir1Absolute = new File(subdirAbsolute, dir1.getPath());
+      Path dir1 = new Path("dir1");
       fileSys.mkdirs(dir1);
       assertTrue(fileSys.isDirectory(dir1));
-      assertTrue(fileSys.isDirectory(dir1Absolute));
       
       // delete the directory and make sure it went away
       fileSys.delete(dir1);
       assertTrue(!fileSys.exists(dir1));
-      assertTrue(!fileSys.exists(dir1Absolute));
       
       // create files and manipulate them.
-      File file1 = new File("file1");
-      File file2 = new File("sub/file2");
-      File file2_abs = new File(subdirAbsolute, file2.getPath());
+      Path file1 = new Path("file1");
+      Path file2 = new Path("sub/file2");
       writeFile(fileSys, file1);
       fileSys.copyFromLocalFile(file1, file2);
       assertTrue(fileSys.exists(file1));
       assertTrue(fileSys.isFile(file1));
-      cleanupFile(fileSys, file2_abs);
+      cleanupFile(fileSys, file2);
       fileSys.copyToLocalFile(file1, file2);
-      cleanupFile(fileSys, file2_abs);
+      cleanupFile(fileSys, file2);
       
       // try a rename
       fileSys.rename(file1, file2);
       assertTrue(!fileSys.exists(file1));
-      assertTrue(fileSys.exists(file2_abs));
+      assertTrue(fileSys.exists(file2));
       fileSys.rename(file2, file1);
       
       // try reading a file

+ 97 - 0
src/test/org/apache/hadoop/fs/TestPath.java

@@ -0,0 +1,97 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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;
+
+import java.util.*;
+import junit.framework.TestCase;
+
+public class TestPath extends TestCase {
+  public void testToString() {
+    toStringTest("/");
+    toStringTest("/foo");
+    toStringTest("/foo/bar");
+    toStringTest("foo");
+    toStringTest("foo/bar");
+    toStringTest("");
+    if (Path.WINDOWS) {
+      toStringTest("c:");
+      toStringTest("c:/");
+      toStringTest("c:foo");
+      toStringTest("c:foo/bar");
+      toStringTest("c:foo/bar");
+      toStringTest("c:/foo/bar");
+    }
+  }
+
+  private void toStringTest(String pathString) {
+    assertEquals(pathString, new Path(pathString).toString());
+  }
+
+  public void testNormalize() {
+    assertEquals("/", new Path("//").toString());
+    assertEquals("/foo", new Path("/foo/").toString());
+    assertEquals("/foo", new Path("/foo/").toString());
+    assertEquals("foo", new Path("foo/").toString());
+    assertEquals("foo", new Path("foo//").toString());
+    assertEquals("foo/bar", new Path("foo//bar").toString());
+    if (Path.WINDOWS) {
+      assertEquals("c:/a/b", new Path("c:\\a\\b").toString());
+    }
+  }
+
+  public void testIsAbsolute() {
+    assertTrue(new Path("/").isAbsolute());
+    assertTrue(new Path("/foo").isAbsolute());
+    assertFalse(new Path("foo").isAbsolute());
+    assertFalse(new Path("foo/bar").isAbsolute());
+    assertFalse(new Path("").isAbsolute());
+    if (Path.WINDOWS) {
+      assertTrue(new Path("c:/a/b").isAbsolute());
+      assertFalse(new Path("c:a/b").isAbsolute());
+    }
+  }
+
+  public void testParent() {
+    assertEquals(new Path("/foo"), new Path("/foo/bar").getParent());
+    assertEquals(new Path("foo"), new Path("foo/bar").getParent());
+    assertEquals(new Path("/"), new Path("/foo").getParent());
+    if (Path.WINDOWS) {
+      assertEquals(new Path("c:/"), new Path("c:/foo").getParent());
+    }
+  }
+
+  public void testChild() {
+    assertEquals(new Path(""), new Path("", ""));
+    assertEquals(new Path("/"), new Path("/", ""));
+    assertEquals(new Path("/"), new Path("", "/"));
+    assertEquals(new Path("/foo"), new Path("/", "foo"));
+    assertEquals(new Path("/foo/bar"), new Path("/foo", "bar"));
+    assertEquals(new Path("/foo/bar/baz"), new Path("/foo/bar", "baz"));
+    assertEquals(new Path("/foo/bar/baz"), new Path("/foo", "bar/baz"));
+    assertEquals(new Path("foo"), new Path("", "foo"));
+    assertEquals(new Path("foo/bar"), new Path("foo", "bar"));
+    assertEquals(new Path("foo/bar/baz"), new Path("foo", "bar/baz"));
+    assertEquals(new Path("foo/bar/baz"), new Path("foo/bar", "baz"));
+    assertEquals(new Path("/foo"), new Path("/bar", "/foo"));
+    if (Path.WINDOWS) {
+      assertEquals(new Path("c:/foo"), new Path("c:/bar", "/foo"));
+      assertEquals(new Path("c:/foo"), new Path("/bar", "c:/foo"));
+      assertEquals(new Path("c:/foo"), new Path("d:/bar", "c:/foo"));
+    }
+  }
+
+}

+ 22 - 20
src/test/org/apache/hadoop/io/TestSequenceFile.java

@@ -38,7 +38,7 @@ public class TestSequenceFile extends TestCase {
     int count = 1024 * 10;
     int megabytes = 1;
     int factor = 5;
-    String file = System.getProperty("test.build.data",".") + "/test.seq";
+    Path file = new Path(System.getProperty("test.build.data",".")+"/test.seq");
  
     int seed = new Random().nextInt();
 
@@ -65,9 +65,9 @@ public class TestSequenceFile extends TestCase {
   }
 
   private static void writeTest(FileSystem fs, int count, int seed,
-                                String file, boolean compress)
+                                Path file, boolean compress)
     throws IOException {
-    new File(file).delete();
+    fs.delete(file);
     LOG.fine("creating with " + count + " records");
     SequenceFile.Writer writer =
       new SequenceFile.Writer(fs, file, RandomDatum.class, RandomDatum.class,
@@ -83,7 +83,7 @@ public class TestSequenceFile extends TestCase {
     writer.close();
   }
 
-  private static void readTest(FileSystem fs, int count, int seed, String file)
+  private static void readTest(FileSystem fs, int count, int seed, Path file)
     throws IOException {
     RandomDatum k = new RandomDatum();
     RandomDatum v = new RandomDatum();
@@ -107,16 +107,16 @@ public class TestSequenceFile extends TestCase {
 
 
   private static void sortTest(FileSystem fs, int count, int megabytes, 
-                               int factor, boolean fast, String file)
+                               int factor, boolean fast, Path file)
     throws IOException {
-    new File(file+".sorted").delete();
+    fs.delete(new Path(file+".sorted"));
     SequenceFile.Sorter sorter = newSorter(fs, fast, megabytes, factor);
     LOG.fine("sorting " + count + " records");
-    sorter.sort(file, file+".sorted");
+    sorter.sort(file, file.suffix(".sorted"));
     LOG.fine("done sorting " + count + " records");
   }
 
-  private static void checkSort(FileSystem fs, int count, int seed, String file)
+  private static void checkSort(FileSystem fs, int count, int seed, Path file)
     throws IOException {
     LOG.fine("sorting " + count + " records in memory for check");
     RandomDatum.Generator generator = new RandomDatum.Generator(seed);
@@ -132,7 +132,8 @@ public class TestSequenceFile extends TestCase {
     RandomDatum k = new RandomDatum();
     RandomDatum v = new RandomDatum();
     Iterator iterator = map.entrySet().iterator();
-    SequenceFile.Reader reader = new SequenceFile.Reader(fs, file + ".sorted", conf);
+    SequenceFile.Reader reader =
+      new SequenceFile.Reader(fs, file.suffix(".sorted"), conf);
     for (int i = 0; i < count; i++) {
       Map.Entry entry = (Map.Entry)iterator.next();
       RandomDatum key = (RandomDatum)entry.getKey();
@@ -151,21 +152,21 @@ public class TestSequenceFile extends TestCase {
   }
 
   private static void mergeTest(FileSystem fs, int count, int seed, 
-                                String file, boolean fast, int factor, 
+                                Path file, boolean fast, int factor, 
                                 int megabytes)
     throws IOException {
 
     LOG.fine("creating "+factor+" files with "+count/factor+" records");
 
     SequenceFile.Writer[] writers = new SequenceFile.Writer[factor];
-    String[] names = new String[factor];
-    String[] sortedNames = new String[factor];
+    Path[] names = new Path[factor];
+    Path[] sortedNames = new Path[factor];
     
     for (int i = 0; i < factor; i++) {
-      names[i] = file+"."+i;
-      sortedNames[i] = names[i] + ".sorted";
-      fs.delete(new File(names[i]));
-      fs.delete(new File(sortedNames[i]));
+      names[i] = file.suffix("."+i);
+      sortedNames[i] = names[i].suffix(".sorted");
+      fs.delete(names[i]);
+      fs.delete(sortedNames[i]);
       writers[i] =
         new SequenceFile.Writer(fs, names[i], RandomDatum.class,RandomDatum.class);
     }
@@ -189,8 +190,9 @@ public class TestSequenceFile extends TestCase {
     }
 
     LOG.fine("merging " + factor + " files with " + count/factor + " records");
-    fs.delete(new File(file+".sorted"));
-    newSorter(fs, fast, megabytes, factor).merge(sortedNames, file+".sorted");
+    fs.delete(new Path(file+".sorted"));
+    newSorter(fs, fast, megabytes, factor)
+      .merge(sortedNames, file.suffix(".sorted"));
   }
 
   private static SequenceFile.Sorter newSorter(FileSystem fs, 
@@ -216,7 +218,7 @@ public class TestSequenceFile extends TestCase {
     boolean fast = false;
     boolean merge = false;
     boolean compress = false;
-    String file = null;
+    Path file = null;
     String usage = "Usage: SequenceFile (-local | -dfs <namenode:port>) [-count N] [-megabytes M] [-factor F] [-nocreate] [-check] [-fast] [-merge] [-compress] file";
     
     if (args.length == 0) {
@@ -247,7 +249,7 @@ public class TestSequenceFile extends TestCase {
               compress = true;
           } else {
               // file is required parameter
-              file = args[i];
+              file = new Path(args[i]);
           }
         }
         LOG.info("count = " + count);

+ 16 - 16
src/test/org/apache/hadoop/mapred/MapredLoadTest.java

@@ -233,14 +233,14 @@ public class MapredLoadTest {
         // Write the answer key to a file.  
         //
         FileSystem fs = FileSystem.get(conf);
-        File testdir = new File("mapred.loadtest");
+        Path testdir = new Path("mapred.loadtest");
         fs.mkdirs(testdir);
 
-        File randomIns = new File(testdir, "genins");
+        Path randomIns = new Path(testdir, "genins");
         fs.mkdirs(randomIns);
 
-        File answerkey = new File(randomIns, "answer.key");
-        SequenceFile.Writer out = new SequenceFile.Writer(fs, answerkey.getPath(), IntWritable.class, IntWritable.class);
+        Path answerkey = new Path(randomIns, "answer.key");
+        SequenceFile.Writer out = new SequenceFile.Writer(fs, answerkey, IntWritable.class, IntWritable.class);
         try {
             for (int i = 0; i < range; i++) {
                 out.append(new IntWritable(i), new IntWritable(dist[i]));
@@ -267,18 +267,18 @@ public class MapredLoadTest {
         // Because there's just one reduce task, we emit a single big
         // file of random numbers.
         //
-        File randomOuts = new File(testdir, "genouts");
+        Path randomOuts = new Path(testdir, "genouts");
         fs.mkdirs(randomOuts);
 
 
         JobConf genJob = new JobConf(conf);
-        genJob.setInputDir(randomIns);
+        genJob.setInputPath(randomIns);
         genJob.setInputKeyClass(IntWritable.class);
         genJob.setInputValueClass(IntWritable.class);
         genJob.setInputFormat(SequenceFileInputFormat.class);
         genJob.setMapperClass(RandomGenMapper.class);
 
-        genJob.setOutputDir(randomOuts);
+        genJob.setOutputPath(randomOuts);
         genJob.setOutputKeyClass(IntWritable.class);
         genJob.setOutputValueClass(IntWritable.class);
         genJob.setOutputFormat(TextOutputFormat.class);
@@ -316,16 +316,16 @@ public class MapredLoadTest {
         // you have multiple reduces at once.
         //
         int intermediateReduces = 10;
-        File intermediateOuts = new File(testdir, "intermediateouts");
+        Path intermediateOuts = new Path(testdir, "intermediateouts");
         fs.mkdirs(intermediateOuts);
         JobConf checkJob = new JobConf(conf);
-        checkJob.setInputDir(randomOuts);
+        checkJob.setInputPath(randomOuts);
         checkJob.setInputKeyClass(LongWritable.class);
         checkJob.setInputValueClass(UTF8.class);
         checkJob.setInputFormat(TextInputFormat.class);
         checkJob.setMapperClass(RandomCheckMapper.class);
 
-        checkJob.setOutputDir(intermediateOuts);
+        checkJob.setOutputPath(intermediateOuts);
         checkJob.setOutputKeyClass(IntWritable.class);
         checkJob.setOutputValueClass(IntWritable.class);
         checkJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -341,16 +341,16 @@ public class MapredLoadTest {
         // But by having a single reduce task here, we end up merging
         // all the files.
         //
-        File finalOuts = new File(testdir, "finalouts");        
+        Path finalOuts = new Path(testdir, "finalouts");        
         fs.mkdirs(finalOuts);
         JobConf mergeJob = new JobConf(conf);
-        mergeJob.setInputDir(intermediateOuts);
+        mergeJob.setInputPath(intermediateOuts);
         mergeJob.setInputKeyClass(IntWritable.class);
         mergeJob.setInputValueClass(IntWritable.class);
         mergeJob.setInputFormat(SequenceFileInputFormat.class);
         mergeJob.setMapperClass(MergeMapper.class);
         
-        mergeJob.setOutputDir(finalOuts);
+        mergeJob.setOutputPath(finalOuts);
         mergeJob.setOutputKeyClass(IntWritable.class);
         mergeJob.setOutputValueClass(IntWritable.class);
         mergeJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -366,8 +366,8 @@ public class MapredLoadTest {
         // in the original key.
         //
         boolean success = true;
-        File recomputedkey = new File(finalOuts, "part-00000");
-        SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey.getPath(), conf);
+        Path recomputedkey = new Path(finalOuts, "part-00000");
+        SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey, conf);
         int totalseen = 0;
         try {
             IntWritable key = new IntWritable();
@@ -407,7 +407,7 @@ public class MapredLoadTest {
         //
         // Write to "results" whether the test succeeded or not.
         //
-        File resultFile = new File(testdir, "results");
+        Path resultFile = new Path(testdir, "results");
         BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fs.create(resultFile)));
         try {
             bw.write("Success=" + success + "\n");

+ 4 - 4
src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java

@@ -34,8 +34,8 @@ public class TestSequenceFileInputFormat extends TestCase {
   public void testFormat() throws Exception {
     JobConf job = new JobConf(conf);
     FileSystem fs = FileSystem.getNamed("local", conf);
-    File dir = new File(System.getProperty("test.build.data",".") + "/mapred");
-    File file = new File(dir, "test.seq");
+    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+    Path file = new Path(dir, "test.seq");
     
     Reporter reporter = new Reporter() {
         public void setStatus(String status) throws IOException {}
@@ -47,7 +47,7 @@ public class TestSequenceFileInputFormat extends TestCase {
 
     fs.delete(dir);
 
-    job.setInputDir(dir);
+    job.setInputPath(dir);
 
     // for a variety of lengths
     for (int length = 0; length < MAX_LENGTH;
@@ -57,7 +57,7 @@ public class TestSequenceFileInputFormat extends TestCase {
 
       // create a file with length entries
       SequenceFile.Writer writer =
-        new SequenceFile.Writer(fs, file.toString(),
+        new SequenceFile.Writer(fs, file,
                                 IntWritable.class, BytesWritable.class);
       try {
         for (int i = 0; i < length; i++) {

+ 3 - 3
src/test/org/apache/hadoop/mapred/TestTextInputFormat.java

@@ -34,8 +34,8 @@ public class TestTextInputFormat extends TestCase {
   public void testFormat() throws Exception {
     JobConf job = new JobConf(conf);
     FileSystem fs = FileSystem.getNamed("local", conf);
-    File dir = new File(System.getProperty("test.build.data",".") + "/mapred");
-    File file = new File(dir, "test.txt");
+    Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
+    Path file = new Path(dir, "test.txt");
 
     Reporter reporter = new Reporter() {
         public void setStatus(String status) throws IOException {}
@@ -46,7 +46,7 @@ public class TestTextInputFormat extends TestCase {
     Random random = new Random(seed);
 
     fs.delete(dir);
-    job.setInputDir(dir);
+    job.setInputPath(dir);
 
     // for a variety of lengths
     for (int length = 0; length < MAX_LENGTH;