Browse Source

HADOOP-2563. Remove deprecated FileSystem::listPaths.
Contributed by lohit vijayarenu.



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@642627 13f79535-47bb-0310-9956-ffa450edef68

Christopher Douglas 17 years ago
parent
commit
92057088e7
24 changed files with 113 additions and 202 deletions
  1. 3 0
      CHANGES.txt
  2. 3 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
  3. 4 25
      src/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  4. 19 45
      src/java/org/apache/hadoop/fs/FileSystem.java
  5. 14 12
      src/java/org/apache/hadoop/fs/FileUtil.java
  6. 11 11
      src/java/org/apache/hadoop/fs/FsShell.java
  7. 2 2
      src/java/org/apache/hadoop/fs/Trash.java
  8. 2 39
      src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
  9. 4 3
      src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
  10. 5 6
      src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java
  11. 7 2
      src/java/org/apache/hadoop/mapred/FileInputFormat.java
  12. 2 1
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  13. 2 1
      src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
  14. 4 3
      src/java/org/apache/hadoop/mapred/Task.java
  15. 0 25
      src/test/org/apache/hadoop/dfs/TestFileStatus.java
  16. 2 2
      src/test/org/apache/hadoop/fs/DistributedFSCheck.java
  17. 3 3
      src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
  18. 5 4
      src/test/org/apache/hadoop/fs/kfs/TestKosmosFileSystem.java
  19. 11 10
      src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java
  20. 2 1
      src/test/org/apache/hadoop/mapred/BigMapOutput.java
  21. 2 2
      src/test/org/apache/hadoop/mapred/SortValidator.java
  22. 2 1
      src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java
  23. 2 2
      src/test/org/apache/hadoop/mapred/pipes/WordCountInputFormat.java
  24. 2 1
      src/webapps/job/jobhistory.jsp

+ 3 - 0
CHANGES.txt

@@ -62,6 +62,9 @@ Trunk (unreleased changes)
     HADOOP-2854. Remove deprecated o.a.h.ipc.Server::getUserInfo().
     HADOOP-2854. Remove deprecated o.a.h.ipc.Server::getUserInfo().
     (lohit vijayarenu via cdouglas)
     (lohit vijayarenu via cdouglas)
 
 
+    HADOOP-2563. Remove deprecated FileSystem::listPaths.
+    (lohit vijayarenu via cdouglas)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)

+ 3 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java

@@ -29,6 +29,7 @@ import java.util.jar.JarOutputStream;
 import java.util.zip.ZipOutputStream;
 import java.util.zip.ZipOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
@@ -132,7 +133,8 @@ public class TestMultipleArchiveFiles extends TestStreaming
         throw new Exception("Job Failed");
         throw new Exception("Job Failed");
       }
       }
       StringBuffer output = new StringBuffer(256);
       StringBuffer output = new StringBuffer(256);
-      Path[] fileList = fileSys.listPaths(new Path(OUTPUT_DIR));
+      Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+                                            new Path(OUTPUT_DIR)));
       for (int i = 0; i < fileList.length; i++){
       for (int i = 0; i < fileList.length; i++){
         BufferedReader bread =
         BufferedReader bread =
           new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));
           new BufferedReader(new InputStreamReader(fileSys.open(fileList[i])));

+ 4 - 25
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -453,28 +453,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     return fs.listStatus(f, DEFAULT_FILTER);
     return fs.listStatus(f, DEFAULT_FILTER);
   }
   }
   
   
-  /** 
-   * Filter raw files in the given pathes using the default checksum filter. 
-   * @param files a list of paths
-   * @return a list of files under the source paths
-   * @exception IOException
-   */
-  @Deprecated
-  public Path[] listPaths(Path[] files) throws IOException {
-    return fs.listPaths(files, DEFAULT_FILTER);
-  }
-
-  /** 
-   * Filter raw files in the given path using the default checksum filter. 
-   * @param f source path
-   * @return a list of files under the source path
-   * @exception IOException
-   */
-  @Deprecated
-  public Path[] listPaths(Path f) throws IOException {
-    return fs.listPaths(f, DEFAULT_FILTER);
-  }
-
   @Override
   @Override
   public boolean mkdirs(Path f) throws IOException {
   public boolean mkdirs(Path f) throws IOException {
     return fs.mkdirs(f);
     return fs.mkdirs(f);
@@ -524,9 +502,10 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         fs.copyToLocalFile(checksumFile, dst);
         fs.copyToLocalFile(checksumFile, dst);
       }
       }
     } else {
     } else {
-      Path[] srcs = listPaths(src);
-      for (Path srcFile : srcs) {
-        copyToLocalFile(srcFile, new Path(dst, srcFile.getName()), copyCrc);
+      FileStatus[] srcs = listStatus(src);
+      for (FileStatus srcFile : srcs) {
+        copyToLocalFile(srcFile.getPath(), 
+                        new Path(dst, srcFile.getPath().getName()), copyCrc);
       }
       }
     }
     }
   }
   }

