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 en...@apache.org on 2008/03/17 10:38:45 UTC

svn commit: r637803 - in /hadoop/core/trunk: ./ src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/ src/java/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/fs/ src/test/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/rec...

Author: enis
Date: Mon Mar 17 02:38:42 2008
New Revision: 637803

URL: http://svn.apache.org/viewvc?rev=637803&view=rev
Log:
HADOOP-2822.  Remove depreceted code for classes InputFormatBase and PhasedFileSystem. Contributed by Amareshwari Sriramadasu.


Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt
    hadoop/core/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java
    hadoop/core/trunk/src/java/org/apache/hadoop/mapred/PhasedFileSystem.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java
    hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/EmptyInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java
    hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
    hadoop/core/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Mon Mar 17 02:38:42 2008
@@ -34,6 +34,9 @@
     and isDir(String) from ClientProtocol. ClientProtocol version changed
     from 26 to 27. (Tsz Wo (Nicholas), SZE via cdouglas)
 
+    HADOOP-2822. Remove depreceted code for classes InputFormatBase and 
+    PhasedFileSystem. (Amareshwari Sriramadasu via enis)
+
   NEW FEATURES
 
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)

Modified: hadoop/core/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt (original)
+++ hadoop/core/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt Mon Mar 17 02:38:42 2008
@@ -23,7 +23,7 @@
 [:~]$ $HADOOP_HOME/bin/hadoop jar hadoop-datajoin-examples.jar org.apache.hadoop.contrib.utils.join.DataJoinJob datajoin/input datajoin/output Text 1 org.apache.hadoop.contrib.utils.join.SampleDataJoinMapper org.apache.hadoop.contrib.utils.join.SampleDataJoinReducer org.apache.hadoop.contrib.utils.join.SampleTaggedMapOutput Text
 Using TextInputFormat: Text
 Using TextOutputFormat: Text
