فهرست منبع

HADOOP-2822. Remove depreceted code for classes InputFormatBase and PhasedFileSystem. Contributed by Amareshwari Sriramadasu.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@637803 13f79535-47bb-0310-9956-ffa450edef68
Enis Soztutar 17 سال پیش
والد
کامیت
52b438e6c9

+ 3 - 0
CHANGES.txt

@@ -34,6 +34,9 @@ Trunk (unreleased changes)
     and isDir(String) from ClientProtocol. ClientProtocol version changed
     and isDir(String) from ClientProtocol. ClientProtocol version changed
     from 26 to 27. (Tsz Wo (Nicholas), SZE via cdouglas)
     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
   NEW FEATURES
 
 
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)

+ 1 - 1
src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt

@@ -23,7 +23,7 @@ B.a31   B.a32
 [:~]$ $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
 [:~]$ $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 TextInputFormat: Text
 Using TextOutputFormat: 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 submitted
 Job job_kkzk08 is still running.
 Job job_kkzk08 is still running.
 07/06/01 19:58:24 INFO mapred.LocalJobRunner: collectedCount    5
 07/06/01 19:58:24 INFO mapred.LocalJobRunner: collectedCount    5

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

@@ -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 {
-
-}
-

+ 0 - 351
src/java/org/apache/hadoop/mapred/PhasedFileSystem.java

@@ -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;
-    }
-  }
-}

+ 1 - 1
src/test/org/apache/hadoop/fs/DFSCIOTest.java

@@ -67,7 +67,7 @@ public class DFSCIOTest extends TestCase {
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "DFSCIOTest_results.log";
   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 Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/DFSCIOTest");
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/DFSCIOTest");

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

@@ -52,7 +52,7 @@ public class DistributedFSCheck extends TestCase {
   private static final long MEGA = 0x100000;
   private static final long MEGA = 0x100000;
   
   
   private static Configuration fsConfig = new Configuration();
   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 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 MAP_INPUT_DIR = new Path(TEST_ROOT_DIR, "map_input");
   private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");
   private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");

+ 1 - 1
src/test/org/apache/hadoop/fs/TestDFSIO.java

@@ -68,7 +68,7 @@ public class TestDFSIO extends TestCase {
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "TestDFSIO_results.log";
   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 Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/TestDFSIO");
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/TestDFSIO");

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

@@ -25,13 +25,13 @@ import org.apache.hadoop.fs.FileSystem;
   * InputFormat which simulates the absence of input data
   * InputFormat which simulates the absence of input data
   * by returning zero split.
   * 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 {
   public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits) throws IOException {
     return new FileSplit[0];
     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);
     return new SequenceFileRecordReader(job, (FileSplit) split);
   }
   }
 }
 }

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

@@ -29,7 +29,7 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
 
 
 public class TestSequenceFileAsBinaryInputFormat extends TestCase {
 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;
   private static final int RECORDS = 10000;
 
 
   public void testBinary() throws IOException {
   public void testBinary() throws IOException {

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
 
 
 public class TestSequenceFileAsTextInputFormat extends TestCase {
 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 int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();
   private static Configuration conf = new Configuration();

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
 
 
 public class TestSequenceFileInputFilter extends TestCase {
 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 int MAX_LENGTH = 15000;
   private static final Configuration conf = new Configuration();
   private static final Configuration conf = new Configuration();

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.conf.*;
 
 
 public class TestSequenceFileInputFormat extends TestCase {
 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 int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();
   private static Configuration conf = new Configuration();

+ 2 - 2
src/test/org/apache/hadoop/record/TestRecordWritable.java

@@ -28,14 +28,14 @@ import org.apache.hadoop.conf.*;
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.InputFormat;
 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.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 
 public class TestRecordWritable extends TestCase {
 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 int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();
   private static Configuration conf = new Configuration();