You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sz...@apache.org on 2008/12/23 23:14:26 UTC

svn commit: r729141 - in /hadoop/core/trunk: ./ src/contrib/streaming/src/java/org/apache/hadoop/streaming/ src/core/org/apache/hadoop/fs/ src/core/org/apache/hadoop/fs/ftp/ src/core/org/apache/hadoop/fs/kfs/ src/core/org/apache/hadoop/fs/s3/ src/core/...

Author: szetszwo
Date: Tue Dec 23 14:14:25 2008
New Revision: 729141

URL: http://svn.apache.org/viewvc?rev=729141&view=rev
Log:
HADOOP-4779. Remove deprecated FileSystem methods: delete(Path f), getBlockSize(Path f), getLength(Path f) and getReplication(Path src).  (szetszwo)

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/FileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/FilterFileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3/S3FileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
    hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobConf.java
    hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/UtilsForTests.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue Dec 23 14:14:25 2008
@@ -4,9 +4,13 @@
 
   INCOMPATIBLE CHANGES
 
-    HADOOP-4895. Removes deprecated methods DFSClient.getHints(..) and
+    HADOOP-4895. Remove deprecated methods DFSClient.getHints(..) and
     DFSClient.isDirectory(..).  (szetszwo)
 
+    HADOOP-4779. Remove deprecated FileSystem methods: delete(Path f),
+    getBlockSize(Path f), getLength(Path f) and getReplication(Path src).
+    (szetszwo)
+
   NEW FEATURES
 
   IMPROVEMENTS

Modified: hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java (original)
+++ hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java Tue Dec 23 14:14:25 2008
@@ -319,7 +319,7 @@
   }
 
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
-    int len = (int) fs.getLength(p);
+    int len = (int) fs.getFileStatus(p).getLen();
     byte[] buf = new byte[len];
     FSDataInputStream in = fs.open(p);
     String contents = null;

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/FileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/FileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/FileSystem.java Tue Dec 23 14:14:25 2008
@@ -17,22 +17,33 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.*;
-import java.net.*;
-import java.util.*;
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import javax.security.auth.login.LoginException;
 
-import org.apache.commons.logging.*;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.util.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
 
 /****************************************************************
  * An abstract base class for a fairly generic filesystem.  It
@@ -529,19 +540,6 @@
    */
   public abstract FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException;
-  
-  /**
-   * Get replication.
-   * 
-   * @deprecated Use getFileStatus() instead
-   * @param src file name
-   * @return file replication
-   * @throws IOException
-   */ 
-  @Deprecated
-  public short getReplication(Path src) throws IOException {
-    return getFileStatus(src).getReplication();
-  }
 
   /**
    * Set replication for an existing file.
@@ -563,10 +561,6 @@
    */
   public abstract boolean rename(Path src, Path dst) throws IOException;
     
-  /** Delete a file. */
-  /** @deprecated Use delete(Path, boolean) instead */ @Deprecated 
-  public abstract boolean delete(Path f) throws IOException;
-  
   /** Delete a file.
    *
    * @param f the path to delete.
@@ -650,12 +644,6 @@
     }
   }
     
-  /** The number of bytes in a file. */
-  /** @deprecated Use getFileStatus() instead */ @Deprecated
-  public long getLength(Path f) throws IOException {
-    return getFileStatus(f).getLen();
-  }
-    
   /** Return the {@link ContentSummary} of a given {@link Path}. */
   public ContentSummary getContentSummary(Path f) throws IOException {
     FileStatus status = getFileStatus(f);
@@ -1224,16 +1212,6 @@
     return used;
   }
 
-  /**
-   * Get the block size for a particular file.
-   * @param f the filename
-   * @return the number of bytes in a block
-   */
-  /** @deprecated Use getFileStatus() instead */ @Deprecated
-  public long getBlockSize(Path f) throws IOException {
-    return getFileStatus(f).getBlockSize();
-  }
-    
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
   public long getDefaultBlockSize() {

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/FilterFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/FilterFileSystem.java Tue Dec 23 14:14:25 2008
@@ -138,11 +138,6 @@
     return fs.rename(src, dst);
   }
   
