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().
     (lohit vijayarenu via cdouglas)
 
+    HADOOP-2563. Remove deprecated FileSystem::listPaths.
+    (lohit vijayarenu via cdouglas)
+
   NEW FEATURES
 
     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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.dfs.MiniDFSCluster;
@@ -132,7 +133,8 @@ public class TestMultipleArchiveFiles extends TestStreaming
         throw new Exception("Job Failed");
       }
       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++){
         BufferedReader bread =
           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);
   }
   
-  /** 
-   * 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
   public boolean mkdirs(Path f) throws IOException {
     return fs.mkdirs(f);
@@ -524,9 +502,10 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
         fs.copyToLocalFile(checksumFile, dst);
       }
     } 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
    * a directory.
@@ -699,6 +687,21 @@ public abstract class FileSystem extends Configured implements Closeable {
     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
    * path filter.
@@ -711,7 +714,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    *         applying the filter
    * @exception IOException
    */
-  private FileStatus[] listStatus(Path[] files, PathFilter filter)
+  public FileStatus[] listStatus(Path[] files, PathFilter filter)
       throws IOException {
     ArrayList<FileStatus> results = new ArrayList<FileStatus>();
     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()]);
   }
 
-  /** 
-   * 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
    * 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.*/
   public long getUsed() throws IOException{
     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;
   }

+ 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) 
   throws IOException {
-    Path[] paths = fs.listPaths(dir);
+    FileStatus[] paths = fs.listStatus(dir);
     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 {
-          fullyDelete(fs, p);
+          fullyDelete(fs, p.getPath());
         }
       }
     }
