You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by sz...@apache.org on 2012/02/04 04:40:56 UTC

svn commit: r1240450 [2/6] - in /hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project: ./ bin/ conf/ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/ hadoop-mapreduce-client/hadoop...

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileOutputCommitter.java Sat Feb  4 03:40:45 2012
@@ -19,14 +19,12 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.net.URI;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /** An {@link OutputCommitter} that commits files specified 
@@ -42,280 +40,140 @@ public class FileOutputCommitter extends
   /**
    * Temporary directory name 
    */
-  public static final String TEMP_DIR_NAME = "_temporary";
-  public static final String SUCCEEDED_FILE_NAME = "_SUCCESS";
-  static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = 
-    "mapreduce.fileoutputcommitter.marksuccessfuljobs";
-
-  public void setupJob(JobContext context) throws IOException {
+  public static final String TEMP_DIR_NAME = 
+    org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.PENDING_DIR_NAME;
+  public static final String SUCCEEDED_FILE_NAME = 
+    org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCEEDED_FILE_NAME;
+  static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER =
+    org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER;
+  
+  private static Path getOutputPath(JobContext context) {
+    JobConf conf = context.getJobConf();
+    return FileOutputFormat.getOutputPath(conf);
+  }
+  
+  private static Path getOutputPath(TaskAttemptContext context) {
     JobConf conf = context.getJobConf();
-    Path outputPath = FileOutputFormat.getOutputPath(conf);
-    if (outputPath != null) {
-      Path tmpDir = 
-          new Path(outputPath, getJobAttemptBaseDirName(context) + 
-              Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME);
-      FileSystem fileSys = tmpDir.getFileSystem(conf);
-      if (!fileSys.mkdirs(tmpDir)) {
-        LOG.error("Mkdirs failed to create " + tmpDir.toString());
-      }
+    return FileOutputFormat.getOutputPath(conf);
+  }
+  
+  private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter wrapped = null;
+  
+  private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter 
+  getWrapped(JobContext context) throws IOException {
+    if(wrapped == null) {
+      wrapped = new org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter(
+          getOutputPath(context), context);
     }
+    return wrapped;
   }
-
-  // True if the job requires output.dir marked on successful job.
-  // Note that by default it is set to true.
-  private boolean shouldMarkOutputDir(JobConf conf) {
-    return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true);
+  
+  private org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter 
+  getWrapped(TaskAttemptContext context) throws IOException {
+    if(wrapped == null) {
+      wrapped = new org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter(
+          getOutputPath(context), context);
+    }
+    return wrapped;
   }
   
-  public void commitJob(JobContext context) throws IOException {
-    //delete the task temp directory from the current jobtempdir
-    JobConf conf = context.getJobConf();
-    Path outputPath = FileOutputFormat.getOutputPath(conf);
-    if (outputPath != null) {
-      FileSystem outputFileSystem = outputPath.getFileSystem(conf);
-      Path tmpDir = new Path(outputPath, getJobAttemptBaseDirName(context) +
-          Path.SEPARATOR + FileOutputCommitter.TEMP_DIR_NAME);
-      FileSystem fileSys = tmpDir.getFileSystem(context.getConfiguration());
-      if (fileSys.exists(tmpDir)) {
-        fileSys.delete(tmpDir, true);
-      } else {
-        LOG.warn("Task temp dir could not be deleted " + tmpDir);
-      }
+  /**
+   * Compute the path where the output of a given job attempt will be placed. 
+   * @param context the context of the job.  This is used to get the
+   * application attempt id.
+   * @return the path to store job attempt data.
+   */
+  @Private
+  Path getJobAttemptPath(JobContext context) {
+    return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
+        .getJobAttemptPath(context, getOutputPath(context));
+  }
 
-      //move the job output to final place
-      Path jobOutputPath = 
-          new Path(outputPath, getJobAttemptBaseDirName(context));
-      moveJobOutputs(outputFileSystem, 
-          jobOutputPath, outputPath, jobOutputPath);
+  @Private
+  Path getTaskAttemptPath(TaskAttemptContext context) throws IOException {
+    return getTaskAttemptPath(context, getOutputPath(context));
+  }
 
-      // delete the _temporary folder in the output folder
-      cleanupJob(context);
-      // check if the output-dir marking is required
-      if (shouldMarkOutputDir(context.getJobConf())) {
-        // create a _success file in the output folder
-        markOutputDirSuccessful(context);
-      }
+  private Path getTaskAttemptPath(TaskAttemptContext context, Path out) throws IOException {
+    Path workPath = FileOutputFormat.getWorkOutputPath(context.getJobConf());
+    if(workPath == null) {
+      return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
+      .getTaskAttemptPath(context, out);
     }
+    return workPath;
   }
   
-  // Create a _success file in the job's output folder
-  private void markOutputDirSuccessful(JobContext context) throws IOException {
-    JobConf conf = context.getJobConf();
-    // get the o/p path
-    Path outputPath = FileOutputFormat.getOutputPath(conf);
-    if (outputPath != null) {
-      // get the filesys
-      FileSystem fileSys = outputPath.getFileSystem(conf);
-      // create a file in the output folder to mark the job completion
-      Path filePath = new Path(outputPath, SUCCEEDED_FILE_NAME);
-      fileSys.create(filePath).close();
-    }
+  /**
+   * Compute the path where the output of a committed task is stored until
+   * the entire job is committed.
+   * @param context the context of the task attempt
+   * @return the path where the output of a committed task is stored until
+   * the entire job is committed.
+   */
+  Path getCommittedTaskPath(TaskAttemptContext context) {
+    return org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
+        .getCommittedTaskPath(context, getOutputPath(context));
   }
 
-  private void moveJobOutputs(FileSystem fs, final Path origJobOutputPath,
-      Path finalOutputDir, Path jobOutput) throws IOException {
-    LOG.debug("Told to move job output from " + jobOutput
-        + " to " + finalOutputDir + 
-        " and orig job output path is " + origJobOutputPath);  
-    if (fs.isFile(jobOutput)) {
-      Path finalOutputPath = 
-          getFinalPath(fs, finalOutputDir, jobOutput, origJobOutputPath);
-      if (!fs.rename(jobOutput, finalOutputPath)) {
-        if (!fs.delete(finalOutputPath, true)) {
-          throw new IOException("Failed to delete earlier output of job");
-        }
-        if (!fs.rename(jobOutput, finalOutputPath)) {
-          throw new IOException("Failed to save output of job");
-        }
-      }
-      LOG.debug("Moved job output file from " + jobOutput + " to " + 
-          finalOutputPath);
-    } else if (fs.getFileStatus(jobOutput).isDirectory()) {
-      LOG.debug("Job output file " + jobOutput + " is a dir");      
-      FileStatus[] paths = fs.listStatus(jobOutput);
-      Path finalOutputPath = 
-          getFinalPath(fs, finalOutputDir, jobOutput, origJobOutputPath);
-      fs.mkdirs(finalOutputPath);
-      LOG.debug("Creating dirs along job output path " + finalOutputPath);
-      if (paths != null) {
-        for (FileStatus path : paths) {
-          moveJobOutputs(fs, origJobOutputPath, finalOutputDir, path.getPath());
-        }
-      }
-    }
+  public Path getWorkPath(TaskAttemptContext context, Path outputPath) 
+  throws IOException {
+    return getTaskAttemptPath(context, outputPath);
+  }
+  
+  @Override
+  public void setupJob(JobContext context) throws IOException {
+    getWrapped(context).setupJob(context);
+  }
+  
+  @Override
+  public void commitJob(JobContext context) throws IOException {
+    getWrapped(context).commitJob(context);
   }
   
   @Override
   @Deprecated
   public void cleanupJob(JobContext context) throws IOException {
-    JobConf conf = context.getJobConf();
-    // do the clean up of temporary directory
-    Path outputPath = FileOutputFormat.getOutputPath(conf);
-    if (outputPath != null) {
-      Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME);
-      FileSystem fileSys = tmpDir.getFileSystem(conf);
-      context.getProgressible().progress();
-      if (fileSys.exists(tmpDir)) {
-        fileSys.delete(tmpDir, true);
-      } else {
-        LOG.warn("Output Path is Null in cleanup");
-      }
-    }
+    getWrapped(context).cleanupJob(context);
   }
 
   @Override
   public void abortJob(JobContext context, int runState) 
   throws IOException {
-    // simply delete the _temporary dir from the o/p folder of the job
-    cleanupJob(context);
+    JobStatus.State state;
+    if(runState == JobStatus.State.RUNNING.getValue()) {
+      state = JobStatus.State.RUNNING;
+    } else if(runState == JobStatus.State.SUCCEEDED.getValue()) {
+      state = JobStatus.State.SUCCEEDED;
+    } else if(runState == JobStatus.State.FAILED.getValue()) {
+      state = JobStatus.State.FAILED;
+    } else if(runState == JobStatus.State.PREP.getValue()) {
+      state = JobStatus.State.PREP;
+    } else if(runState == JobStatus.State.KILLED.getValue()) {
+      state = JobStatus.State.KILLED;
+    } else {
+      throw new IllegalArgumentException(runState+" is not a valid runState.");
+    }
+    getWrapped(context).abortJob(context, state);
   }
   
   public void setupTask(TaskAttemptContext context) throws IOException {
-    // FileOutputCommitter's setupTask doesn't do anything. Because the
-    // temporary task directory is created on demand when the 
-    // task is writing.
+    getWrapped(context).setupTask(context);
   }
-		  
-  public void commitTask(TaskAttemptContext context) 
-  throws IOException {
-    Path taskOutputPath = getTempTaskOutputPath(context);
-    TaskAttemptID attemptId = context.getTaskAttemptID();
-    JobConf job = context.getJobConf();
-    if (taskOutputPath != null) {
-      FileSystem fs = taskOutputPath.getFileSystem(job);
-      context.getProgressible().progress();
-      if (fs.exists(taskOutputPath)) {
-        // Move the task outputs to the current job attempt output dir
-        JobConf conf = context.getJobConf();
-        Path outputPath = FileOutputFormat.getOutputPath(conf);
-        FileSystem outputFileSystem = outputPath.getFileSystem(conf);
-        Path jobOutputPath = new Path(outputPath, getJobTempDirName(context));
-        moveTaskOutputs(context, outputFileSystem, jobOutputPath, 
-            taskOutputPath);
-
-        // Delete the temporary task-specific output directory
-        if (!fs.delete(taskOutputPath, true)) {
-          LOG.info("Failed to delete the temporary output" + 
-          " directory of task: " + attemptId + " - " + taskOutputPath);
-        }
-        LOG.info("Saved output of task '" + attemptId + "' to " + 
-                 jobOutputPath);
-      }
-    }
-  }
-		  
-  private void moveTaskOutputs(TaskAttemptContext context,
-                               FileSystem fs,
-                               Path jobOutputDir,
-                               Path taskOutput) 
-  throws IOException {
-    TaskAttemptID attemptId = context.getTaskAttemptID();
-    context.getProgressible().progress();
-    LOG.debug("Told to move taskoutput from " + taskOutput
-        + " to " + jobOutputDir);    
-    if (fs.isFile(taskOutput)) {
-      Path finalOutputPath = getFinalPath(fs, jobOutputDir, taskOutput, 
-                                          getTempTaskOutputPath(context));
-      if (!fs.rename(taskOutput, finalOutputPath)) {
-        if (!fs.delete(finalOutputPath, true)) {
-          throw new IOException("Failed to delete earlier output of task: " + 
-                                 attemptId);
-        }
-        if (!fs.rename(taskOutput, finalOutputPath)) {
-          throw new IOException("Failed to save output of task: " + 
-        		  attemptId);
-        }
-      }
-      LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
-    } else if(fs.getFileStatus(taskOutput).isDirectory()) {
-      LOG.debug("Taskoutput " + taskOutput + " is a dir");
-      FileStatus[] paths = fs.listStatus(taskOutput);
-      Path finalOutputPath = getFinalPath(fs, jobOutputDir, taskOutput, 
-	          getTempTaskOutputPath(context));
-      fs.mkdirs(finalOutputPath);
-      LOG.debug("Creating dirs along path " + finalOutputPath);
-      if (paths != null) {
-        for (FileStatus path : paths) {
-          moveTaskOutputs(context, fs, jobOutputDir, path.getPath());
-        }
-      }
-    }
+  
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    getWrapped(context).commitTask(context, getTaskAttemptPath(context));
   }
 
+  @Override
   public void abortTask(TaskAttemptContext context) throws IOException {
-    Path taskOutputPath =  getTempTaskOutputPath(context);
-    if (taskOutputPath != null) {
-      FileSystem fs = taskOutputPath.getFileSystem(context.getJobConf());
-      context.getProgressible().progress();
-      fs.delete(taskOutputPath, true);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  private Path getFinalPath(FileSystem fs, Path jobOutputDir, Path taskOutput, 
-                            Path taskOutputPath) throws IOException {
-    URI taskOutputUri = taskOutput.makeQualified(fs).toUri();
-    URI taskOutputPathUri = taskOutputPath.makeQualified(fs).toUri();
-    URI relativePath = taskOutputPathUri.relativize(taskOutputUri);
-    if (taskOutputUri == relativePath) { 
-      //taskOutputPath is not a parent of taskOutput
-      throw new IOException("Can not get the relative path: base = " + 
-          taskOutputPathUri + " child = " + taskOutputUri);
-    }
-    if (relativePath.getPath().length() > 0) {
-      return new Path(jobOutputDir, relativePath.getPath());
-    } else {
-      return jobOutputDir;
-    }
+    getWrapped(context).abortTask(context, getTaskAttemptPath(context));
   }
 
+  @Override
   public boolean needsTaskCommit(TaskAttemptContext context) 
   throws IOException {
-    Path taskOutputPath = getTempTaskOutputPath(context);
-    if (taskOutputPath != null) {
-      context.getProgressible().progress();
-      // Get the file-system for the task output directory
-      FileSystem fs = taskOutputPath.getFileSystem(context.getJobConf());
-      // since task output path is created on demand, 
-      // if it exists, task needs a commit
-      if (fs.exists(taskOutputPath)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  Path getTempTaskOutputPath(TaskAttemptContext taskContext) 
-      throws IOException {
-    JobConf conf = taskContext.getJobConf();
-    Path outputPath = FileOutputFormat.getOutputPath(conf);
-    if (outputPath != null) {
-      Path p = new Path(outputPath,
-                     (FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
-                      "_" + taskContext.getTaskAttemptID().toString()));
-      FileSystem fs = p.getFileSystem(conf);
-      return p.makeQualified(fs);
-    }
-    return null;
-  }
-  
-  Path getWorkPath(TaskAttemptContext taskContext, Path basePath) 
-  throws IOException {
-    // ${mapred.out.dir}/_temporary
-    Path jobTmpDir = new Path(basePath, FileOutputCommitter.TEMP_DIR_NAME);
-    FileSystem fs = jobTmpDir.getFileSystem(taskContext.getJobConf());
-    if (!fs.exists(jobTmpDir)) {
-      throw new IOException("The temporary job-output directory " + 
-          jobTmpDir.toString() + " doesn't exist!"); 
-    }
-    // ${mapred.out.dir}/_temporary/_${taskid}
-    String taskid = taskContext.getTaskAttemptID().toString();
-    Path taskTmpDir = new Path(jobTmpDir, "_" + taskid);
-    if (!fs.mkdirs(taskTmpDir)) {
-      throw new IOException("Mkdirs failed to create " 
-          + taskTmpDir.toString());
-    }
-    return taskTmpDir;
+    return getWrapped(context).needsTaskCommit(context, getTaskAttemptPath(context));
   }
   
   @Override
@@ -326,54 +184,6 @@ public class FileOutputCommitter extends
   @Override
   public void recoverTask(TaskAttemptContext context)
       throws IOException {
-    Path outputPath = FileOutputFormat.getOutputPath(context.getJobConf());
-    context.progress();
-    Path jobOutputPath = new Path(outputPath, getJobTempDirName(context));
-    int previousAttempt =         
-        context.getConfiguration().getInt(
-            MRConstants.APPLICATION_ATTEMPT_ID, 0) - 1;
-    if (previousAttempt < 0) {
-      LOG.warn("Cannot recover task output for first attempt...");
-      return;
-    }
-
-    FileSystem outputFileSystem = 
-        outputPath.getFileSystem(context.getJobConf());
-    Path pathToRecover = 
-        new Path(outputPath, getJobAttemptBaseDirName(previousAttempt));
-    if (outputFileSystem.exists(pathToRecover)) {
-      // Move the task outputs to their final place
-      LOG.debug("Trying to recover task from " + pathToRecover
-          + " into " + jobOutputPath);
-      moveJobOutputs(outputFileSystem, 
-          pathToRecover, jobOutputPath, pathToRecover);
-      LOG.info("Saved output of job to " + jobOutputPath);
-    }
-  }
-
-  protected static String getJobAttemptBaseDirName(JobContext context) {
-    int appAttemptId = 
-        context.getJobConf().getInt(
-            MRConstants.APPLICATION_ATTEMPT_ID, 0);
-    return getJobAttemptBaseDirName(appAttemptId);
-  }
-
-  protected static String getJobTempDirName(TaskAttemptContext context) {
-    int appAttemptId = 
-        context.getJobConf().getInt(
-            MRConstants.APPLICATION_ATTEMPT_ID, 0);
-    return getJobAttemptBaseDirName(appAttemptId);
-  }
-
-  protected static String getJobAttemptBaseDirName(int appAttemptId) {
-    return FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR + 
-      + appAttemptId;
-  }
-
-  protected static String getTaskAttemptBaseDirName(
-      TaskAttemptContext context) {
-    return getJobTempDirName(context) + Path.SEPARATOR + 
-      FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
-      "_" + context.getTaskAttemptID().toString();
+    getWrapped(context).recoverTask(context);
   }
 }

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileSplit.java Sat Feb  4 03:40:45 2012
@@ -29,10 +29,7 @@ import org.apache.hadoop.fs.Path;
 /** A section of an input file.  Returned by {@link
  * InputFormat#getSplits(JobConf, int)} and passed to
  * {@link InputFormat#getRecordReader(InputSplit,JobConf,Reporter)}. 
- * @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.FileSplit}
- *  instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class FileSplit extends org.apache.hadoop.mapreduce.InputSplit 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ID.java Sat Feb  4 03:40:45 2012
@@ -30,7 +30,6 @@ import org.apache.hadoop.classification.
  * @see TaskID
  * @see TaskAttemptID
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public abstract class ID extends org.apache.hadoop.mapreduce.ID {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputFormat.java Sat Feb  4 03:40:45 2012
@@ -63,9 +63,7 @@ import org.apache.hadoop.fs.FileSystem;
  * @see RecordReader
  * @see JobClient
  * @see FileInputFormat
- * @deprecated Use {@link org.apache.hadoop.mapreduce.InputFormat} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface InputFormat<K, V> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/InputSplit.java Sat Feb  4 03:40:45 2012
@@ -34,9 +34,7 @@ import org.apache.hadoop.io.Writable;
  * 
  * @see InputFormat
  * @see RecordReader
- * @deprecated Use {@link org.apache.hadoop.mapreduce.InputSplit} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface InputSplit extends Writable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java Sat Feb  4 03:40:45 2012
@@ -29,6 +29,9 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.ClusterStatus.BlackListInfo;
 import org.apache.hadoop.mapreduce.Cluster;
 import org.apache.hadoop.mapreduce.ClusterMetrics;
@@ -40,13 +43,10 @@ import org.apache.hadoop.mapreduce.filec
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.tools.CLI;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -132,9 +132,7 @@ import org.apache.hadoop.util.ToolRunner
  * @see ClusterStatus
  * @see Tool
  * @see DistributedCache
- * @deprecated Use {@link Job} and {@link Cluster} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class JobClient extends CLI {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java Sat Feb  4 03:40:45 2012
@@ -20,7 +20,6 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
-
 import java.net.URL;
 import java.net.URLDecoder;
 import java.util.Enumeration;
@@ -28,24 +27,26 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.compress.CompressionCodec;
-
+import org.apache.hadoop.mapred.lib.HashPartitioner;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
-import org.apache.hadoop.mapred.lib.HashPartitioner;
 import org.apache.hadoop.mapred.lib.KeyFieldBasedComparator;
 import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -107,9 +108,7 @@ import org.apache.log4j.Level;
  * @see ClusterStatus
  * @see Tool
  * @see DistributedCache
- * @deprecated Use {@link Configuration} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class JobConf extends Configuration {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConfigurable.java Sat Feb  4 03:40:45 2012
@@ -22,7 +22,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 
 /** That what may be configured. */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface JobConfigurable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContext.java Sat Feb  4 03:40:45 2012
@@ -22,10 +22,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.Progressable;
 
-/**
- * @deprecated Use {@link org.apache.hadoop.mapreduce.JobContext} instead.
- */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface JobContext extends org.apache.hadoop.mapreduce.JobContext {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobContextImpl.java Sat Feb  4 03:40:45 2012
@@ -21,10 +21,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.Progressable;
 
-/**
- * @deprecated Use {@link org.apache.hadoop.mapreduce.JobContext} instead.
- */
-@Deprecated
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class JobContextImpl 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobID.java Sat Feb  4 03:40:45 2012
@@ -41,7 +41,6 @@ import org.apache.hadoop.classification.
  * @see TaskID
  * @see TaskAttemptID
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class JobID extends org.apache.hadoop.mapreduce.JobID {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobPriority.java Sat Feb  4 03:40:45 2012
@@ -22,9 +22,7 @@ import org.apache.hadoop.classification.
 
 /**
  * Used to describe the priority of the running job. 
- * @deprecated Use {@link org.apache.hadoop.mapreduce.JobPriority} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public enum JobPriority {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java Sat Feb  4 03:40:45 2012
@@ -29,9 +29,7 @@ import org.apache.hadoop.mapreduce.Queue
 /**
  * Class that contains the information regarding the Job Queues which are 
  * maintained by the Hadoop Map/Reduce framework.
- * @deprecated Use {@link QueueInfo} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class JobQueueInfo extends QueueInfo {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java Sat Feb  4 03:40:45 2012
@@ -29,9 +29,7 @@ import org.apache.hadoop.security.author
  * not intended to be a comprehensive piece of data.
  * For that, look at JobProfile.
  *************************************************
- *@deprecated Use {@link org.apache.hadoop.mapreduce.JobStatus} instead
  **/
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueLineRecordReader.java Sat Feb  4 03:40:45 2012
@@ -31,12 +31,7 @@ import org.apache.hadoop.io.Text;
  * separator character. The separator can be specified in config file 
  * under the attribute name mapreduce.input.keyvaluelinerecordreader.key.value.separator. The default
  * separator is the tab character ('\t').
- * 
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.KeyValueLineRecordReader} 
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class KeyValueLineRecordReader implements RecordReader<Text, Text> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/KeyValueTextInputFormat.java Sat Feb  4 03:40:45 2012
@@ -34,12 +34,7 @@ import org.apache.hadoop.io.compress.Spl
  * Either linefeed or carriage-return are used to signal end of line. Each line
  * is divided into key and value parts by a separator byte. If no such a byte
  * exists, the key will be the entire line and value will be empty.
- * 
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat} 
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class KeyValueTextInputFormat extends FileInputFormat<Text, Text>

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java Sat Feb  4 03:40:45 2012
@@ -41,10 +41,7 @@ import org.apache.commons.logging.Log;
 
 /**
  * Treats keys as offset in file and value as line. 
- * @deprecated Use 
- *   {@link org.apache.hadoop.mapreduce.lib.input.LineRecordReader} instead.
  */
-@Deprecated
 @InterfaceAudience.LimitedPrivate({"MapReduce", "Pig"})
 @InterfaceStability.Unstable
 public class LineRecordReader implements RecordReader<LongWritable, Text> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MROutputFiles.java Sat Feb  4 03:40:45 2012
@@ -23,7 +23,6 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRConfig;

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapFileOutputFormat.java Sat Feb  4 03:40:45 2012
@@ -36,10 +36,7 @@ import org.apache.hadoop.util.Progressab
 import org.apache.hadoop.util.ReflectionUtils;
 
 /** An {@link OutputFormat} that writes {@link MapFile}s.
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class MapFileOutputFormat 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapReduceBase.java Sat Feb  4 03:40:45 2012
@@ -23,7 +23,6 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.Closeable;
-import org.apache.hadoop.mapred.JobConfigurable;
 
 /** 
  * Base class for {@link Mapper} and {@link Reducer} implementations.
@@ -31,7 +30,6 @@ import org.apache.hadoop.mapred.JobConfi
  * <p>Provides default no-op implementations for a few methods, most non-trivial
  * applications need to override some of them.</p>
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class MapReduceBase implements Closeable, JobConfigurable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapRunnable.java Sat Feb  4 03:40:45 2012
@@ -30,9 +30,7 @@ import org.apache.hadoop.classification.
  * control on map processing e.g. multi-threaded, asynchronous mappers etc.</p>
  * 
  * @see Mapper
- * @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface MapRunnable<K1, V1, K2, V2>

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java Sat Feb  4 03:40:45 2012
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Mapper.java Sat Feb  4 03:40:45 2012
@@ -129,9 +129,7 @@ import org.apache.hadoop.io.compress.Com
  * @see MapReduceBase
  * @see MapRunnable
  * @see SequenceFile
- * @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface Mapper<K1, V1, K2, V2> extends JobConfigurable, Closeable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Master.java Sat Feb  4 03:40:45 2012
@@ -21,12 +21,16 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
+@Private
+@Unstable
 public class Master {
   
   public enum State {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileInputFormat.java Sat Feb  4 03:40:45 2012
@@ -36,9 +36,7 @@ import org.apache.hadoop.fs.Path;
  * Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)}
  * to construct <code>RecordReader</code>'s for <code>MultiFileSplit</code>'s.
  * @see MultiFileSplit
- * @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileInputFormat} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public abstract class MultiFileInputFormat<K, V>

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MultiFileSplit.java Sat Feb  4 03:40:45 2012
@@ -18,20 +18,16 @@
 
 package org.apache.hadoop.mapred;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.lib.CombineFileSplit;
 
 /**
@@ -42,9 +38,7 @@ import org.apache.hadoop.mapred.lib.Comb
  * reading one record per file.
  * @see FileSplit
  * @see MultiFileInputFormat 
- * @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileSplit} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class MultiFileSplit extends CombineFileSplit {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputCommitter.java Sat Feb  4 03:40:45 2012
@@ -56,9 +56,7 @@ import org.apache.hadoop.classification.
  * @see FileOutputCommitter 
  * @see JobContext
  * @see TaskAttemptContext 
- * @deprecated Use {@link org.apache.hadoop.mapreduce.OutputCommitter} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public abstract class OutputCommitter 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputFormat.java Sat Feb  4 03:40:45 2012
@@ -44,9 +44,7 @@ import org.apache.hadoop.util.Progressab
  * 
  * @see RecordWriter
  * @see JobConf
- * @deprecated Use {@link org.apache.hadoop.mapreduce.OutputFormat} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface OutputFormat<K, V> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/OutputLogFilter.java Sat Feb  4 03:40:45 2012
@@ -29,9 +29,6 @@ import org.apache.hadoop.fs.PathFilter;
  * This can be used to list paths of output directory as follows:
  *   Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
  *                                   new OutputLogFilter()));
- * @deprecated Use 
- *   {@link org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputLogFilter} 
- *   instead.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Partitioner.java Sat Feb  4 03:40:45 2012
@@ -32,9 +32,7 @@ import org.apache.hadoop.classification.
  * record) is sent for reduction.</p>
  * 
  * @see Reducer
- * @deprecated Use {@link org.apache.hadoop.mapreduce.Partitioner} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface Partitioner<K2, V2> extends JobConfigurable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/PeriodicStatsAccumulator.java Sat Feb  4 03:40:45 2012
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.mapred;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
 /**
  *
  * This abstract class that represents a bucketed series of
@@ -33,6 +36,8 @@ package org.apache.hadoop.mapred;
  *  bucket and how we interpret the readings by overriding
  *  {@code extendInternal(...)} and {@code initializeInterval()}
  */
+@Private
+@Unstable
 public abstract class PeriodicStatsAccumulator {
   // The range of progress from 0.0D through 1.0D is divided into
   //  count "progress segments".  This object accumulates an

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ProgressSplitsBlock.java Sat Feb  4 03:40:45 2012
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.mapred;
 
-import java.util.List;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 
 /*
  * This object gathers the [currently four] PeriodStatset's that we
  * are gathering for a particular task attempt for packaging and
  * handling as a single object.
  */
+@Private
+@Unstable
 public class ProgressSplitsBlock {
   final PeriodicStatsAccumulator progressWallclockTime;
   final PeriodicStatsAccumulator progressCPUTime;

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueAclsInfo.java Sat Feb  4 03:40:45 2012
@@ -20,9 +20,7 @@ package org.apache.hadoop.mapred;
 /**
  *  Class to encapsulate Queue ACLs for a particular
  *  user.
- * @deprecated Use {@link org.apache.hadoop.mapreduce.QueueAclsInfo} instead
  */
-@Deprecated
 class QueueAclsInfo extends org.apache.hadoop.mapreduce.QueueAclsInfo {
 
   /**

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RecordReader.java Sat Feb  4 03:40:45 2012
@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.io.DataInput;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Reducer.java Sat Feb  4 03:40:45 2012
@@ -162,9 +162,7 @@ import org.apache.hadoop.io.Closeable;
  * @see Partitioner
  * @see Reporter
  * @see MapReduceBase
- * @deprecated Use {@link org.apache.hadoop.mapreduce.Reducer} instead.
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface Reducer<K2, V2, K3, V3> extends JobConfigurable, Closeable {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/RunningJob.java Sat Feb  4 03:40:45 2012
@@ -34,9 +34,7 @@ import org.apache.hadoop.conf.Configurat
  * progress etc.</p> 
  * 
  * @see JobClient
- * @deprecated Use {@link org.apache.hadoop.mapreduce.Job} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface RunningJob {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryInputFormat.java Sat Feb  4 03:40:45 2012
@@ -27,20 +27,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 /**
  * InputFormat reading keys, values from SequenceFiles in binary (raw)
  * format.
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat}
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileAsBinaryInputFormat

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsBinaryOutputFormat.java Sat Feb  4 03:40:45 2012
@@ -23,26 +23,20 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
 
 /** 
  * An {@link OutputFormat} that writes keys, values to 
  * {@link SequenceFile}s in binary(raw) format
- * 
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat}
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileAsBinaryOutputFormat 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextInputFormat.java Sat Feb  4 03:40:45 2012
@@ -29,12 +29,7 @@ import org.apache.hadoop.io.Text;
  * except it generates SequenceFileAsTextRecordReader 
  * which converts the input keys and values to their 
  * String forms by calling toString() method.
- * 
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsTextInputFormat}
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileAsTextInputFormat

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileAsTextRecordReader.java Sat Feb  4 03:40:45 2012
@@ -31,11 +31,7 @@ import org.apache.hadoop.io.WritableComp
  * This class converts the input keys and values to their String forms by calling toString()
  * method. This class to SequenceFileAsTextInputFormat class is as LineRecordReader
  * class to TextInputFormat class.
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileAsTextRecordReader}
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileAsTextRecordReader

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFilter.java Sat Feb  4 03:40:45 2012
@@ -29,11 +29,7 @@ import org.apache.hadoop.util.Reflection
 /**
  * A class that allows a map/red job to work on a sample of sequence files.
  * The sample is decided by the filter class set by the job.
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFilter}
- * instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileInputFilter<K, V>

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java Sat Feb  4 03:40:45 2012
@@ -29,12 +29,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
 
-/** An {@link InputFormat} for {@link SequenceFile}s. 
- * @deprecated Use 
- *  {@link org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat} 
- *  instead.
+/** 
+ * An {@link InputFormat} for {@link SequenceFile}s. 
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileInputFormat<K, V> extends FileInputFormat<K, V> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java Sat Feb  4 03:40:45 2012
@@ -21,25 +21,22 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
-
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.*;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
 
-/** An {@link OutputFormat} that writes {@link SequenceFile}s. 
- * @deprecated Use 
- *   {@link org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat} 
- *   instead.
+/** 
+ * An {@link OutputFormat} that writes {@link SequenceFile}s. 
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileOutputFormat <K,V> extends FileOutputFormat<K, V> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java Sat Feb  4 03:40:45 2012
@@ -29,7 +29,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.util.ReflectionUtils;
 
-/** An {@link RecordReader} for {@link SequenceFile}s. */
+/** 
+ * An {@link RecordReader} for {@link SequenceFile}s. 
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SequenceFileRecordReader<K, V> implements RecordReader<K, V> {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java Sat Feb  4 03:40:45 2012
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.mapred;
 
-import java.io.DataInputStream;
-import java.io.File;
+import static org.apache.hadoop.mapred.MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.LongBuffer;
@@ -27,15 +27,12 @@ import java.util.zip.CheckedOutputStream
 import java.util.zip.Checksum;
 
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.SecureIOUtils;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.fs.FSDataInputStream;
-import static org.apache.hadoop.mapred.MapTask.MAP_OUTPUT_INDEX_RECORD_LENGTH;
 
 class SpillRecord {
 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java Sat Feb  4 03:40:45 2012
@@ -525,7 +525,7 @@ abstract public class Task implements Wr
     if (outputPath != null) {
       if ((committer instanceof FileOutputCommitter)) {
         FileOutputFormat.setWorkOutputPath(conf, 
-          ((FileOutputCommitter)committer).getTempTaskOutputPath(taskContext));
+          ((FileOutputCommitter)committer).getTaskAttemptPath(taskContext));
       } else {
         FileOutputFormat.setWorkOutputPath(conf, outputPath);
       }

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContext.java Sat Feb  4 03:40:45 2012
@@ -22,11 +22,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.Progressable;
 
-/**
- * @deprecated Use {@link org.apache.hadoop.mapreduce.TaskAttemptContext}
- *   instead.
- */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface TaskAttemptContext 

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java Sat Feb  4 03:40:45 2012
@@ -22,11 +22,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.util.Progressable;
 
-/**
- * @deprecated Use {@link org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl}
- *   instead.
- */
-@Deprecated
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TaskAttemptContextImpl

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskAttemptID.java Sat Feb  4 03:40:45 2012
@@ -45,7 +45,6 @@ import org.apache.hadoop.mapreduce.TaskT
  * @see JobID
  * @see TaskID
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class TaskAttemptID extends org.apache.hadoop.mapreduce.TaskAttemptID {

Modified: hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java?rev=1240450&r1=1240449&r2=1240450&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java (original)
+++ hadoop/common/branches/branch-0.23-PB/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskCompletionEvent.java Sat Feb  4 03:40:45 2012
@@ -24,10 +24,7 @@ import org.apache.hadoop.classification.
 /**
  * This is used to track task completion events on 
  * job tracker. 
- * @deprecated Use 
- * {@link org.apache.hadoop.mapreduce.TaskCompletionEvent} instead
  */
-@Deprecated
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class TaskCompletionEvent