+ 19 - 45
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -641,18 +641,6 @@ public abstract class FileSystem extends Configured implements Closeable {
       }     
       }     
     };
     };
     
     
-  /** List files in a directory. */
-  @Deprecated
-  public Path[] listPaths(Path f) throws IOException {
-    FileStatus[] stat = listStatus(f);
-    if (stat == null) return null;
-    Path[] ret = new Path[stat.length];
-    for (int i = 0; i < stat.length; ++i) {
-      ret[i] = stat[i].getPath();
-    }
-    return ret;
-  }
-
   /**
   /**
    * List the statuses of the files/directories in the given path if the path is
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
    * a directory.
@@ -699,6 +687,21 @@ public abstract class FileSystem extends Configured implements Closeable {
     return results.toArray(new FileStatus[results.size()]);
     return results.toArray(new FileStatus[results.size()]);
   }
   }
 
 
+  /**
+   * Filter files/directories in the given list of paths using default
+   * path filter.
+   * 
+   * @param files
+   *          a list of paths
+   * @return a list of statuses for the files under the given paths after
+   *         applying the filter default Path filter
+   * @exception IOException
+   */
+  public FileStatus[] listStatus(Path[] files)
+      throws IOException {
+    return listStatus(files, DEFAULT_FILTER);
+  }
+
   /**
   /**
    * Filter files/directories in the given list of paths using user-supplied
    * Filter files/directories in the given list of paths using user-supplied
    * path filter.
    * path filter.
@@ -711,7 +714,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    *         applying the filter
    *         applying the filter
    * @exception IOException
    * @exception IOException
    */
    */
