Explorar o código

HADOOP-655. Remove most deprecated code.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@471931 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting %!s(int64=18) %!d(string=hai) anos
pai
achega
fac2875931
Modificáronse 30 ficheiros con 93 adicións e 755 borrados
  1. 9 0
      CHANGES.txt
  2. 2 2
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java
  3. 2 2
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamOutputFormat.java
  4. 0 12
      src/java/org/apache/hadoop/dfs/DataNodeReport.java
  5. 0 3
      src/java/org/apache/hadoop/dfs/DatanodeInfo.java
  6. 0 16
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  7. 0 1
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  8. 0 12
      src/java/org/apache/hadoop/fs/DF.java
  9. 0 107
      src/java/org/apache/hadoop/fs/FileSystem.java
  10. 0 6
      src/java/org/apache/hadoop/fs/FileUtil.java
  11. 4 5
      src/java/org/apache/hadoop/io/ArrayFile.java
  12. 20 31
      src/java/org/apache/hadoop/io/MapFile.java
  13. 24 76
      src/java/org/apache/hadoop/io/SequenceFile.java
  14. 4 2
      src/java/org/apache/hadoop/io/SetFile.java
  15. 0 50
      src/java/org/apache/hadoop/ipc/RPC.java
  16. 0 11
      src/java/org/apache/hadoop/ipc/Server.java
  17. 0 5
      src/java/org/apache/hadoop/mapred/FileSplit.java
  18. 0 12
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  19. 2 2
      src/java/org/apache/hadoop/mapred/IsolationRunner.java
  20. 0 19
      src/java/org/apache/hadoop/mapred/JobConf.java
  21. 0 8
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  22. 0 7
      src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
  23. 0 136
      src/java/org/apache/hadoop/record/Utils.java
  24. 3 2
      src/java/org/apache/hadoop/util/CopyFiles.java
  25. 0 206
      src/java/org/apache/hadoop/util/LogFormatter.java
  26. 2 1
      src/test/org/apache/hadoop/io/TestArrayFile.java
  27. 2 2
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java
  28. 2 2
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
  29. 16 16
      src/test/org/apache/hadoop/record/test/TestMapRed.java
  30. 1 1
      src/test/org/apache/hadoop/record/test/TestWritable.java

+ 9 - 0
CHANGES.txt

@@ -3,6 +3,15 @@ Hadoop Change Log
 
 Trunk (unreleased changes)
 
+ 1. HADOOP-655.  Remove most deprecated code.  A few deprecated things
+    remain, notably UTF8 and some methods that are still required.
+    Also cleaned up constructors for SequenceFile, MapFile, SetFile,
+    and ArrayFile a bit.  (cutting)
+
+
+Release 0.8.0 - 2006-11-03
+
+
  1. HADOOP-477.  Extend contrib/streaming to scan the PATH environment
     variables when resolving executable program names.
     (Dhruba Borthakur via cutting) 

+ 2 - 2
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java