-07/06/01 19:58:23 INFO mapred.InputFormatBase: Total input paths to process : 2
+07/06/01 19:58:23 INFO mapred.FileInputFormat: Total input paths to process : 2
 Job job_kkzk08 is submitted
 Job job_kkzk08 is still running.
 07/06/01 19:58:24 INFO mapred.LocalJobRunner: collectedCount    5

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java Mon Mar 17 02:38:42 2008
@@ -1,27 +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.mapred;
-
-/** A base class for {@link InputFormat}. 
- *  @deprecated replaced by {@link FileInputFormat}
- */
-public abstract class InputFormatBase extends FileInputFormat {
-
-}
-

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/mapred/PhasedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/mapred/PhasedFileSystem.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/mapred/PhasedFileSystem.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/mapred/PhasedFileSystem.java Mon Mar 17 02:38:42 2008
@@ -1,351 +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.mapred;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.net.URI;
-
-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.FilterFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.Progressable;
-
-/**
- * This class acts as a proxy to the actual file system being used. 
- * It writes files to a temporary location and on
- * commit, moves to final location. On abort or a failure in 
- * commit the temporary file is deleted  
- * PhasedFileSystem works in context of a task. A different instance of 
- * PhasedFileSystem should be used for every task.  
- * Temporary files are written in  ("mapred.system.dir")/<jobid>/<taskid>
- * If one tasks opens a large number of files in succession then its 
- * better to commit(Path) individual files when done. Otherwise
- * commit() can be used to commit all open files at once.
- * 
- * @deprecated {@link PhasedFileSystem} is no longer used
- *             during speculative execution of tasks.
- */
-public class PhasedFileSystem extends FilterFileSystem {
-  // Map from final file name to temporary file name
-  private Map<Path, FileInfo> finalNameToFileInfo = new HashMap<Path, FileInfo>(); 
-  
-  private String jobid; 
-  private String tipid; 
-  private String taskid; 
-  
-  private Path tempDir; 
-  /**
-   * This Constructor is used to wrap a FileSystem object to a 
-   * Phased FilsSystem.  
-   * @param fs base file system object
-   * @param jobid JobId
-   * @param tipid tipId 
-   * @param taskid taskId
-   */
-  public PhasedFileSystem(FileSystem fs, String jobid, 
-                          String tipid, String taskid) {
-    super(fs); 
-    this.jobid = jobid; 
-    this.tipid = tipid; 
-    this.taskid = taskid; 
-    
-    tempDir = new Path(fs.getConf().get("mapred.system.dir")); 
-    this.setConf(fs.getConf());
-  }
-  /**
-   * This Constructor is used to wrap a FileSystem object to a 
-   * Phased FilsSystem.  
-   * @param fs base file system object
-   * @param conf JobConf
-   */
-  public PhasedFileSystem(FileSystem fs, JobConf conf) {
-    super(fs); 
-    this.jobid = conf.get("mapred.job.id"); 
-    this.tipid = conf.get("mapred.tip.id"); 
-    this.taskid = conf.get("mapred.task.id"); 
-    
-    tempDir = new Path(fs.getConf().get("mapred.system.dir"));
-    this.setConf(fs.getConf());
-  }
-  
-  private Path setupFile(Path finalFile, boolean overwrite) throws IOException{
-    if (finalNameToFileInfo.containsKey(finalFile)){
-      if (!overwrite){
-        throw new IOException("Error, file already exists : " + 
-                              finalFile.toString()); 
-      }else{
-        // delete tempp file and let create a new one. 
-        FileInfo fInfo = finalNameToFileInfo.get(finalFile); 
-        try{
-          fInfo.getOpenFileStream().close();
-        }catch(IOException ioe){
-          // ignore if already closed
-        }
-        if (fs.exists(fInfo.getTempPath())){
-          fs.delete(fInfo.getTempPath(), true);
-        }
-        finalNameToFileInfo.remove(finalFile); 
-      }
-    }
-    
-    String uniqueFile = jobid + "/" + tipid + "/" + taskid + "/" + finalFile.getName();
-    
-    Path tempPath = new Path(tempDir, new Path(uniqueFile)); 
-    FileInfo fInfo = new FileInfo(tempPath, finalFile, overwrite); 
-    
-    finalNameToFileInfo.put(finalFile, fInfo);
-    
-    return tempPath; 
-  }
-  
-  public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
-                                   short replication, long blockSize, Progressable progress)
-    throws IOException {
-    if (fs.exists(f) && !overwrite){
-      throw new IOException("Error creating file - already exists : " + f); 
-    }
-    FSDataOutputStream stream = 
-      fs.create(setupFile(f, overwrite), overwrite, bufferSize, replication, 
-                blockSize, progress);
-    finalNameToFileInfo.get(f).setOpenFileStream(stream); 
-    return stream; 
-  }
-  
-  /**
-   * Commits a single file file to its final locations as passed in create* methods. 
-   * If a file already exists in final location then temporary file is deleted. 
-   * @param fPath path to final file. 
-   * @throws IOException thrown if commit fails
-   */
-  public void commit(Path fPath) throws IOException{
-    commit(fPath, true); 
-  }
-  
-  // use extra method arg to avoid concurrentModificationException 
-  // if committing using this method while iterating.  
-  private void commit(Path fPath , boolean removeFromMap)throws IOException{
-    FileInfo fInfo = finalNameToFileInfo.get(fPath); 
-    if (null == fInfo){
-      throw new IOException("Error committing file! File was not created " + 
-                            "with PhasedFileSystem : " + fPath); 
-    }
-    try{
-      fInfo.getOpenFileStream().close();
-    }catch(IOException ioe){
-      // ignore if already closed
-      LOG.debug("Ignoring exception : " + ioe.getMessage());
-    }
-    Path tempPath = fInfo.getTempPath(); 
-    // ignore .crc files 
-    if (!tempPath.toString().endsWith(".crc")){
-      if (!fs.exists(fPath) || fInfo.isOverwrite()){
-        if (!fs.exists(fPath.getParent())){
-          fs.mkdirs(fPath.getParent());
-        }
-        
-        if (fs.exists(fPath) && fInfo.isOverwrite()){
-          fs.delete(fPath, true); 
-        }
-        
-        try {
-          if (!fs.rename(fInfo.getTempPath(), fPath)){
-            // delete the temp file if rename failed
-            fs.delete(fInfo.getTempPath(), true);
-          }
-        }catch(IOException ioe){
-          // rename failed, log error and delete temp files
-          LOG.error("PhasedFileSystem failed to commit file : " + fPath 
-                    + " error : " + ioe.getMessage()); 
-          fs.delete(fInfo.getTempPath(), true);
-        }
-      }else{
-        // delete temp file
-        fs.delete(fInfo.getTempPath(), true);
-      }
-      // done with the file
-      if (removeFromMap){
-        finalNameToFileInfo.remove(fPath);
-      }
-    }
-  }
-
-  /**
-   * Commits files to their final locations as passed in create* methods. 
-   * If a file already exists in final location then temporary file is deleted. 
-   * This methods ignores crc files (ending with .crc). This method doesnt close
-   * the file system so it can still be used to create new files. 
-   * @throws IOException if any file fails to commit
-   */
-  public void commit() throws IOException {
-    for(Path fPath : finalNameToFileInfo.keySet()){
-      commit(fPath, false);  
-    }
-    // safe to clear map now
-    finalNameToFileInfo.clear();
-  } 
-  /**
-   * Aborts a single file. The temporary created file is deleted. 
-   * @param p the path to final file as passed in create* call
-   * @throws IOException if File delete fails  
-   */
-  public void abort(Path p)throws IOException{
-    abort(p, true); 
-  }
-  
-  // use extra method arg to avoid concurrentModificationException 
-  // if aborting using this method while iterating.  
-  private void abort(Path p, boolean removeFromMap) throws IOException{
-    FileInfo fInfo = finalNameToFileInfo.get(p); 
-    if (null != fInfo){
-      try{
-        fInfo.getOpenFileStream().close();
-      }catch(IOException ioe){
-        // ignore if already closed
-      }
-      fs.delete(fInfo.getTempPath(), true); 
-      if (removeFromMap){
-        finalNameToFileInfo.remove(p);
-      }
-    }
-  }
-  /**
-   * Aborts the file creation, all uncommitted files created by this PhasedFileSystem 
-   * instance are deleted. This does not close baseFS because handle to baseFS may still 
-   * exist can be used to create new files. 
-   * 
-   * @throws IOException
-   */
-  public void abort() throws IOException {
-    for(Path fPath : finalNameToFileInfo.keySet()){
-      abort(fPath, false); 
-    }
-    // safe to clean now
-    finalNameToFileInfo.clear();
-  }
-  
-  @Override
-  public boolean setReplication(
-                                Path src, short replication)
-    throws IOException {
-    // throw IOException for interface compatibility with 
-    // base class. 
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public boolean rename(
-                        Path src, Path dst)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public boolean delete(
-                        Path f)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public void copyFromLocalFile(
-                                boolean delSrc, Path src, Path dst)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public void copyToLocalFile(
-                              boolean delSrc, Path src, Path dst)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public Path startLocalOutput(
-                               Path fsOutputFile, Path tmpLocalFile)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public void completeLocalOutput(
-                                  Path fsOutputFile, Path tmpLocalFile)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public String[][] getFileCacheHints(
-                                      Path f, long start, long len)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  @Override
-  public String getName() {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  private class FileInfo {
-    private Path tempPath;
-    private Path finalPath; 
-    private OutputStream openFileStream; 
-    private boolean overwrite;
-    
-    FileInfo(Path tempPath, Path finalPath, boolean overwrite){
-      this.tempPath = tempPath; 
-      this.finalPath = finalPath; 
-      this.overwrite = overwrite; 
-    }
-    public OutputStream getOpenFileStream() {
-      return openFileStream;
-    }
-    public void setOpenFileStream(
-                                  OutputStream openFileStream) {
-      this.openFileStream = openFileStream;
-    }
-    public Path getFinalPath() {
-      return finalPath;
-    }
-    public void setFinalPath(
-                             Path finalPath) {
-      this.finalPath = finalPath;
-    }
-    public boolean isOverwrite() {
-      return overwrite;
-    }
-    public void setOverwrite(
-                             boolean overwrite) {
-      this.overwrite = overwrite;
-    }
-    public Path getTempPath() {
-      return tempPath;
-    }
-    public void setTempPath(
-                            Path tempPath) {
-      this.tempPath = tempPath;
-    }
-  }
-}

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java Mon Mar 17 02:38:42 2008
@@ -67,7 +67,7 @@
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "DFSCIOTest_results.log";
   
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
   private static Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/DFSCIOTest");

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=637803&r1=637802&r2=637803&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 Mon Mar 17 02:38:42 2008
@@ -52,7 +52,7 @@
   private static final long MEGA = 0x100000;
   
   private static Configuration fsConfig = new Configuration();
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
   private static Path TEST_ROOT_DIR = new Path(System.getProperty("test.build.data","/benchmarks/DistributedFSCheck"));
   private static Path MAP_INPUT_DIR = new Path(TEST_ROOT_DIR, "map_input");
   private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java Mon Mar 17 02:38:42 2008
@@ -68,7 +68,7 @@
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "TestDFSIO_results.log";
   
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
   private static Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/TestDFSIO");

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/EmptyInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/EmptyInputFormat.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/EmptyInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/EmptyInputFormat.java Mon Mar 17 02:38:42 2008
@@ -25,13 +25,13 @@
   * InputFormat which simulates the absence of input data
   * by returning zero split.
   */
-public class EmptyInputFormat extends InputFormatBase {
+public class EmptyInputFormat<K, V> extends FileInputFormat<K, V> {
 
   public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits) throws IOException {
     return new FileSplit[0];
   }
 
-  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+  public RecordReader<K, V> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
     return new SequenceFileRecordReader(job, (FileSplit) split);
   }
 }

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java Mon Mar 17 02:38:42 2008
@@ -29,7 +29,7 @@
 import org.apache.commons.logging.*;
 
 public class TestSequenceFileAsBinaryInputFormat extends TestCase {
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
   private static final int RECORDS = 10000;
 
   public void testBinary() throws IOException {

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java Mon Mar 17 02:38:42 2008
@@ -29,7 +29,7 @@
 import org.apache.hadoop.conf.*;
 
 public class TestSequenceFileAsTextInputFormat extends TestCase {
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java Mon Mar 17 02:38:42 2008
@@ -29,7 +29,7 @@
 import org.apache.hadoop.conf.*;
 
 public class TestSequenceFileInputFilter extends TestCase {
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
 
   private static final int MAX_LENGTH = 15000;
   private static final Configuration conf = new Configuration();

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java Mon Mar 17 02:38:42 2008
@@ -29,7 +29,7 @@
 import org.apache.hadoop.conf.*;
 
 public class TestSequenceFileInputFormat extends TestCase {
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java?rev=637803&r1=637802&r2=637803&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/record/TestRecordWritable.java Mon Mar 17 02:38:42 2008
@@ -28,14 +28,14 @@
 import org.apache.commons.logging.*;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputFormatBase;
+import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 public class TestRecordWritable extends TestCase {
-  private static final Log LOG = InputFormatBase.LOG;
+  private static final Log LOG = FileInputFormat.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();