-  private FileStatus[] listStatus(Path[] files, PathFilter filter)
+  public FileStatus[] listStatus(Path[] files, PathFilter filter)
       throws IOException {
       throws IOException {
     ArrayList<FileStatus> results = new ArrayList<FileStatus>();
     ArrayList<FileStatus> results = new ArrayList<FileStatus>();
     for (int i = 0; i < files.length; i++) {
     for (int i = 0; i < files.length; i++) {
@@ -720,35 +723,6 @@ public abstract class FileSystem extends Configured implements Closeable {
     return results.toArray(new FileStatus[results.size()]);
     return results.toArray(new FileStatus[results.size()]);
   }
   }
 
 
-  /** 
-   * Filter files in the given pathes using the default checksum filter. 
-   * @param files a list of paths
-   * @return a list of files under the source paths
-   * @exception IOException
-   */
-  @Deprecated
-  public Path[] listPaths(Path[] files) throws IOException {
-    return listPaths(files, DEFAULT_FILTER);
-  }
-
-  /** Filter files in a directory. */
-  @Deprecated
-  public Path[] listPaths(Path f, PathFilter filter) throws IOException {
-    return FileUtil.stat2Paths(listStatus(f, filter));
-  }
-    
-  /** 
-   * Filter files in a list directories using user-supplied path filter. 
-   * @param files a list of paths
-   * @return a list of files under the source paths
-   * @exception IOException
-   */
-  @Deprecated
-  public Path[] listPaths(Path[] files, PathFilter filter)
-    throws IOException {
-    return FileUtil.stat2Paths(listStatus(files, filter));
-  }
-    
   /**
   /**
    * <p>Return all the files that match filePattern and are not checksum
    * <p>Return all the files that match filePattern and are not checksum
    * files. Results are sorted by their names.
    * files. Results are sorted by their names.
@@ -1215,9 +1189,9 @@ public abstract class FileSystem extends Configured implements Closeable {
   /** Return the total size of all files in the filesystem.*/
   /** Return the total size of all files in the filesystem.*/
   public long getUsed() throws IOException{
   public long getUsed() throws IOException{
     long used = 0;
     long used = 0;
-    Path[] files = listPaths(new Path("/"));
-    for(Path file:files){
-      used += getContentLength(file);
+    FileStatus[] files = listStatus(new Path("/"));
+    for(FileStatus file:files){
+      used += file.getLen();
     }
     }
     return used;
     return used;
   }
   }

+ 14 - 12
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -89,13 +89,13 @@ public class FileUtil {
    */
    */
   public static void fullyDelete(FileSystem fs, Path dir) 
   public static void fullyDelete(FileSystem fs, Path dir) 
   throws IOException {
   throws IOException {
-    Path[] paths = fs.listPaths(dir);
+    FileStatus[] paths = fs.listStatus(dir);
     if (paths != null) {
     if (paths != null) {
-      for (Path p : paths) {
-        if (fs.isFile(p))  {
-          fs.delete(p, true);
+      for (FileStatus p : paths) {
+        if (!p.isDir())  {
+          fs.delete(p.getPath(), true);
         } else {
         } else {
-          fullyDelete(fs, p);
+          fullyDelete(fs, p.getPath());
         }
         }
       }
       }
     }
     }
@@ -185,9 +185,10 @@ public class FileUtil {
       if (!dstFS.mkdirs(dst)) {
       if (!dstFS.mkdirs(dst)) {
         return false;
         return false;
       }
       }
-      Path contents[] = srcFS.listPaths(src);
+      FileStatus contents[] = srcFS.listStatus(src);
       for (int i = 0; i < contents.length; i++) {
       for (int i = 0; i < contents.length; i++) {
-        copy(srcFS, contents[i], dstFS, new Path(dst, contents[i].getName()),
+        copy(srcFS, contents[i].getPath(), dstFS, 
+             new Path(dst, contents[i].getPath().getName()),
              deleteSource, overwrite, conf);
              deleteSource, overwrite, conf);
       }
       }
     } else if (srcFS.isFile(src)) {
     } else if (srcFS.isFile(src)) {
@@ -218,10 +219,10 @@ public class FileUtil {
     OutputStream out = dstFS.create(dstFile);
     OutputStream out = dstFS.create(dstFile);
     
     
     try {
     try {
-      Path contents[] = srcFS.listPaths(srcDir);
+      FileStatus contents[] = srcFS.listStatus(srcDir);
       for (int i = 0; i < contents.length; i++) {
       for (int i = 0; i < contents.length; i++) {
-        if (srcFS.isFile(contents[i])) {
-          InputStream in = srcFS.open(contents[i]);
+        if (!contents[i].isDir()) {
+          InputStream in = srcFS.open(contents[i].getPath());
           try {
           try {
             IOUtils.copyBytes(in, out, conf, false);
             IOUtils.copyBytes(in, out, conf, false);
             if (addString!=null)
             if (addString!=null)
@@ -282,9 +283,10 @@ public class FileUtil {
       if (!dst.mkdirs()) {
       if (!dst.mkdirs()) {
         return false;
         return false;
       }
       }
-      Path contents[] = srcFS.listPaths(src);
+      FileStatus contents[] = srcFS.listStatus(src);
       for (int i = 0; i < contents.length; i++) {
       for (int i = 0; i < contents.length; i++) {
-        copy(srcFS, contents[i], new File(dst, contents[i].getName()),
+        copy(srcFS, contents[i].getPath(), 
+             new File(dst, contents[i].getPath().getName()),
              deleteSource, conf);
              deleteSource, conf);
       }
       }
     } else if (srcFS.isFile(src)) {
     } else if (srcFS.isFile(src)) {

+ 11 - 11
src/java/org/apache/hadoop/fs/FsShell.java

@@ -275,8 +275,9 @@ public class FsShell extends Configured implements Tool {
     } else {
     } else {
       // once FileUtil.copy() supports tmp file, we don't need to mkdirs().
       // once FileUtil.copy() supports tmp file, we don't need to mkdirs().
       dst.mkdirs();
       dst.mkdirs();
-      for(Path path : srcFS.listPaths(src)) {
-        copyToLocal(srcFS, path, new File(dst, path.getName()), copyCrc);
+      for(FileStatus path : srcFS.listStatus(src)) {
+        copyToLocal(srcFS, path.getPath(), 
+                    new File(dst, path.getPath().getName()), copyCrc);
       }
       }
     }
     }
   }
   }
@@ -566,17 +567,17 @@ public class FsShell extends Configured implements Tool {
       setFileReplication(src, srcFs, newRep, waitingList);
       setFileReplication(src, srcFs, newRep, waitingList);
       return;
       return;
     }
     }
-    Path items[] = srcFs.listPaths(src);
+    FileStatus items[] = srcFs.listStatus(src);
     if (items == null) {
     if (items == null) {
       throw new IOException("Could not get listing for " + src);
       throw new IOException("Could not get listing for " + src);
     } else {
     } else {
 
 
       for (int i = 0; i < items.length; i++) {
       for (int i = 0; i < items.length; i++) {
-        Path cur = items[i];
-        if (!srcFs.getFileStatus(cur).isDir()) {
-          setFileReplication(cur, srcFs, newRep, waitingList);
+        if (!items[i].isDir()) {
+          setFileReplication(items[i].getPath(), srcFs, newRep, waitingList);
         } else if (recursive) {
         } else if (recursive) {
-          setReplication(newRep, srcFs, cur, recursive, waitingList);
+          setReplication(newRep, srcFs, items[i].getPath(), recursive, 
+                         waitingList);
         }
         }
       }
       }
     }
     }
@@ -667,7 +668,7 @@ public class FsShell extends Configured implements Tool {
   void du(String src) throws IOException {
   void du(String src) throws IOException {
     Path srcPath = new Path(src);
     Path srcPath = new Path(src);
     FileSystem srcFs = srcPath.getFileSystem(getConf());
     FileSystem srcFs = srcPath.getFileSystem(getConf());
-    Path items[] = srcFs.listPaths(srcFs.globPaths(srcPath));
+    FileStatus items[] = srcFs.listStatus(srcFs.globPaths(srcPath));
     if ((items == null) || ((items.length == 0) && 
     if ((items == null) || ((items.length == 0) && 
         (!srcFs.exists(srcPath)))){
         (!srcFs.exists(srcPath)))){
       throw new FileNotFoundException("Cannot access " + src
       throw new FileNotFoundException("Cannot access " + src
@@ -675,8 +676,7 @@ public class FsShell extends Configured implements Tool {
     } else {
     } else {
       System.out.println("Found " + items.length + " items");
       System.out.println("Found " + items.length + " items");
       for (int i = 0; i < items.length; i++) {
       for (int i = 0; i < items.length; i++) {
-        Path cur = items[i];
-        System.out.println(cur + "\t" + srcFs.getContentLength(cur));
+        System.out.println(items[i].getPath() + "\t" + items[i].getLen());
       }
       }
     }
     }
   }
   }
@@ -1157,7 +1157,7 @@ public class FsShell extends Configured implements Tool {
     public abstract void run(FileStatus file, FileSystem fs) throws IOException;
     public abstract void run(FileStatus file, FileSystem fs) throws IOException;
   }
   }
   
   
-  ///helper for runCmdHandler*() returns listPaths()
+  ///helper for runCmdHandler*() returns listStatus()
   private static FileStatus[] cmdHandlerListStatus(CmdHandler handler, 
   private static FileStatus[] cmdHandlerListStatus(CmdHandler handler, 
                                                    FileSystem srcFs,
                                                    FileSystem srcFs,
                                                    Path path) {
                                                    Path path) {

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

@@ -142,13 +142,13 @@ public class Trash extends Configured {
 
 
   /** Delete old checkpoints. */
   /** Delete old checkpoints. */
   public void expunge() throws IOException {
   public void expunge() throws IOException {
-    Path[] dirs = fs.listPaths(trash);            // scan trash sub-directories
+    FileStatus[] dirs = fs.listStatus(trash);            // scan trash sub-directories
     if( dirs == null){
     if( dirs == null){
       return;
       return;
     }
     }
     long now = System.currentTimeMillis();
     long now = System.currentTimeMillis();
     for (int i = 0; i < dirs.length; i++) {
     for (int i = 0; i < dirs.length; i++) {
-      Path path = dirs[i];
+      Path path = dirs[i].getPath();
       String dir = path.toUri().getPath();
       String dir = path.toUri().getPath();
       String name = path.getName();
       String name = path.getName();
       if (name.equals(CURRENT.getName()))         // skip current
       if (name.equals(CURRENT.getName()))         // skip current

+ 2 - 39
src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java

@@ -211,44 +211,7 @@ public class KosmosFileSystem extends FileSystem {
                                   kfsImpl.getModificationTime(srep), path);
                                   kfsImpl.getModificationTime(srep), path);
         }
         }
     }
     }
-
     
     
-    public Path[] listPaths(Path path) throws IOException {
-	Path absolute = makeAbsolute(path);
-        String srep = absolute.toUri().getPath();
-
-	if (kfsImpl.isFile(srep))
-	    return new Path[] { path } ;
-
-	String[] entries = kfsImpl.readdir(srep);
-
-        if (entries == null)
-            return null;
-
-        // kfsreaddir() returns "." and ".."; strip them before
-        // passing back to hadoop fs.
-	int numEntries = 0;
-	for (int i = 0; i < entries.length; i++) {
-	    if ((entries[i].compareTo(".") == 0) || (entries[i].compareTo("..") == 0))
-		continue;
-	    numEntries++;
-	}
-	if (numEntries == 0) {
-	    return null;
-        }
-	Path[] pathEntries = new Path[numEntries];
-	int j = 0;
-	for (int i = 0; i < entries.length; i++) {
-	    if ((entries[i].compareTo(".") == 0) || (entries[i].compareTo("..") == 0))
-		continue;
-
-	    pathEntries[j] = new Path(path, entries[i]);
-	    j++;
-	}
-	return pathEntries;
-
-    }
-
     public FSDataOutputStream create(Path file, FsPermission permission,
     public FSDataOutputStream create(Path file, FsPermission permission,
                                      boolean overwrite, int bufferSize,
                                      boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
 				     short replication, long blockSize, Progressable progress)
@@ -301,7 +264,7 @@ public class KosmosFileSystem extends FileSystem {
       if (kfsImpl.isFile(srep))
       if (kfsImpl.isFile(srep))
         return kfsImpl.remove(srep) == 0;
         return kfsImpl.remove(srep) == 0;
 
 
-      Path[] dirEntries = listPaths(absolute);
+      FileStatus[] dirEntries = listStatus(absolute);
       if ((!recursive) && (dirEntries != null) && 
       if ((!recursive) && (dirEntries != null) && 
             (dirEntries.length != 0)) {
             (dirEntries.length != 0)) {
         throw new IOException("Directory " + path.toString() + 
         throw new IOException("Directory " + path.toString() + 
@@ -309,7 +272,7 @@ public class KosmosFileSystem extends FileSystem {
       }
       }
       if (dirEntries != null) {
       if (dirEntries != null) {
         for (int i = 0; i < dirEntries.length; i++) {
         for (int i = 0; i < dirEntries.length; i++) {
-          delete(new Path(absolute, dirEntries[i]), recursive);
+          delete(new Path(absolute, dirEntries[i].getPath()), recursive);
         }
         }
       }
       }
       return kfsImpl.rmdir(srep) == 0;
       return kfsImpl.rmdir(srep) == 0;

+ 4 - 3
src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -277,7 +278,7 @@ public class S3FileSystem extends FileSystem {
        store.deleteBlock(block);
        store.deleteBlock(block);
      }
      }
    } else {
    } else {
-     Path[] contents = listPaths(absolutePath);
+     FileStatus[] contents = listStatus(absolutePath);
      if (contents == null) {
      if (contents == null) {
        return false;
        return false;
      }
      }
@@ -285,8 +286,8 @@ public class S3FileSystem extends FileSystem {
        throw new IOException("Directory " + path.toString() 
        throw new IOException("Directory " + path.toString() 
            + " is not empty.");
            + " is not empty.");
      }
      }
-     for (Path p:contents) {
-       if (!delete(p, recursive)) {
+     for (FileStatus p:contents) {
+       if (!delete(p.getPath(), recursive)) {
          return false;
          return false;
        }
        }
      }
      }

+ 5 - 6
src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java

@@ -97,20 +97,19 @@ public class CompletedJobStatusStore implements Runnable {
   private void deleteJobStatusDirs() {
   private void deleteJobStatusDirs() {
     try {
     try {
       long currentTime = System.currentTimeMillis();
       long currentTime = System.currentTimeMillis();
-      Path[] jobInfoFiles = fs.listPaths(
+      FileStatus[] jobInfoFiles = fs.listStatus(
               new Path[]{new Path(jobInfoDir)});
               new Path[]{new Path(jobInfoDir)});
 
 
       //noinspection ForLoopReplaceableByForEach
       //noinspection ForLoopReplaceableByForEach
-      for (Path jobInfo : jobInfoFiles) {
+      for (FileStatus jobInfo : jobInfoFiles) {
         try {
         try {
-          FileStatus status = fs.getFileStatus(jobInfo);
-          if ((currentTime - status.getModificationTime()) > retainTime) {
-            fs.delete(jobInfo, true);
+          if ((currentTime - jobInfo.getModificationTime()) > retainTime) {
+            fs.delete(jobInfo.getPath(), true);
           }
           }
         }
         }
         catch (IOException ie) {
         catch (IOException ie) {
           LOG.warn("Could not do housekeeping for [ " +
           LOG.warn("Could not do housekeeping for [ " +
-                  jobInfo + "] job info : " + ie.getMessage(), ie);
+                  jobInfo.getPath() + "] job info : " + ie.getMessage(), ie);
         }
         }
       }
       }
     }
     }

+ 7 - 2
src/java/org/apache/hadoop/mapred/FileInputFormat.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
@@ -154,7 +155,9 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
     for (Path p: dirs) {
     for (Path p: dirs) {
       FileSystem fs = p.getFileSystem(job); 
       FileSystem fs = p.getFileSystem(job); 
       Path[] matches =
       Path[] matches =
-        fs.listPaths(fs.globPaths(p, inputFilter), inputFilter);
+        FileUtil.stat2Paths(fs.listStatus(fs.globPaths(p, inputFilter), 
+                            inputFilter));
+
       for (Path match: matches) {
       for (Path match: matches) {
         result.add(fs.makeQualified(match));
         result.add(fs.makeQualified(match));
       }
       }
@@ -176,7 +179,9 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
       if (fs.exists(p)) {
       if (fs.exists(p)) {
         // make sure all paths are files to avoid exception
         // make sure all paths are files to avoid exception
         // while generating splits
         // while generating splits
-        for (Path subPath : fs.listPaths(p, hiddenFileFilter)) {
+        Path[] subPaths = FileUtil.stat2Paths(fs.listStatus(p, 
+                                              hiddenFileFilter));
+        for (Path subPath : subPaths) {
           FileSystem subFS = subPath.getFileSystem(job); 
           FileSystem subFS = subPath.getFileSystem(job); 
           if (!subFS.exists(subPath)) {
           if (!subFS.exists(subPath)) {
             result.add(new IOException(
             result.add(new IOException(

+ 2 - 1
src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java

@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileUtil;
 
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.MapFile;
@@ -86,7 +87,7 @@ public class MapFileOutputFormat extends OutputFormatBase {
                                             Configuration conf)
                                             Configuration conf)
     throws IOException {
     throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
     FileSystem fs = dir.getFileSystem(conf);
-    Path[] names = fs.listPaths(dir);
+    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
 
 
     // sort names, so that hash partitioning works
     // sort names, so that hash partitioning works
     Arrays.sort(names);
     Arrays.sort(names);

+ 2 - 1
src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java

@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileUtil;
 
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
@@ -81,7 +82,7 @@ public class SequenceFileOutputFormat extends OutputFormatBase {
   public static SequenceFile.Reader[] getReaders(Configuration conf, Path dir)
   public static SequenceFile.Reader[] getReaders(Configuration conf, Path dir)
     throws IOException {
     throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
     FileSystem fs = dir.getFileSystem(conf);
-    Path[] names = fs.listPaths(dir);
+    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
     
     
     // sort names, so that hash partitioning works
     // sort names, so that hash partitioning works
     Arrays.sort(names);
     Arrays.sort(names);

+ 4 - 3
src/java/org/apache/hadoop/mapred/Task.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -476,12 +477,12 @@ abstract class Task implements Writable, Configurable {
       }
       }
       LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
       LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
     } else if(fs.isDirectory(taskOutput)) {
     } else if(fs.isDirectory(taskOutput)) {
-      Path[] paths = fs.listPaths(taskOutput);
+      FileStatus[] paths = fs.listStatus(taskOutput);
       Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput);
       Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput);
       fs.mkdirs(finalOutputPath);
       fs.mkdirs(finalOutputPath);
       if (paths != null) {
       if (paths != null) {
-        for (Path path : paths) {
-          moveTaskOutputs(fs, jobOutputDir, path);
+        for (FileStatus path : paths) {
+          moveTaskOutputs(fs, jobOutputDir, path.getPath());
         }
         }
       }
       }
     }
     }

+ 0 - 25
src/test/org/apache/hadoop/dfs/TestFileStatus.java

@@ -147,31 +147,6 @@ public class TestFileStatus extends TestCase {
       // of the two files
       // of the two files
       assertTrue(dir + " size should be " + (blockSize/2), 
       assertTrue(dir + " size should be " + (blockSize/2), 
                  blockSize/2 == fs.getContentLength(dir));
                  blockSize/2 == fs.getContentLength(dir));
-
-      // The following are test cases for listPaths which is a deprecated
-      // API. These tests shoudl go away when the API is removed.
-
-      // issue a listPaths on directory /test/mkdirs and verify that the
-      // size of the files inside it are valid
-      Path[] files = fs.listPaths(dir);
-      assertTrue(dir + " should have two files", files.length == 2);
-      for (int i = 0; i < files.length; i++) {
-        DfsPath dfspath = (DfsPath) files[i];
-        assertTrue(files[i] + " should be of size " + (blockSize/4), 
-                   blockSize/4 == dfspath.getContentsLength());
-        assertTrue(files[i] + " should be of size " + (blockSize/4), 
-                   blockSize/4 == fs.getContentLength(dfspath));
-      }
-
-      // issue a listPath on directory /test and verify that the
-      // size returned for /test/mkdirs directory is correct.
-      Path[] dirs = fs.listPaths(parentDir);
-      assertTrue(parentDir + " should have one sub directory", 
-                 dirs.length == 1);
-      DfsPath dfsdir = (DfsPath) dirs[0];
-      assertTrue(dirs[0] + " should be of size " + blockSize/2,
-                 fs.getContentLength(dfsdir) == blockSize/2);
-      
     } finally {
     } finally {
       fs.close();
       fs.close();
       cluster.shutdown();
       cluster.shutdown();

+ 2 - 2
src/test/org/apache/hadoop/fs/DistributedFSCheck.java

@@ -116,11 +116,11 @@ public class DistributedFSCheck extends TestCase {
       return;
       return;
     }
     }
     
     
-    Path children[] = fs.listPaths(rootFile);
+    FileStatus children[] = fs.listStatus(rootFile);
     if (children == null)
     if (children == null)
       throw new IOException("Could not get listing for " + rootFile);
       throw new IOException("Could not get listing for " + rootFile);
     for (int i = 0; i < children.length; i++)
     for (int i = 0; i < children.length; i++)
-      listSubtree(children[i], writer);
+      listSubtree(children[i].getPath(), writer);
   }
   }
 
 
   /**
   /**

+ 3 - 3
src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java

@@ -54,7 +54,7 @@ public class KFSEmulationImpl implements IFSImpl {
     // as part of the emulation, KFS adds ./.. as directory entries
     // as part of the emulation, KFS adds ./.. as directory entries
     // when doing a directory listing.
     // when doing a directory listing.
     public String[] readdir(String path) throws IOException {
     public String[] readdir(String path) throws IOException {
-        Path[] p = localFS.listPaths(new Path(path));
+        FileStatus[] p = localFS.listStatus(new Path(path));
         String[] entries = null;
         String[] entries = null;
 
 
         if (p == null) {
         if (p == null) {
@@ -73,11 +73,11 @@ public class KFSEmulationImpl implements IFSImpl {
             entries[0] = new String(".");
             entries[0] = new String(".");
             entries[1] = new String("..");
             entries[1] = new String("..");
             for (int i = 0; i < p.length; i++)
             for (int i = 0; i < p.length; i++)
-                entries[i+2] = p[i].toString();
+                entries[i+2] = p[i].getPath().toString();
         } else {
         } else {
             entries = new String[p.length];
             entries = new String[p.length];
             for (int i = 0; i < p.length; i++)
             for (int i = 0; i < p.length; i++)
-                entries[i] = p[i].toString();
+                entries[i] = p[i].getPath().toString();
         }
         }
         return entries;
         return entries;
     }
     }

+ 5 - 4
src/test/org/apache/hadoop/fs/kfs/TestKosmosFileSystem.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 
 
@@ -70,7 +71,7 @@ public class TestKosmosFileSystem extends TestCase {
         assertFalse(kosmosFileSystem.exists(new Path("test1")));
         assertFalse(kosmosFileSystem.exists(new Path("test1")));
         assertFalse(kosmosFileSystem.isDirectory(new Path("test/dir.2")));
         assertFalse(kosmosFileSystem.isDirectory(new Path("test/dir.2")));
 
 
-        Path[] p = kosmosFileSystem.listPaths(baseDir);
+        FileStatus[] p = kosmosFileSystem.listStatus(baseDir);
         assertEquals(p.length, 1);
         assertEquals(p.length, 1);
 
 
         kosmosFileSystem.delete(baseDir, true);
         kosmosFileSystem.delete(baseDir, true);
@@ -97,15 +98,15 @@ public class TestKosmosFileSystem extends TestCase {
         s1.close();
         s1.close();
         s2.close();
         s2.close();
 
 
-        Path[] p = kosmosFileSystem.listPaths(subDir1);
+        FileStatus[] p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p.length, 2);
         assertEquals(p.length, 2);
 
 
         kosmosFileSystem.delete(file1, true);
         kosmosFileSystem.delete(file1, true);
-        p = kosmosFileSystem.listPaths(subDir1);
+        p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p.length, 1);
         assertEquals(p.length, 1);
 
 
         kosmosFileSystem.delete(file2, true);
         kosmosFileSystem.delete(file2, true);
-        p = kosmosFileSystem.listPaths(subDir1);
+        p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p, null);
         assertEquals(p, null);
 
 
         kosmosFileSystem.delete(baseDir, true);
         kosmosFileSystem.delete(baseDir, true);

+ 11 - 10
src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
 
 
 public abstract class S3FileSystemBaseTest extends TestCase {
 public abstract class S3FileSystemBaseTest extends TestCase {
   
   
@@ -102,32 +103,32 @@ public abstract class S3FileSystemBaseTest extends TestCase {
   public void testListPathsRaw() throws Exception {
   public void testListPathsRaw() throws Exception {
     Path[] testDirs = { new Path("/test/hadoop/a"), new Path("/test/hadoop/b"),
     Path[] testDirs = { new Path("/test/hadoop/a"), new Path("/test/hadoop/b"),
                         new Path("/test/hadoop/c/1"), };
                         new Path("/test/hadoop/c/1"), };
-    assertNull(s3FileSystem.listPaths(testDirs[0]));
+    assertNull(s3FileSystem.listStatus(testDirs[0]));
 
 
     for (Path path : testDirs) {
     for (Path path : testDirs) {
       assertTrue(s3FileSystem.mkdirs(path));
       assertTrue(s3FileSystem.mkdirs(path));
     }
     }
 
 
-    Path[] paths = s3FileSystem.listPaths(new Path("/"));
+    FileStatus[] paths = s3FileSystem.listStatus(new Path("/"));
 
 
     assertEquals(1, paths.length);
     assertEquals(1, paths.length);
-    assertEquals(new Path("/test").makeQualified(s3FileSystem), paths[0]);
+    assertEquals(new Path("/test").makeQualified(s3FileSystem), paths[0].getPath());
 
 
-    paths = s3FileSystem.listPaths(new Path("/test"));
+    paths = s3FileSystem.listStatus(new Path("/test"));
     assertEquals(1, paths.length);
     assertEquals(1, paths.length);
     assertEquals(new Path("/test/hadoop").makeQualified(s3FileSystem),
     assertEquals(new Path("/test/hadoop").makeQualified(s3FileSystem),
-        paths[0]);
+        paths[0].getPath());
 
 
-    paths = s3FileSystem.listPaths(new Path("/test/hadoop"));
+    paths = s3FileSystem.listStatus(new Path("/test/hadoop"));
     assertEquals(3, paths.length);
     assertEquals(3, paths.length);
     assertEquals(new Path("/test/hadoop/a").makeQualified(s3FileSystem),
     assertEquals(new Path("/test/hadoop/a").makeQualified(s3FileSystem),
-        paths[0]);
+        paths[0].getPath());
     assertEquals(new Path("/test/hadoop/b").makeQualified(s3FileSystem),
     assertEquals(new Path("/test/hadoop/b").makeQualified(s3FileSystem),
-        paths[1]);
+        paths[1].getPath());
     assertEquals(new Path("/test/hadoop/c").makeQualified(s3FileSystem),
     assertEquals(new Path("/test/hadoop/c").makeQualified(s3FileSystem),
-        paths[2]);
+        paths[2].getPath());
 
 
-    paths = s3FileSystem.listPaths(new Path("/test/hadoop/a"));
+    paths = s3FileSystem.listStatus(new Path("/test/hadoop/a"));
     assertEquals(0, paths.length);
     assertEquals(0, paths.length);
   }
   }
 
 

+ 2 - 1
src/test/org/apache/hadoop/mapred/BigMapOutput.java

@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile;
@@ -53,7 +54,7 @@ public class BigMapOutput extends Configured implements Tool {
   throws IOException {
   throws IOException {
     // Check if the input path exists and is non-empty
     // Check if the input path exists and is non-empty
     if (fs.exists(dir)) {
     if (fs.exists(dir)) {
-      Path[] list = fs.listPaths(dir);
+      FileStatus[] list = fs.listStatus(dir);
       if (list != null && list.length > 0) {
       if (list != null && list.length > 0) {
         throw new IOException("Input path: " + dir + " already exists... ");
         throw new IOException("Input path: " + dir + " already exists... ");
       }
       }

+ 2 - 2
src/test/org/apache/hadoop/mapred/SortValidator.java

@@ -319,9 +319,9 @@ public class SortValidator extends Configured implements Tool {
       jobConf.setJobName("sortvalidate-recordstats-checker");
       jobConf.setJobName("sortvalidate-recordstats-checker");
 
 
       int noSortReduceTasks = 
       int noSortReduceTasks = 
-        fs.listPaths(sortOutput, sortPathsFilter).length;
+        fs.listStatus(sortOutput, sortPathsFilter).length;
       jobConf.setInt("sortvalidate.sort.reduce.tasks", noSortReduceTasks);
       jobConf.setInt("sortvalidate.sort.reduce.tasks", noSortReduceTasks);
-      int noSortInputpaths = fs.listPaths(sortInput).length;
+      int noSortInputpaths = fs.listStatus(sortInput).length;
 
 
       jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
       jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class);
       jobConf.setOutputFormat(SequenceFileOutputFormat.class);
       jobConf.setOutputFormat(SequenceFileOutputFormat.class);

+ 2 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -76,7 +77,7 @@ public class TestMiniMRClasspath extends TestCase {
     JobClient.runJob(conf);
     JobClient.runJob(conf);
     StringBuffer result = new StringBuffer();
     StringBuffer result = new StringBuffer();
     {
     {
-      Path[] parents = fs.listPaths(outDir.getParent());
+      Path[] parents = FileUtil.stat2Paths(fs.listStatus(outDir.getParent()));
       Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
       Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
               new OutputLogFilter()));
               new OutputLogFilter()));
       for(int i=0; i < fileList.length; ++i) {
       for(int i=0; i < fileList.length; ++i) {

+ 2 - 2
src/test/org/apache/hadoop/mapred/pipes/WordCountInputFormat.java

@@ -54,8 +54,8 @@ public class WordCountInputFormat
     ArrayList<InputSplit> result = new ArrayList<InputSplit>();
     ArrayList<InputSplit> result = new ArrayList<InputSplit>();
     FileSystem local = FileSystem.getLocal(conf);
     FileSystem local = FileSystem.getLocal(conf);
     for(Path dir: conf.getInputPaths()) {
     for(Path dir: conf.getInputPaths()) {
-      for(Path file: local.listPaths(dir)) {
-        result.add(new WordCountInputSplit(file));
+      for(FileStatus file: local.listStatus(dir)) {
+        result.add(new WordCountInputSplit(file.getPath()));
       }
       }
     }
     }
     return result.toArray(new InputSplit[result.size()]);
     return result.toArray(new InputSplit[result.size()]);

+ 2 - 1
src/webapps/job/jobhistory.jsp

@@ -36,7 +36,8 @@
       out.println("Null file system. May be namenode is in safemode!");
       out.println("Null file system. May be namenode is in safemode!");
       return;
       return;
     }
     }
-    Path[] jobFiles = fs.listPaths(new Path(historyLogDir), jobLogFileFilter);
+    Path[] jobFiles = FileUtil.stat2Paths(fs.listStatus(new Path(historyLogDir),
+                                          jobLogFileFilter));
     if (null == jobFiles )  {
     if (null == jobFiles )  {
       out.println("NULL files !!!"); 
       out.println("NULL files !!!"); 
       return ; 
       return ;