@@ -140,8 +140,8 @@ public class StreamInputFormat extends InputFormatBase {
     final long start = split.getStart();
     final long end = start + split.getLength();
 
-    String splitName = split.getFile() + ":" + start + "-" + end;
-    final FSDataInputStream in = fs.open(split.getFile());
+    String splitName = split.getPath() + ":" + start + "-" + end;
+    final FSDataInputStream in = fs.open(split.getPath());
 
     // will open the file and seek to the start of the split
     // Factory dispatch based on available params..

+ 2 - 2
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamOutputFormat.java

@@ -19,12 +19,12 @@
 package org.apache.hadoop.streaming;
 
 import java.io.IOException;
-import java.io.File;
 
 import org.apache.hadoop.mapred.*;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
@@ -39,7 +39,7 @@ public class StreamOutputFormat implements OutputFormat {
 
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job, String name, Progressable progr) throws IOException {
 
-    File file = new File(job.getOutputDir(), name);
+    Path file = new Path(job.getOutputPath(), name);
 
     final FSDataOutputStream out = fs.create(file);
 

+ 0 - 12
src/java/org/apache/hadoop/dfs/DataNodeReport.java

@@ -1,12 +0,0 @@
-package org.apache.hadoop.dfs;
-
-/** A report on the status of a DataNode.
- *
- * @see DistributedFileSystem#getDataNodeStats
- * @deprecated Use {@link DatanodeInfo} instead.
- */
-public class DataNodeReport extends DatanodeInfo {
-  public String toString() {
-    return super.getDatanodeReport();
-  }
-}

+ 0 - 3
src/java/org/apache/hadoop/dfs/DatanodeInfo.java

@@ -71,9 +71,6 @@ public class DatanodeInfo extends DatanodeID {
   /** number of active connections */
   public int getXceiverCount() { return xceiverCount; }
 
-  /** @deprecated Use {@link #getLastUpdate()} instead. */
-  public long lastUpdate() { return getLastUpdate(); }
-
   /** A formatted string for reporting the status of the DataNode. */
   public String getDatanodeReport() {
     StringBuffer buffer = new StringBuffer();

+ 0 - 16
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -326,15 +326,6 @@ class FSDirectory implements FSConstants {
       metricsRecord = Metrics.createRecord("dfs", "namenode");
     }
 
-    /** Create new dfs name directories.  Caution: this destroys all files
-     * in this filesystem.
-     * @deprecated use @link FSImage#format(File[], Configuration) instead */
-    public static void format(File[] dirs, Configuration conf) throws IOException {
-      for (int idx = 0; idx < dirs.length; idx++) {
-        FSImage.format( dirs[idx] );
-      }
-    }
-    
     /**
      * Shutdown the filestore
      */
@@ -670,13 +661,6 @@ class FSDirectory implements FSConstants {
         }
     }
 
-    /**
-     * @deprecated use @link #mkdirs(String) instead
-     */
-    public boolean mkdirs(UTF8 src) {
-        return mkdirs(src.toString());
-    }
-
     /**
      * Create directory entries for every item
      */

+ 0 - 1
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -2434,7 +2434,6 @@ class FSNamesystem implements FSConstants {
       
       /**
        * Checks consistency of the class state.
-       * @deprecated This is for debugging purposes.
        */
       void isConsistent() throws IOException {
         if( blockTotal == -1 && blockSafe == -1 ) {

+ 0 - 12
src/java/org/apache/hadoop/fs/DF.java

@@ -42,18 +42,6 @@ public class DF {
   private int percentUsed;
   private String mount;
   
-  /** @deprecated
-   */
-  public DF(String path, Configuration conf ) throws IOException {
-    this( new File(path), conf );
-  }
-
-  /** @deprecated
-   */
-  public DF(String path, long dfInterval) throws IOException {
-    this( new File(path), dfInterval );
-  }
-  
   public DF(File path, Configuration conf ) throws IOException {
     this( path, conf.getLong( "dfs.df.interval", DF.DF_INTERVAL_DEFAULT ));
   }

+ 0 - 107
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -139,11 +139,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract String[][] getFileCacheHints(Path f, long start, long len) throws IOException;
 
-    /** @deprecated Call {@link #open(Path)} instead. */
-    public FSDataInputStream open(File f) throws IOException {
-      return open(new Path(f.toString()));
-    }
-
     /**
      * Opens an FSDataInputStream at the indicated Path.
      * @param f the file name to open
@@ -167,11 +162,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract FSInputStream openRaw(Path f) throws IOException;
 
-    /** @deprecated Call {@link #create(Path)} instead. */
-    public FSDataOutputStream create(File f) throws IOException {
-      return create(new Path(f.toString()));
-    }
-
     /**
      * Opens an FSDataOutputStream at the indicated Path.
      * Files are overwritten by default.
@@ -317,11 +307,6 @@ public abstract class FileSystem extends Configured {
                                              long blockSize, Progressable progress)
       throws IOException;
     
-    /** @deprecated Call {@link #createNewFile(Path)} instead. */
-    public boolean createNewFile(File f) throws IOException {
-      return createNewFile(new Path(f.toString()));
-    }
-
     /**
      * Creates the given Path as a brand-new zero-length file.  If
      * create fails, or if it already existed, return false.
@@ -376,11 +361,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract boolean setReplicationRaw(Path src, short replication) throws IOException;
 
-    /** @deprecated Call {@link #rename(Path, Path)} instead. */
-    public boolean rename(File src, File dst) throws IOException {
-      return rename(new Path(src.toString()), new Path(dst.toString()));
-    }
-
     /**
      * Renames Path src to Path dst.  Can take place on local fs
      * or remote DFS.
@@ -414,11 +394,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract boolean renameRaw(Path src, Path dst) throws IOException;
 
-    /** @deprecated Call {@link #delete(Path)} instead. */
-    public boolean delete(File f) throws IOException {
-      return delete(new Path(f.toString()));
-    }
-
     /** Delete a file. */
     public boolean delete(Path f) throws IOException {
       if (isDirectory(f)) {
@@ -434,27 +409,12 @@ public abstract class FileSystem extends Configured {
      */
     public abstract boolean deleteRaw(Path f) throws IOException;
 
-    /** @deprecated call {@link #exists(Path)} instead */
-    public boolean exists(File f) throws IOException {
-      return exists(new Path(f.toString()));
-    }
-
     /** Check if exists. */
     public abstract boolean exists(Path f) throws IOException;
 
-    /** @deprecated Call {@link #isDirectory(Path)} instead. */
-    public boolean isDirectory(File f) throws IOException {
-      return isDirectory(new Path(f.toString()));
-    }
-
     /** True iff the named path is a directory. */
     public abstract boolean isDirectory(Path f) throws IOException;
 
-    /** @deprecated Call {@link #isFile(Path)} instead. */
-    public boolean isFile(File f) throws IOException {
-      return isFile(new Path(f.toString()));
-    }
-
     /** True iff the named path is a regular file. */
     public boolean isFile(Path f) throws IOException {
         if (exists(f) && ! isDirectory(f)) {
@@ -464,26 +424,9 @@ public abstract class FileSystem extends Configured {
         }
     }
     
-    /** @deprecated Call {@link #getLength(Path)} instead. */
-    public long getLength(File f) throws IOException {
-      return getLength(new Path(f.toString()));
-    }
-
     /** The number of bytes in a file. */
     public abstract long getLength(Path f) throws IOException;
 
-    /** @deprecated Call {@link #listPaths(Path)} instead. */
-    public File[] listFiles(File f) throws IOException {
-      Path[] paths = listPaths(new Path(f.toString()));
-      if (paths == null)
-        return null;
-      File[] result = new File[paths.length];
-      for (int i = 0 ; i < paths.length; i++) {
-        result[i] = new File(paths[i].toString());
-      }
-      return result;
-    }
-
     final private static PathFilter DEFAULT_FILTER = new PathFilter() {
       public boolean accept(Path file) {
         return !isChecksumFile(file);
@@ -498,23 +441,6 @@ public abstract class FileSystem extends Configured {
     /** List files in a directory. */
     public abstract Path[] listPathsRaw(Path f) throws IOException;
 
-    /** @deprecated Call {@link #listPaths(Path)} instead. */
-    public File[] listFiles(File f, final FileFilter filt) throws IOException {
-      Path[] paths = listPaths(new Path(f.toString()),
-                               new PathFilter() {
-                                 public boolean accept(Path p) {
-                                   return filt.accept(new File(p.toString()));
-                                 }
-                               });
-      if (paths == null)
-        return null;
-      File[] result = new File[paths.length];
-      for (int i = 0 ; i < paths.length; i++) {
-        result[i] = new File(paths[i].toString());
-      }
-      return result;
-    }
-    
     /** Filter raw files in a directory. */
     private void listPaths(ArrayList<Path> results, Path f, PathFilter filter)
       throws IOException {
@@ -773,11 +699,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract Path getWorkingDirectory();
     
-    /** @deprecated Call {@link #mkdirs(Path)} instead. */
-    public boolean mkdirs(File f) throws IOException {
-      return mkdirs(new Path(f.toString()));
-    }
-
     /**
      * Make the given file and all non-existent parents into
      * directories. Has the semantics of Unix 'mkdir -p'.
@@ -785,21 +706,11 @@ public abstract class FileSystem extends Configured {
      */
     public abstract boolean mkdirs(Path f) throws IOException;
 
-    /** @deprecated Call {@link #lock(Path,boolean)} instead. */
-    public void lock(File f, boolean shared) throws IOException {
-      lock(new Path(f.toString()), shared);
-    }
-
     /**
      * Obtain a lock on the given Path
      */
     public abstract void lock(Path f, boolean shared) throws IOException;
 
-    /** @deprecated Call {@link #release(Path)} instead. */
-    public void release(File f) throws IOException {
-      release(new Path(f.toString()));
-    }
-
     /**
      * Release the lock
      */
@@ -823,19 +734,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract void copyToLocalFile(Path src, Path dst) throws IOException;
 
-    /**
-     * the same as copyToLocalFile(Path src, File dst), except that
-     * the source is removed afterward.
-     */
-    // not implemented yet
-    //public abstract void moveToLocalFile(Path src, File dst) throws IOException;
-
-    /** @deprecated Call {@link #startLocalOutput(Path, Path)} instead. */
-    public File startLocalOutput(File src, File dst) throws IOException {
-      return new File(startLocalOutput(new Path(src.toString()),
-                                       new Path(dst.toString())).toString());
-    }
-
     /**
      * Returns a local File that the user can write output to.  The caller
      * provides both the eventual FS target name and the local working
@@ -844,11 +742,6 @@ public abstract class FileSystem extends Configured {
      */
     public abstract Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException;
 
-    /** @deprecated Call {@link #completeLocalOutput(Path, Path)} instead. */
-    public void completeLocalOutput(File src, File dst) throws IOException {
-      completeLocalOutput(new Path(src.toString()), new Path(dst.toString()));
-    }
-
     /**
      * Called when we're all done writing to the target.  A local FS will
      * do nothing, because we've written to exactly the right place.  A remote

+ 0 - 6
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -30,12 +30,6 @@ import org.apache.hadoop.conf.Configuration;
  */
 public class FileUtil {
   
-  /** @deprecated Call {@link #fullyDelete(File)}. */
-  public static boolean fullyDelete(File dir, Configuration conf)
-    throws IOException {
-    return fullyDelete(dir);
-  }
-
   /**
    * Delete a directory and all its contents.  If
    * we return false, the directory may be partially-deleted.

+ 4 - 5
src/java/org/apache/hadoop/io/ArrayFile.java

@@ -34,12 +34,11 @@ public class ArrayFile extends MapFile {
   public static class Writer extends MapFile.Writer {
     private LongWritable count = new LongWritable(0);
 
-    /** Create the named file for values of the named class.
-     * @deprecated specify {@link CompressionType} and {@link Progressable}
-     */
-    public Writer(FileSystem fs, String file, Class valClass)
+    /** Create the named file for values of the named class. */
+    public Writer(Configuration conf, FileSystem fs,
+                  String file, Class valClass)
       throws IOException {
-      super(fs, file, LongWritable.class, valClass);
+      super(conf, fs, file, LongWritable.class, valClass);
     }
 
     /** Create the named file for values of the named class. */

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

@@ -66,21 +66,25 @@ public class MapFile {
     private WritableComparable lastKey;
 
 
-    /** Create the named map for keys of the named class. */
+    /** Create the named map for keys of the named class.
+     * @deprecated
+     */
     public Writer(FileSystem fs, String dirName,
                   Class keyClass, Class valClass)
       throws IOException {
-      this(fs, dirName, WritableComparator.get(keyClass), valClass, false);
+      this(new Configuration(), fs, dirName,
+           WritableComparator.get(keyClass), valClass);
     }
 
-    /** Create the named map for keys of the named class.
-     * @deprecated specify {@link CompressionType} and {@link Progressable}
-     */
-    public Writer(FileSystem fs, String dirName,
-                  Class keyClass, Class valClass, boolean compress)
+    /** Create the named map for keys of the named class. */
+    public Writer(Configuration conf, FileSystem fs, String dirName,
+                  Class keyClass, Class valClass)
       throws IOException {
-      this(fs, dirName, WritableComparator.get(keyClass), valClass, compress);
+      this(conf, fs, dirName,
+           WritableComparator.get(keyClass), valClass,
+           SequenceFile.getCompressionType(conf));
     }
+
     /** Create the named map for keys of the named class. */
     public Writer(Configuration conf, FileSystem fs, String dirName,
                   Class keyClass, Class valClass,
@@ -97,32 +101,17 @@ public class MapFile {
       this(conf,fs,dirName,WritableComparator.get(keyClass),valClass,compress);
     }
 
-    /** Create the named map using the named key comparator.
-     * @deprecated specify {@link CompressionType} and {@link Progressable}
-     */
-    public Writer(FileSystem fs, String dirName,
+    /** Create the named map using the named key comparator. */
+    public Writer(Configuration conf, FileSystem fs, String dirName,
                   WritableComparator comparator, Class valClass)
       throws IOException {
-      this(fs, dirName, comparator, valClass, false);
-    }
-    /** Create the named map using the named key comparator.
-     * @deprecated specify {@link CompressionType} and {@link Progressable}
-     */
-    public Writer(FileSystem fs, String dirName,
-                  WritableComparator comparator, Class valClass,
-                  boolean compress)
-      throws IOException {
-
-      this(new Configuration(), fs, dirName, comparator, valClass,
-           compress ? CompressionType.RECORD : CompressionType.NONE);
+      this(conf, fs, dirName, comparator, valClass,
+           SequenceFile.getCompressionType(conf));
     }
-
-    /** Create the named map using the named key comparator.
-     * @deprecated specify a {@link Progressable}
-     */
+    /** Create the named map using the named key comparator. */
     public Writer(Configuration conf, FileSystem fs, String dirName,
-                  WritableComparator comparator, Class valClass,
-                  SequenceFile.CompressionType compress)
+                 WritableComparator comparator, Class valClass,
+                 SequenceFile.CompressionType compress)
       throws IOException {
       this(conf, fs, dirName, comparator, valClass, compress, null);
     }
@@ -484,7 +473,7 @@ public class MapFile {
     Writable key = (Writable)keyClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     Writable value = (Writable)valueClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     SequenceFile.Writer indexWriter = null;
-    if (!dryrun) indexWriter = new SequenceFile.Writer(fs, index, keyClass, LongWritable.class);
+    if (!dryrun) indexWriter = SequenceFile.createWriter(fs, conf, index, keyClass, LongWritable.class);
     try {
       long pos = 0L;
       LongWritable position = new LongWritable();

+ 24 - 76
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -88,6 +88,24 @@ public class SequenceFile {
     job.set("io.seqfile.compression.type", val.toString());
   }
   
+  /**
+   * Construct the preferred type of SequenceFile Writer.
+   * @param fs The configured filesystem. 
+   * @param conf The configuration.
+   * @param name The name of the file. 
+   * @param keyClass The 'key' type.
+   * @param valClass The 'value' type.
+   * @return Returns the handle to the constructed SequenceFile Writer.
+   * @throws IOException
+   */
+  public static Writer 
+  createWriter(FileSystem fs, Configuration conf, Path name, 
+      Class keyClass, Class valClass) 
+  throws IOException {
+    return createWriter(fs,conf,name,keyClass,valClass,
+                        getCompressionType(conf));
+  }
+  
   /**
    * Construct the preferred type of SequenceFile Writer.
    * @param fs The configured filesystem. 
@@ -372,73 +390,25 @@ public class SequenceFile {
       }
     }
 
-    /** @deprecated Call {@link #SequenceFile.Writer(FileSystem,Path,Class,Class)}. */
-    public Writer(FileSystem fs, String name, Class keyClass, Class valClass)
-      throws IOException {
-      this(fs, new Path(name), keyClass, valClass, false);
-    }
-
     /** Implicit constructor: needed for the period of transition!*/
     Writer()
     {}
     
-    /** Create the named file. */
-    /** @deprecated Call {@link #SequenceFile.Writer(FileSystem,Configuration,Path,Class,Class)}. */
-    public Writer(FileSystem fs, Path name, Class keyClass, Class valClass)
-      throws IOException {
-      this(fs, name, keyClass, valClass, false);
-    }
-    
-    /** Create the named file with write-progress reporter. */
-    /** @deprecated Call {@link #SequenceFile.Writer(FileSystem,Configuration,Path,Class,Class,Progressable)}. */
-    public Writer(FileSystem fs, Path name, Class keyClass, Class valClass,
-            Progressable progress)
-      throws IOException {
-      this(fs, name, keyClass, valClass, false, progress);
-    }
-    
-    /** Create the named file.
-     * @param compress if true, values are compressed.
-     */
-    /** @deprecated Call {@link #SequenceFile.Writer(FileSystem,Configuration,Path,Class,Class)}. */
-    public Writer(FileSystem fs, Path name,
-                  Class keyClass, Class valClass, boolean compress)
-      throws IOException {
-      init(name, fs.create(name), keyClass, valClass, compress, null); 
-
-      initializeFileHeader();
-      writeFileHeader();
-      finalizeFileHeader();
-    }
-    
-    /** Create the named file with write-progress reporter.
-     * @param compress if true, values are compressed.
-     */
-    /** @deprecated Call {@link #SequenceFile.Writer(FileSystem,Configuration,Path,Class,Class,Progressable)}. */
-    public Writer(FileSystem fs, Path name,
-                  Class keyClass, Class valClass, boolean compress,
-                  Progressable progress)
-      throws IOException {
-      init(name, fs.create(name, progress), keyClass, valClass, 
-          compress, null);
-      
-      initializeFileHeader();
-      writeFileHeader();
-      finalizeFileHeader();
-    }
-    
     /** Create the named file. */
     public Writer(FileSystem fs, Configuration conf, Path name, 
         Class keyClass, Class valClass)
       throws IOException {
-      this(fs, name, keyClass, valClass, false);
+      this(fs, conf, name, keyClass, valClass, null);
     }
     
     /** Create the named file with write-progress reporter. */
     public Writer(FileSystem fs, Configuration conf, Path name, 
         Class keyClass, Class valClass, Progressable progress)
       throws IOException {
-      this(fs, name, keyClass, valClass, false, progress);
+      init(name, fs.create(name, progress), keyClass, valClass, false, null);
+      initializeFileHeader();
+      writeFileHeader();
+      finalizeFileHeader();
     }
 
     /** Write to an arbitrary stream using a specified buffer size. */
@@ -561,22 +531,6 @@ public class SequenceFile {
       out.write(buffer.getData(), 0, buffer.getLength()); // data
     }
 
-    /** 
-     * Append a key/value pair. 
-     * @deprecated Call {@link #appendRaw(byte[], int, int, SequenceFile.ValueBytes)}. 
-     */
-    public synchronized void append(byte[] data, int start, int length,
-                                    int keyLength) throws IOException {
-      if (keyLength == 0)
-        throw new IOException("zero length keys not allowed");
-
-      checkAndWriteSync();                        // sync
-      out.writeInt(length);                       // total record length
-      out.writeInt(keyLength);                    // key portion length
-      out.write(data, start, length);             // data
-
-    }
-    
     public synchronized void appendRaw(
         byte[] keyData, int keyOffset, int keyLength, ValueBytes val) 
     throws IOException {
@@ -926,12 +880,6 @@ public class SequenceFile {
     private CompressionInputStream valInFilter = null;
     private DataInputStream valIn = null;
 
-    /** @deprecated Call {@link #SequenceFile.Reader(FileSystem,Path,Configuration)}.*/
-    public Reader(FileSystem fs, String file, Configuration conf)
-      throws IOException {
-      this(fs, new Path(file), conf);
-    }
-
     /** Open the named file. */
     public Reader(FileSystem fs, Path file, Configuration conf)
       throws IOException {
@@ -1304,7 +1252,7 @@ public class SequenceFile {
         return next(buffer);
       }
     }
-    
+
     public ValueBytes createValueBytes() {
       ValueBytes val = null;
       if (!decompress || blockCompressed) {

+ 4 - 2
src/java/org/apache/hadoop/io/SetFile.java

@@ -37,10 +37,12 @@ public class SetFile extends MapFile {
       super(fs, dirName, keyClass, NullWritable.class);
     }
 
-    /** Create the named set using the named key comparator. */
+    /** Create the named set using the named key comparator.
+     * @deprecated
+     */
     public Writer(FileSystem fs, String dirName, WritableComparator comparator)
       throws IOException {
-      super(fs, dirName, comparator, NullWritable.class);
+      super(new Configuration(), fs, dirName, comparator, NullWritable.class);
     }
 
     /** Create a set naming the element class and compression type. */

+ 0 - 50
src/java/org/apache/hadoop/ipc/RPC.java

@@ -292,46 +292,12 @@ public class RPC {
     return new Server(instance, conf, bindAddress,port, numHandlers, verbose);
   }
 
-  
-  /** Construct a server for a protocol implementation instance listening on a
-   * port.
-   * 
-   * @deprecated the bind address should always be specified
-   */
-  public static Server getServer(final Object instance, final int port, Configuration conf) {
-    return getServer(instance, port, 1, false, conf);
-  }
-
-  /** Construct a server for a protocol implementation instance listening on a
-   * port. 
-   *
-   * @deprecated the bind address should always be specified
-   */
-  public static Server getServer(final Object instance,final int port,
-                                 final int numHandlers,
-                                 final boolean verbose, Configuration conf) {
-    return new Server(instance, conf, port, numHandlers, verbose);
-  }
-  
-  
-  
   /** An RPC Server. */
   public static class Server extends org.apache.hadoop.ipc.Server {
     private Object instance;
     private Class implementation;
     private boolean verbose;
 
-    /** Construct an RPC server.
-     * @param instance the instance whose methods will be called
-     * @param conf the configuration to use
-     * @param port the port to listen for connections on
-     * 
-     * @deprecated the bind address should always be specified
-     */
-    public Server(Object instance, Configuration conf, int port) {
-      this(instance, conf,  "0.0.0.0", port, 1, false);
-    }
-
     /** Construct an RPC server.
      * @param instance the instance whose methods will be called
      * @param conf the configuration to use
@@ -358,22 +324,6 @@ public class RPC {
       this.verbose = verbose;
     }
 
-    /** Construct an RPC server.
-     * @param instance the instance whose methods will be called
-     * @param conf the configuration to use
-     * @param port the port to listen for connections on
-     * @param numHandlers the number of method handler threads to run
-     * @param verbose whether each call should be logged
-     * 
-     * @deprecated the bind address should always be specified
-     */
-    public Server(Object instance, Configuration conf,  int port,
-                  int numHandlers, boolean verbose) {
-      super("0.0.0.0", port, Invocation.class, numHandlers, conf);
-      this.instance = instance;
-      this.implementation = instance.getClass();
-      this.verbose = verbose;
-    }
     public Writable call(Writable param) throws IOException {
       try {
         Invocation call = (Invocation)param;

+ 0 - 11
src/java/org/apache/hadoop/ipc/Server.java

@@ -578,17 +578,6 @@ public abstract class Server {
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
     this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000);
   }
-  
-  
-  /** Constructs a server listening on the named port.  Parameters passed must
-   * be of the named class.  The <code>handlerCount</handlerCount> determines
-   * the number of handler threads that will be used to process calls.
-   * 
-   * @deprecated the bind address should always be specified
-   */
-  protected Server(int port, Class paramClass, int handlerCount, Configuration conf) {
-    this("0.0.0.0",port,paramClass,handlerCount,conf);
-  }
 
   /** Sets the timeout used for network i/o. */
   public void setTimeout(int timeout) { this.timeout = timeout; }

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

@@ -38,11 +38,6 @@ public class FileSplit implements Writable {
   
   FileSplit() {}
 
-  /** @deprecated Call {@link #FileSplit(Path,long,long)} instead. */
-  public FileSplit(File file, long start, long length) {
-    this(new Path(file.toString()), start, length);
-  }
-
   /** Constructs a split.
    *
    * @param file the file name

+ 0 - 12
src/java/org/apache/hadoop/mapred/InputFormatBase.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;                              // deprecated
 
 import java.util.ArrayList;
 
@@ -59,17 +58,6 @@ public abstract class InputFormatBase implements InputFormat {
                                                Reporter reporter)
     throws IOException;
 
-  /** @deprecated Call {@link #listFiles(FileSystem,JobConf)} instead. */
-  protected File[] listFiles(FileSystem fs, JobConf job)
-    throws IOException {
-    Path[] paths = listPaths(fs, job);
-    File[] result = new File[paths.length];
-    for (int i = 0 ; i < paths.length; i++) {
-      result[i] = new File(paths[i].toString());
-    }
-    return result;
-  }
-
   /** List input directories.
    * Subclasses may override to, e.g., select only files matching a regular
    * expression. 

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

@@ -111,8 +111,8 @@ public class IsolationRunner {
       Path f = namer.getInputFile(i, taskId);
       if(! fs.exists(f)) {
         LOG.info("Create missing input: " + f);
-        SequenceFile.Writer out = new SequenceFile.Writer(fs, f, keyClass,
-                                                          valueClass);
+        SequenceFile.Writer out =
+          SequenceFile.createWriter(fs, conf, f, keyClass, valueClass);
         out.close();
       }
     }    

+ 0 - 19
src/java/org/apache/hadoop/mapred/JobConf.java

@@ -140,28 +140,17 @@ public class JobConf extends Configuration {
     }
   }
 
-  /** @deprecated Call {@link #getLocalPath(String)} instead. */
-  public File getLocalFile(String subdir, String name) throws IOException {
-    return new File(getLocalPath(subdir+Path.SEPARATOR+name).toString());
-  }
-
   /** Constructs a local file name.  Files are distributed among configured
    * local directories.*/
   public Path getLocalPath(String pathString) throws IOException {
     return getLocalPath("mapred.local.dir", pathString);
   }
 
-  /** @deprecated Call {@link #setInputPath(Path)} instead.*/
-  public void setInputDir(File dir) { setInputPath(new Path(dir.toString())); }
-
   public void setInputPath(Path dir) {
     dir = new Path(getWorkingDirectory(), dir);
     set("mapred.input.dir", dir);
   }
 
-  /** @deprecated Call {@link #addInputPath(Path)} instead.*/
-  public void addInputDir(File dir) { addInputPath(new Path(dir.toString())); }
-
   public void addInputPath(Path dir) {
     dir = new Path(getWorkingDirectory(), dir);
     String dirs = get("mapred.input.dir");
@@ -260,19 +249,11 @@ public class JobConf extends Configuration {
     }
   }
   
-  /** @deprecated Call {@link #getOutputPath()} instead.*/
-  public File getOutputDir() { return new File(getOutputPath().toString()); }
-
   public Path getOutputPath() { 
     String name = get("mapred.output.dir");
     return name == null ? null: new Path(name);
   }
 
-  /** @deprecated Call {@link #setOutputPath(Path)} instead.*/
-  public void setOutputDir(File dir) {
-    setOutputPath(new Path(dir.toString()));
-  }
-
   public void setOutputPath(Path dir) {
     dir = new Path(getWorkingDirectory(), dir);
     set("mapred.output.dir", dir);

+ 0 - 8
src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;                              // deprecated
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -61,13 +60,6 @@ public class MapFileOutputFormat extends OutputFormatBase {
       };
   }
 
-  /** @deprecated Call {@link #getReaders(FileSystem, Path, Configuration)}. */
-  public static MapFile.Reader[] getReaders(FileSystem fs, File dir,
-                                            Configuration conf)
-    throws IOException {
-    return getReaders(fs, new Path(dir.toString()), conf);
-  }
-
   /** Open the output generated by this format. */
   public static MapFile.Reader[] getReaders(FileSystem fs, Path dir,
                                             Configuration conf)

+ 0 - 7
src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.File;                              // deprecated
 import java.util.Arrays;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -73,12 +72,6 @@ public class SequenceFileOutputFormat extends OutputFormatBase {
       };
   }
 
-  /** @deprecated Call {@link #getReaders(Configuration, Path)} instead. */
-  public static SequenceFile.Reader[] getReaders(Configuration conf, File dir) 
-    throws IOException {
-    return getReaders(conf, new Path(dir.toString()));
-  }
-
   /** Open the output generated by this format. */
   public static SequenceFile.Reader[] getReaders(Configuration conf, Path dir)
     throws IOException {

+ 0 - 136
src/java/org/apache/hadoop/record/Utils.java

@@ -36,142 +36,6 @@ public class Utils {
     private Utils() {
     }
    
-    /**
-     * Serializes an integer to a binary stream with zero-compressed encoding.
-     * For -120 <= i <= 127, only one byte is used with the actual value.
-     * For other values of i, the first byte value indicates whether the
-     * integer is positive or negative, and the number of bytes that follow.
-     * If the first byte value v is between -121 and -124, the following integer
-     * is positive, with number of bytes that follow are -(v+120).
-     * If the first byte value v is between -125 and -128, the following integer
-     * is negative, with number of bytes that follow are -(v+124). Bytes are
-     * stored in the high-non-zero-byte-first order.
-     *
-     * @param stream Binary output stream
-     * @param i Integer to be serialized
-     * @throws java.io.IOException 
-     * @deprecated replaced by {@link #org.apache.hadoop.io.WritableUtils.writeVInt}
-      */
-    static void writeInt(DataOutput stream, int i) throws IOException {
-        if (i >= -120 && i <= 127) {
-            stream.writeByte((byte)i);
-            return;
-        }
-        
-        int len = -120;
-        if (i < 0) {
-            i &= 0x7FFFFFFF; // reset the sign bit
-            len = -124;
-        }
-        
-        int tmp = i;
-        while (tmp != 0) {
-            tmp = tmp >> 8;
-            len--;
-        }
-        
-        stream.writeByte((byte)len);
-        
-        len = (len < -124) ? -(len + 124) : -(len+120);
-        
-        for (int idx = len; idx != 0; idx--) {
-            int shiftbits = (idx - 1) * 8;
-            int mask = 0xFF << shiftbits;
-            stream.writeByte((byte)((i & mask) >> shiftbits));
-        }
-    }
-    
-    /**
-     * Serializes a long to a binary stream with zero-compressed encoding.
-     * For -112 <= i <= 127, only one byte is used with the actual value.
-     * For other values of i, the first byte value indicates whether the
-     * long is positive or negative, and the number of bytes that follow.
-     * If the first byte value v is between -113 and -120, the following long
-     * is positive, with number of bytes that follow are -(v+112).
-     * If the first byte value v is between -121 and -128, the following long
-     * is negative, with number of bytes that follow are -(v+120). Bytes are
-     * stored in the high-non-zero-byte-first order.
-     * 
-     * @param stream Binary output stream
-     * @param i Long to be serialized
-     * @throws java.io.IOException
-     * @deprecated replaced by {@link #org.apache.hadoop.io.WritableUtils.writeVLong}
-     */
-    static void writeLong(DataOutput stream, long i) throws IOException {
-        if (i >= -112 && i <= 127) {
-            stream.writeByte((byte)i);
-            return;
-        }
-        
-        int len = -112;
-        if (i < 0) {
-            i &= 0x7FFFFFFFFFFFFFFFL; // reset the sign bit
-            len = -120;
-        }
-        
-        long tmp = i;
-        while (tmp != 0) {
-            tmp = tmp >> 8;
-            len--;
-        }
-        
-        stream.writeByte((byte)len);
-        
-        len = (len < -120) ? -(len + 120) : -(len + 112);
-        
-        for (int idx = len; idx != 0; idx--) {
-            int shiftbits = (idx - 1) * 8;
-            long mask = 0xFFL << shiftbits;
-            stream.writeByte((byte)((i & mask) >> shiftbits));
-        }
-    }
-    
-    /**
-     * Reads a zero-compressed encoded integer from input stream and returns it.
-     * @param stream Binary input stream
-     * @throws java.io.IOException 
-     * @return deserialized integer from stream.
-     * @deprecated replaced by {@link #org.apache.hadoop.io.WritableUtils.readVInt}
-     */
-    static int readInt(DataInput stream) throws IOException {
-        int len = stream.readByte();
-        if (len >= -120) {
-            return len;
-        }
-        len = (len < -124) ? -(len + 124) : -(len + 120);
-        byte[] barr = new byte[len];
-        stream.readFully(barr);
-        int i = 0;
-        for (int idx = 0; idx < len; idx++) {
-            i = i << 8;
-            i = i | (barr[idx] & 0xFF);
-        }
-        return i;
-    }
-    
-    /**
-     * Reads a zero-compressed encoded long from input stream and returns it.
-     * @param stream Binary input stream
-     * @throws java.io.IOException 
-     * @return deserialized long from stream.
-     * @deprecated replaced by {@link #org.apache.hadoop.io.WritableUtils.readVLong}
-      */
-    static long readLong(DataInput stream) throws IOException {
-        int len = stream.readByte();
-        if (len >= -112) {
-            return len;
-        }
-        len = (len < -120) ? -(len + 120) : -(len + 112);
-        byte[] barr = new byte[len];
-        stream.readFully(barr);
-        long i = 0;
-        for (int idx = 0; idx < len; idx++) {
-            i = i << 8;
-            i = i | (barr[idx] & 0xFF);
-        }
-        return i;
-    }
-    
     /**
      * equals function that actually compares two buffers.
      *

+ 3 - 2
src/java/org/apache/hadoop/util/CopyFiles.java

@@ -340,7 +340,7 @@ public class CopyFiles extends ToolBase {
       for(int idx=0; idx < numMaps; ++idx) {
         Path file = new Path(inDir, "part"+idx);
         SequenceFile.Writer writer = 
-          new SequenceFile.Writer(fileSys, file, Text.class, Text.class);
+          SequenceFile.createWriter(fileSys,conf,file,Text.class,Text.class);
         for (int ipath = idx; ipath < nFiles; ipath += numMaps) {
           String path = (String) finalPathList.get(ipath);
           writer.append(new Text(path), new Text(""));
@@ -500,7 +500,8 @@ public class CopyFiles extends ToolBase {
       for(int i=0; i < srcPaths.length; ++i) {
         Path ipFile = new Path(jobInputDir, "part" + i);
         SequenceFile.Writer writer = 
-          new SequenceFile.Writer(fileSystem, ipFile, Text.class, Text.class);
+          SequenceFile.createWriter(fileSystem, conf, ipFile,
+                                    Text.class, Text.class);
         writer.append(new Text(srcPaths[i]), new Text(""));
         writer.close();
       }

+ 0 - 206
src/java/org/apache/hadoop/util/LogFormatter.java

@@ -1,206 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.util;
-
-import java.util.logging.*;
-import java.io.*;
-import java.net.InetAddress;
-import java.text.*;
-import java.util.Date;
-
-import org.apache.hadoop.conf.Configuration;
-
-/** @deprecated use {@link org.apache.commons.logging.LogFactory} instead. */
-public class LogFormatter extends Formatter {
-  private static final String FORMAT = "yyMMdd HHmmss";
-  private static final String NEWLINE = System.getProperty("line.separator");
-
-  private final Date date = new Date();
-  private final SimpleDateFormat formatter = new SimpleDateFormat(FORMAT);
-
-  private static boolean loggedSevere= false;
-
-  private static boolean showTime = true;
-  private static boolean showThreadIDs = false;
-  
-  // install when this class is loaded
-  static {
-    Handler[] handlers = LogFormatter.getLogger("").getHandlers();
-    for (int i = 0; i < handlers.length; i++) {
-      handlers[i].setFormatter(new LogFormatter());
-      handlers[i].setLevel(Level.FINEST);
-    }
-  }
-
-  public static String initFileHandler( Configuration conf, String opName )
-      throws IOException {
-          String logDir=System.getProperty("hadoop.log.dir");
-          String userHome=System.getProperty("user.dir");
-          if( logDir==null ) {
-        	  logDir=System.getProperty("hadoop.home.dir");
-        	  if(logDir==null) {
-        		  logDir=userHome;
-        	  } else {
-                  logDir+=File.separator+"logs";   
-              }
-          }
-          
-          if(!logDir.equals(userHome)) {
-              File logDirFile = new File( logDir );
-              if(!logDirFile.exists()) {
-                  if(!logDirFile.mkdirs()) {
-                      logDir=userHome;
-                  }
-              } else if( !logDirFile.isDirectory()) {
-                  logDir=userHome;
-              }
-          }
-          
-          String hostname;
-          try {
-          	hostname=InetAddress.getLocalHost().getHostName();
-          	int index=hostname.indexOf('.');
-          	if( index != -1 ) {
-          		hostname=hostname.substring(0, index);
-          	}
-          } catch (java.net.UnknownHostException e) {
-          	hostname="localhost";
-          }
-          
-          String id = System.getProperty( "hadoop.id.str", 
-                                          System.getProperty("user.name") );
-          String logFile = logDir+File.separator+"hadoop-"+id
-               +"-"+opName+"-"+hostname+".log";
-
-          int logFileSize = conf.getInt( "hadoop.logfile.size", 10000000 );
-          int logFileCount = conf.getInt( "hadoop.logfile.count", 10 );
-          
-          FileHandler fh=new FileHandler(logFile, logFileSize, logFileCount, false);
-          fh.setFormatter(new LogFormatter());
-          fh.setLevel(Level.FINEST);
-          
-          Logger rootLogger = LogFormatter.getLogger("");
-          rootLogger.info( "directing logs to directory "+logDir );
-          
-          Handler[] handlers = rootLogger.getHandlers();
-          for( int i=0; i<handlers.length; i++ ) {
-          	rootLogger.removeHandler( handlers[i]);
-          }
-          rootLogger.addHandler(fh);
-          
-          return logFile;
-  }
-      
-  /** Gets a logger and, as a side effect, installs this as the default
-   * formatter. */
-  public static Logger getLogger(String name) {
-    // just referencing this class installs it
-    return Logger.getLogger(name);
-  }
-  
-  /** When true, time is logged with each entry. */
-  public static void showTime(boolean showTime) {
-    LogFormatter.showTime = showTime;
-  }
-
-  /** When set true, thread IDs are logged. */
-  public static void setShowThreadIDs(boolean showThreadIDs) {
-    LogFormatter.showThreadIDs = showThreadIDs;
-  }
-
-  /**
-   * Format the given LogRecord.
-   * @param record the log record to be formatted.
-   * @return a formatted log record
-   */
-  public synchronized String format(LogRecord record) {
-    StringBuffer buffer = new StringBuffer();
-
-    // the date
-    if (showTime) {
-      date.setTime(record.getMillis());
-      formatter.format(date, buffer, new FieldPosition(0));
-    }
-    
-    // the thread id
-    if (showThreadIDs) {
-      buffer.append(" 0x");
-      String threadID = Integer.toHexString(record.getThreadID());
-      for (int i = 0; i < 8 - threadID.length(); i++) 
-        buffer.append('0');
-      buffer.append(threadID);
-    }
-
-    // handle SEVERE specially
-    if (record.getLevel() == Level.SEVERE) {
-      buffer.append(" SEVERE");                   // flag it in log
-      loggedSevere= true;                         // set global flag
-    }
-
-    // the message
-    buffer.append(" ");
-    buffer.append(formatMessage(record));
-
-    buffer.append(NEWLINE);
-
-    if (record.getThrown() != null) {
-      try {
-        StringWriter sw = new StringWriter();
-        PrintWriter pw = new PrintWriter(sw);
-        record.getThrown().printStackTrace(pw);
-        pw.close();
-        buffer.append(sw.toString());
-      } catch (Exception ex) {
-      }
-    }
-    return buffer.toString();
-  }
-
-  /**
-   * Returns <code>true</code> if this <code>LogFormatter</code> has
-   * logged something at <code>Level.SEVERE</code>
-   */
-  public static boolean hasLoggedSevere() {
-    return loggedSevere;
-  }
-
-  /** Returns a stream that, when written to, adds log lines. */
-  public static PrintStream getLogStream(final Logger logger,
-                                         final Level level) {
-    return new PrintStream(new ByteArrayOutputStream() {
-        private int scan = 0;
-
-        private boolean hasNewline() {
-          for (; scan < count; scan++) {
-            if (buf[scan] == '\n')
-              return true;
-          }
-          return false;
-        }
-
-        public void flush() throws IOException {
-          if (!hasNewline())
-            return;
-          logger.log(level, toString().trim());
-          reset();
-          scan = 0;
-        }
-      }, true);
-  }
-}

+ 2 - 1
src/test/org/apache/hadoop/io/TestArrayFile.java

@@ -66,9 +66,10 @@ public class TestArrayFile extends TestCase {
 
   private static void writeTest(FileSystem fs, RandomDatum[] data, String file)
     throws IOException {
+    Configuration conf = new Configuration();
     MapFile.delete(fs, file);
     LOG.debug("creating with " + data.length + " debug");
-    ArrayFile.Writer writer = new ArrayFile.Writer(fs, file, RandomDatum.class);
+    ArrayFile.Writer writer = new ArrayFile.Writer(conf, fs, file, RandomDatum.class);
     writer.setIndexInterval(100);
     for (int i = 0; i < data.length; i++)
       writer.append(data[i]);

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

@@ -56,8 +56,8 @@ public class TestSequenceFileInputFilter extends TestCase {
   private static void createSequenceFile(int numRecords) throws Exception {
       // create a file with length entries
       SequenceFile.Writer writer =
-          new SequenceFile.Writer(fs, inFile,
-                  Text.class, BytesWritable.class);
+          SequenceFile.createWriter(fs, conf, inFile,
+                                    Text.class, BytesWritable.class);
       try {
           for (int i = 1; i <= numRecords; i++) {
               Text key = new Text(Integer.toString(i));

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

@@ -61,8 +61,8 @@ public class TestSequenceFileInputFormat extends TestCase {
 
       // create a file with length entries
       SequenceFile.Writer writer =
-        new SequenceFile.Writer(fs, file,
-                                IntWritable.class, BytesWritable.class);
+        SequenceFile.createWriter(fs, conf, file,
+                                  IntWritable.class, BytesWritable.class);
       try {
         for (int i = 0; i < length; i++) {
           IntWritable key = new IntWritable(i);

+ 16 - 16
src/test/org/apache/hadoop/record/test/TestMapRed.java

@@ -247,19 +247,19 @@ public class TestMapRed extends TestCase {
         // Write the answer key to a file.  
         //
         FileSystem fs = FileSystem.get(conf);
-        File testdir = new File("mapred.loadtest");
+        Path testdir = new Path("mapred.loadtest");
         if (!fs.mkdirs(testdir)) {
           throw new IOException("Mkdirs failed to create directory " + testdir.toString());
         }
 
-        File randomIns = new File(testdir, "genins");
+        Path randomIns = new Path(testdir, "genins");
         if (!fs.mkdirs(randomIns)) {
           throw new IOException("Mkdirs failed to create directory " + randomIns.toString());
         }
 
-        File answerkey = new File(randomIns, "answer.key");
+        Path answerkey = new Path(randomIns, "answer.key");
         SequenceFile.Writer out = SequenceFile.createWriter(fs, conf, 
-            new Path(answerkey.getPath()), RecInt.class, RecInt.class, 
+            answerkey, RecInt.class, RecInt.class, 
             CompressionType.NONE);
         try {
             for (int i = 0; i < range; i++) {
@@ -291,18 +291,18 @@ public class TestMapRed extends TestCase {
         // Because there's just one reduce task, we emit a single big
         // file of random numbers.
         //
-        File randomOuts = new File(testdir, "genouts");
+        Path randomOuts = new Path(testdir, "genouts");
         fs.delete(randomOuts);
 
 
         JobConf genJob = new JobConf(conf);
-        genJob.setInputDir(randomIns);
+        genJob.setInputPath(randomIns);
         genJob.setInputKeyClass(RecInt.class);
         genJob.setInputValueClass(RecInt.class);
         genJob.setInputFormat(SequenceFileInputFormat.class);
         genJob.setMapperClass(RandomGenMapper.class);
 
-        genJob.setOutputDir(randomOuts);
+        genJob.setOutputPath(randomOuts);
         genJob.setOutputKeyClass(RecInt.class);
         genJob.setOutputValueClass(RecString.class);
         genJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -340,16 +340,16 @@ public class TestMapRed extends TestCase {
         // you have multiple reduces at once.
         //
         int intermediateReduces = 10;
-        File intermediateOuts = new File(testdir, "intermediateouts");
+        Path intermediateOuts = new Path(testdir, "intermediateouts");
         fs.delete(intermediateOuts);
         JobConf checkJob = new JobConf(conf);
-        checkJob.setInputDir(randomOuts);
+        checkJob.setInputPath(randomOuts);
         checkJob.setInputKeyClass(RecInt.class);
         checkJob.setInputValueClass(RecString.class);
         checkJob.setInputFormat(SequenceFileInputFormat.class);
         checkJob.setMapperClass(RandomCheckMapper.class);
 
-        checkJob.setOutputDir(intermediateOuts);
+        checkJob.setOutputPath(intermediateOuts);
         checkJob.setOutputKeyClass(RecInt.class);
         checkJob.setOutputValueClass(RecString.class);
         checkJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -365,16 +365,16 @@ public class TestMapRed extends TestCase {
         // But by having a single reduce task here, we end up merging
         // all the files.
         //
-        File finalOuts = new File(testdir, "finalouts");        
+        Path finalOuts = new Path(testdir, "finalouts");        
         fs.delete(finalOuts);
         JobConf mergeJob = new JobConf(conf);
-        mergeJob.setInputDir(intermediateOuts);
+        mergeJob.setInputPath(intermediateOuts);
         mergeJob.setInputKeyClass(RecInt.class);
         mergeJob.setInputValueClass(RecString.class);
         mergeJob.setInputFormat(SequenceFileInputFormat.class);
         mergeJob.setMapperClass(MergeMapper.class);
         
-        mergeJob.setOutputDir(finalOuts);
+        mergeJob.setOutputPath(finalOuts);
         mergeJob.setOutputKeyClass(RecInt.class);
         mergeJob.setOutputValueClass(RecInt.class);
         mergeJob.setOutputFormat(SequenceFileOutputFormat.class);
@@ -390,8 +390,8 @@ public class TestMapRed extends TestCase {
         // in the original key.
         //
         boolean success = true;
-        File recomputedkey = new File(finalOuts, "part-00000");
-        SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey.getPath(), conf);
+        Path recomputedkey = new Path(finalOuts, "part-00000");
+        SequenceFile.Reader in = new SequenceFile.Reader(fs, recomputedkey, conf);
         int totalseen = 0;
         try {
             RecInt key = new RecInt();
@@ -431,7 +431,7 @@ public class TestMapRed extends TestCase {
         //
         // Write to "results" whether the test succeeded or not.
         //
-        File resultFile = new File(testdir, "results");
+        Path resultFile = new Path(testdir, "results");
         BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(fs.create(resultFile)));
         try {
             bw.write("Success=" + success + "\n");

+ 1 - 1
src/test/org/apache/hadoop/record/test/TestWritable.java

@@ -67,7 +67,7 @@ public class TestWritable extends TestCase {
 
       // create a file with length entries
       SequenceFile.Writer writer =
-        new SequenceFile.Writer(fs, file,
+        new SequenceFile.Writer(fs, conf, file,
                                 RecInt.class, RecBuffer.class);
       try {
         for (int i = 0; i < length; i++) {