-  /** Delete a file */@Deprecated
-  public boolean delete(Path f) throws IOException {
-    return delete(f, true);
-  }
-  
   /** Delete a file */
   public boolean delete(Path f, boolean recursive) throws IOException {
     return fs.delete(f, recursive);

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/RawLocalFileSystem.java Tue Dec 23 14:14:25 2008
@@ -18,17 +18,23 @@
 
 package org.apache.hadoop.fs;
 
-import java.io.*;
+import java.io.BufferedOutputStream;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
 import java.net.URI;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileLock;
-import java.util.*;
+import java.util.StringTokenizer;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 
 /****************************************************************
  * Implement the FileSystem API for the raw local filesystem.
@@ -252,11 +258,6 @@
     return FileUtil.copy(this, src, this, dst, true, getConf());
   }
   
-  @Deprecated
-  public boolean delete(Path p) throws IOException {
-    return delete(p, true);
-  }
-  
   public boolean delete(Path p, boolean recursive) throws IOException {
     File f = pathToFile(p);
     if (f.isFile()) {

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java Tue Dec 23 14:14:25 2008
@@ -264,13 +264,6 @@
     }
   }
 
-  /** @deprecated Use delete(Path, boolean) instead */
-  @Override
-  @Deprecated
-  public boolean delete(Path file) throws IOException {
-    return delete(file, false);
-  }
-
   @Override
   public boolean delete(Path file, boolean recursive) throws IOException {
     FTPClient client = connect();

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java Tue Dec 23 14:14:25 2008
@@ -176,7 +176,7 @@
 
         if (exists(file)) {
             if (overwrite) {
-                delete(file);
+                delete(file, true);
             } else {
                 throw new IOException("File already exists: " + file);
             }
@@ -234,25 +234,6 @@
       }
       return kfsImpl.rmdir(srep) == 0;
     }
-    
-    @Deprecated
-    public boolean delete(Path path) throws IOException {
-      return delete(path, true);
-    }
-    
-    @Deprecated
-    public long getLength(Path path) throws IOException {
-	Path absolute = makeAbsolute(path);
-        String srep = absolute.toUri().getPath();
-        return kfsImpl.filesize(srep);
-    }
-
-    @Deprecated
-    public short getReplication(Path path) throws IOException {
-	Path absolute = makeAbsolute(path);
-        String srep = absolute.toUri().getPath();
-        return kfsImpl.getReplication(srep);
-    }
 
     public short getDefaultReplication() {
 	return 3;

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3/S3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3/S3FileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3/S3FileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3/S3FileSystem.java Tue Dec 23 14:14:25 2008
@@ -212,7 +212,7 @@
     INode inode = store.retrieveINode(makeAbsolute(file));
     if (inode != null) {
       if (overwrite) {
-        delete(file);
+        delete(file, true);
       } else {
         throw new IOException("File already exists: " + file);
       }
@@ -317,12 +317,6 @@
    return true;
   }
   
-  @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
   /**
    * FileStatus for S3 file systems. 
    */

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java Tue Dec 23 14:14:25 2008
@@ -280,12 +280,6 @@
   }
   
   @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
-  @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
     FileStatus status;
     try {

Modified: hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java (original)
+++ hadoop/core/trunk/src/core/org/apache/hadoop/io/SequenceFile.java Tue Dec 23 14:14:25 2008
@@ -1414,7 +1414,7 @@
 
     private Reader(FileSystem fs, Path file, int bufferSize,
                    Configuration conf, boolean tempReader) throws IOException {
-      this(fs, file, bufferSize, 0, fs.getLength(file), conf, tempReader);
+      this(fs, file, bufferSize, 0, fs.getFileStatus(file).getLen(), conf, tempReader);
     }
     
     private Reader(FileSystem fs, Path file, int bufferSize, long start,
@@ -2605,8 +2605,8 @@
       //get the segments from inNames
       ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
       for (int i = 0; i < inNames.length; i++) {
-        SegmentDescriptor s = new SegmentDescriptor(0, 
-                                                    fs.getLength(inNames[i]), inNames[i]);
+        SegmentDescriptor s = new SegmentDescriptor(0,
+            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
         s.preserveInput(!deleteInputs);
         s.doSync();
         a.add(s);
@@ -2634,8 +2634,8 @@
       //get the segments from inNames
       ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
       for (int i = 0; i < inNames.length; i++) {
-        SegmentDescriptor s = new SegmentDescriptor(0, 
-                                                    fs.getLength(inNames[i]), inNames[i]);
+        SegmentDescriptor s = new SegmentDescriptor(0,
+            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
         s.preserveInput(!deleteInputs);
         s.doSync();
         a.add(s);
@@ -2951,7 +2951,8 @@
             this.close();
             
             SegmentDescriptor tempSegment = 
-              new SegmentDescriptor(0, fs.getLength(outputFile), outputFile);
+              new SegmentDescriptor(0,
+                  fs.getFileStatus(outputFile).getLen(), outputFile);
             //put the segment back in the TreeMap
             sortedSegmentSizes.put(tempSegment, null);
             numSegments = sortedSegmentSizes.size();
@@ -3168,7 +3169,7 @@
       public SegmentContainer(Path inName, Path indexIn) throws IOException {
         //get the segments from indexIn
         FSDataInputStream fsIndexIn = fs.open(indexIn);
-        long end = fs.getLength(indexIn);
+        long end = fs.getFileStatus(indexIn).getLen();
         while (fsIndexIn.getPos() < end) {
           long segmentOffset = WritableUtils.readVLong(fsIndexIn);
           long segmentLength = WritableUtils.readVLong(fsIndexIn);

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java Tue Dec 23 14:14:25 2008
@@ -211,14 +211,6 @@
   }
 
   /**
-   * Get rid of Path f, whether a true file or dir.
-   */
-  @Deprecated
-  public boolean delete(Path f) throws IOException {
-    return dfs.delete(getPathName(f));
-  }
-  
-  /**
    * requires a boolean check to delete a non 
    * empty directory recursively.
    */

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java Tue Dec 23 14:14:25 2008
@@ -328,15 +328,6 @@
     throw new IOException("Not supported");
   }
 
-  @Override
-  /*
-   * @deprecated Use delete(path, boolean)
-   */
-  @Deprecated
-  public boolean delete(Path f) throws IOException {
-    throw new IOException("Not supported");
-  }
-  
   @Override 
   public boolean delete(Path f, boolean recursive) throws IOException {
     throw new IOException("Not supported");

Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobConf.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobConf.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobConf.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/JobConf.java Tue Dec 23 14:14:25 2008
@@ -326,14 +326,14 @@
   public void deleteLocalFiles() throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(this).delete(new Path(localDirs[i]));
+      FileSystem.getLocal(this).delete(new Path(localDirs[i]), true);
     }
   }
 
   public void deleteLocalFiles(String subdir) throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir));
+      FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir), true);
     }
   }
 