@@ -185,9 +185,10 @@ public class FileUtil {
       if (!dstFS.mkdirs(dst)) {
         return false;
       }
-      Path contents[] = srcFS.listPaths(src);
+      FileStatus contents[] = srcFS.listStatus(src);
       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);
       }
     } else if (srcFS.isFile(src)) {
@@ -218,10 +219,10 @@ public class FileUtil {
     OutputStream out = dstFS.create(dstFile);
     
     try {
-      Path contents[] = srcFS.listPaths(srcDir);
+      FileStatus contents[] = srcFS.listStatus(srcDir);
       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 {
             IOUtils.copyBytes(in, out, conf, false);
             if (addString!=null)
@@ -282,9 +283,10 @@ public class FileUtil {
       if (!dst.mkdirs()) {
         return false;
       }
-      Path contents[] = srcFS.listPaths(src);
+      FileStatus contents[] = srcFS.listStatus(src);
       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);
       }
     } 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 {
       // once FileUtil.copy() supports tmp file, we don't need to 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);
       return;
     }
-    Path items[] = srcFs.listPaths(src);
+    FileStatus items[] = srcFs.listStatus(src);
     if (items == null) {
       throw new IOException("Could not get listing for " + src);
     } else {
 
       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) {
-          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 {
     Path srcPath = new Path(src);
     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) && 
         (!srcFs.exists(srcPath)))){
       throw new FileNotFoundException("Cannot access " + src
@@ -675,8 +676,7 @@ public class FsShell extends Configured implements Tool {
     } else {
       System.out.println("Found " + items.length + " items");
       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;
   }
   
-  ///helper for runCmdHandler*() returns listPaths()
+  ///helper for runCmdHandler*() returns listStatus()
   private static FileStatus[] cmdHandlerListStatus(CmdHandler handler, 
                                                    FileSystem srcFs,
                                                    Path path) {

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

@@ -142,13 +142,13 @@ public class Trash extends Configured {
 
   /** Delete old checkpoints. */
   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){
       return;
     }
     long now = System.currentTimeMillis();
     for (int i = 0; i < dirs.length; i++) {
-      Path path = dirs[i];
+      Path path = dirs[i].getPath();
       String dir = path.toUri().getPath();
       String name = path.getName();
       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);
         }
     }
-
     
-    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,
                                      boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
@@ -301,7 +264,7 @@ public class KosmosFileSystem extends FileSystem {
       if (kfsImpl.isFile(srep))
         return kfsImpl.remove(srep) == 0;
 
-      Path[] dirEntries = listPaths(absolute);
+      FileStatus[] dirEntries = listStatus(absolute);
       if ((!recursive) && (dirEntries != null) && 
             (dirEntries.length != 0)) {
         throw new IOException("Directory " + path.toString() + 
@@ -309,7 +272,7 @@ public class KosmosFileSystem extends FileSystem {
       }
       if (dirEntries != null) {
         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;

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

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

@@ -97,20 +97,19 @@ public class CompletedJobStatusStore implements Runnable {
   private void deleteJobStatusDirs() {
     try {
       long currentTime = System.currentTimeMillis();
-      Path[] jobInfoFiles = fs.listPaths(
+      FileStatus[] jobInfoFiles = fs.listStatus(
               new Path[]{new Path(jobInfoDir)});
 
       //noinspection ForLoopReplaceableByForEach
-      for (Path jobInfo : jobInfoFiles) {
+      for (FileStatus jobInfo : jobInfoFiles) {
         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) {
           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.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -154,7 +155,9 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
     for (Path p: dirs) {
       FileSystem fs = p.getFileSystem(job); 
       Path[] matches =
-        fs.listPaths(fs.globPaths(p, inputFilter), inputFilter);
+        FileUtil.stat2Paths(fs.listStatus(fs.globPaths(p, inputFilter), 
+                            inputFilter));
+
       for (Path match: matches) {
         result.add(fs.makeQualified(match));
       }
@@ -176,7 +179,9 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
       if (fs.exists(p)) {
         // make sure all paths are files to avoid exception
         // 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); 
           if (!subFS.exists(subPath)) {
             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.Path;
+import org.apache.hadoop.fs.FileUtil;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
@@ -86,7 +87,7 @@ public class MapFileOutputFormat extends OutputFormatBase {
                                             Configuration conf)
     throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
-    Path[] names = fs.listPaths(dir);
+    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
 
     // sort names, so that hash partitioning works
     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.Path;
+import org.apache.hadoop.fs.FileUtil;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.WritableComparable;
@@ -81,7 +82,7 @@ public class SequenceFileOutputFormat extends OutputFormatBase {
   public static SequenceFile.Reader[] getReaders(Configuration conf, Path dir)
     throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
-    Path[] names = fs.listPaths(dir);
+    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
     
     // sort names, so that hash partitioning works
     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.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -476,12 +477,12 @@ abstract class Task implements Writable, Configurable {
       }
       LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
     } else if(fs.isDirectory(taskOutput)) {
-      Path[] paths = fs.listPaths(taskOutput);
+      FileStatus[] paths = fs.listStatus(taskOutput);
       Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput);
       fs.mkdirs(finalOutputPath);
       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
       assertTrue(dir + " size should be " + (blockSize/2), 
                  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 {
       fs.close();
       cluster.shutdown();

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

@@ -116,11 +116,11 @@ public class DistributedFSCheck extends TestCase {
       return;
     }
     
-    Path children[] = fs.listPaths(rootFile);
+    FileStatus children[] = fs.listStatus(rootFile);
     if (children == null)
       throw new IOException("Could not get listing for " + rootFile);
     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
     // when doing a directory listing.
     public String[] readdir(String path) throws IOException {
-        Path[] p = localFS.listPaths(new Path(path));
+        FileStatus[] p = localFS.listStatus(new Path(path));
         String[] entries = null;
 
         if (p == null) {
@@ -73,11 +73,11 @@ public class KFSEmulationImpl implements IFSImpl {
             entries[0] = new String(".");
             entries[1] = new String("..");
             for (int i = 0; i < p.length; i++)
-                entries[i+2] = p[i].toString();
+                entries[i+2] = p[i].getPath().toString();
         } else {
             entries = new String[p.length];
             for (int i = 0; i < p.length; i++)
-                entries[i] = p[i].toString();
+                entries[i] = p[i].getPath().toString();
         }
         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.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 
@@ -70,7 +71,7 @@ public class TestKosmosFileSystem extends TestCase {
         assertFalse(kosmosFileSystem.exists(new Path("test1")));
         assertFalse(kosmosFileSystem.isDirectory(new Path("test/dir.2")));
 
-        Path[] p = kosmosFileSystem.listPaths(baseDir);
+        FileStatus[] p = kosmosFileSystem.listStatus(baseDir);
         assertEquals(p.length, 1);
 
         kosmosFileSystem.delete(baseDir, true);
@@ -97,15 +98,15 @@ public class TestKosmosFileSystem extends TestCase {
         s1.close();
         s2.close();
 
-        Path[] p = kosmosFileSystem.listPaths(subDir1);
+        FileStatus[] p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p.length, 2);
 
         kosmosFileSystem.delete(file1, true);
-        p = kosmosFileSystem.listPaths(subDir1);
+        p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p.length, 1);
 
         kosmosFileSystem.delete(file2, true);
-        p = kosmosFileSystem.listPaths(subDir1);
+        p = kosmosFileSystem.listStatus(subDir1);
         assertEquals(p, null);
 
         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.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileStatus;
 
 public abstract class S3FileSystemBaseTest extends TestCase {
   
@@ -102,32 +103,32 @@ public abstract class S3FileSystemBaseTest extends TestCase {
   public void testListPathsRaw() throws Exception {
     Path[] testDirs = { new Path("/test/hadoop/a"), new Path("/test/hadoop/b"),
                         new Path("/test/hadoop/c/1"), };
-    assertNull(s3FileSystem.listPaths(testDirs[0]));
+    assertNull(s3FileSystem.listStatus(testDirs[0]));
 
     for (Path path : testDirs) {
       assertTrue(s3FileSystem.mkdirs(path));
     }
 
-    Path[] paths = s3FileSystem.listPaths(new Path("/"));
+    FileStatus[] paths = s3FileSystem.listStatus(new Path("/"));
 
     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(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(new Path("/test/hadoop/a").makeQualified(s3FileSystem),
-        paths[0]);
+        paths[0].getPath());
     assertEquals(new Path("/test/hadoop/b").makeQualified(s3FileSystem),
-        paths[1]);
+        paths[1].getPath());
     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);
   }
 

+ 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.Configured;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -53,7 +54,7 @@ public class BigMapOutput extends Configured implements Tool {
   throws IOException {
     // Check if the input path exists and is non-empty
     if (fs.exists(dir)) {
-      Path[] list = fs.listPaths(dir);
+      FileStatus[] list = fs.listStatus(dir);
       if (list != null && list.length > 0) {
         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");
 
       int noSortReduceTasks = 
-        fs.listPaths(sortOutput, sortPathsFilter).length;
+        fs.listStatus(sortOutput, sortPathsFilter).length;
       jobConf.setInt("sortvalidate.sort.reduce.tasks", noSortReduceTasks);
-      int noSortInputpaths = fs.listPaths(sortInput).length;
+      int noSortInputpaths = fs.listStatus(sortInput).length;
 
       jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.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.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -76,7 +77,7 @@ public class TestMiniMRClasspath extends TestCase {
     JobClient.runJob(conf);
     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,
               new OutputLogFilter()));
       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>();
     FileSystem local = FileSystem.getLocal(conf);
     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()]);

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

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