Modified: hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java (original)
+++ hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java Tue Dec 23 14:14:25 2008
@@ -156,7 +156,7 @@
                 Path mapOut = this.mapoutputFile.getOutputFile(mapId);
                 Path reduceIn = this.mapoutputFile.getInputFileForWrite(
                                   mapId.getTaskID(),reduceId,
-                                  localFs.getLength(mapOut));
+                                  localFs.getFileStatus(mapOut).getLen());
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                   throw new IOException("Mkdirs failed to create "
                       + reduceIn.getParent().toString());

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java Tue Dec 23 14:14:25 2008
@@ -110,7 +110,7 @@
       nrFiles++;
       // For a regular file generate <fName,offset> pairs
       long blockSize = fs.getDefaultBlockSize();
-      long fileLength = fs.getLength(rootFile);
+      long fileLength = fs.getFileStatus(rootFile).getLen();
       for(long offset = 0; offset < fileLength; offset += blockSize)
         writer.append(new UTF8(rootFile.toString()), new LongWritable(offset));
       return;

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java Tue Dec 23 14:14:25 2008
@@ -66,7 +66,7 @@
 
   private static void rmBufferDirs() throws IOException {
     assertTrue(!localFs.exists(BUFFER_PATH_ROOT) ||
-        localFs.delete(BUFFER_PATH_ROOT));
+        localFs.delete(BUFFER_PATH_ROOT, true));
   }
   
   private void validateTempDirCreation(int i) throws IOException {

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java Tue Dec 23 14:14:25 2008
@@ -17,14 +17,12 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+
 import junit.framework.TestCase;
-import java.io.*;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * test for the input truncation bug when mark/reset is used.
@@ -65,7 +63,7 @@
       Path testFile = new Path(TEST_ROOT_DIR, "HADOOP-1489");
       writeFile(fileSys, testFile, fileSize);
       assertTrue(fileSys.exists(testFile));
-      assertTrue(fileSys.getLength(testFile) == fileSize);
+      assertTrue(fileSys.getFileStatus(testFile).getLen() == fileSize);
 
       // Now read the file for ioBufSize bytes
       FSDataInputStream in = fileSys.open(testFile, ioBufSize);

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java Tue Dec 23 14:14:25 2008
@@ -21,17 +21,15 @@
 
 package org.apache.hadoop.fs.kfs;
 
-import java.io.*;
+import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-
+import org.apache.hadoop.fs.BlockLocation;
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.BlockLocation;
 
 
 public class KFSEmulationImpl implements IFSImpl {
@@ -99,7 +97,7 @@
     }
 
     public long filesize(String path) throws IOException {
-        return localFS.getLength(new Path(path));
+        return localFS.getFileStatus(new Path(path)).getLen();
     }
     public short getReplication(String path) throws IOException {
         return 1;

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/UtilsForTests.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/UtilsForTests.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/UtilsForTests.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/UtilsForTests.java Tue Dec 23 14:14:25 2008
@@ -18,15 +18,18 @@
 
 package org.apache.hadoop.mapred;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.text.DecimalFormat;
-import java.io.*;
 import java.util.Arrays;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.examples.RandomWriter;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -36,7 +39,6 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 
@@ -155,7 +157,7 @@
   }
 
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
-    int len = (int) fs.getLength(p);
+    int len = (int) fs.getFileStatus(p).getLen();
     byte[] buf = new byte[len];
     InputStream in = fs.open(p);
     String contents = null;

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java?rev=729141&r1=729140&r2=729141&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java Tue Dec 23 14:14:25 2008
@@ -22,10 +22,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 
-import junit.framework.Test;
 import junit.framework.TestCase;
-import junit.framework.TestSuite;
-import junit.extensions.TestSetup;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -117,7 +114,7 @@
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
   }
 
@@ -137,7 +134,7 @@
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
   }
 
@@ -184,7 +181,7 @@
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
   }