Просмотр исходного кода

MAPREDUCE-4809. Make internal classes required for MAPREDUCE-2454 to be java public. Contributed by Mariappan Asokan.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-2454@1414753 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 12 лет назад
Родитель
Сommit
eb47858efc
10 измененных файлов с 50 добавлено и 17 удалено
  1. 5 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 8 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java
  3. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java
  4. 8 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
  5. 7 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ExceptionReporter.java
  6. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapHost.java
  7. 6 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java
  8. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java
  9. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java
  10. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleClientMetrics.java

+ 5 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -1,5 +1,10 @@
 Hadoop MapReduce Change Log
 
+Branch MR-2454
+
+    MAPREDUCE-4809. Make internal classes required for MAPREDUCE-2454 to be
+    java public. (Mariappan Asokan via acmurthy)
+
 Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES

+ 8 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java

@@ -34,6 +34,8 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -71,7 +73,9 @@ import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
 
 /** A Map task. */
-class MapTask extends Task {
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
+public class MapTask extends Task {
   /**
    * The size of each record in the index file for the map-outputs.
    */
@@ -814,7 +818,9 @@ class MapTask extends Task {
     }
   }
 
-  private class MapOutputBuffer<K extends Object, V extends Object>
+  @InterfaceAudience.LimitedPrivate({"MapReduce"})
+  @InterfaceStability.Unstable
+  public static class MapOutputBuffer<K extends Object, V extends Object>
       implements MapOutputCollector<K, V>, IndexedSortable {
     final int partitions;
     final JobConf job;

+ 4 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/SpillRecord.java

@@ -26,6 +26,8 @@ import java.util.zip.CheckedInputStream;
 import java.util.zip.CheckedOutputStream;
 import java.util.zip.Checksum;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -34,6 +36,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.PureJavaCrc32;
 
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
 class SpillRecord {
 
   /** Backing store */

+ 8 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java

@@ -584,9 +584,9 @@ abstract public class Task implements Writable, Configurable {
     return status;
   }
 
-  @InterfaceAudience.Private
+  @InterfaceAudience.LimitedPrivate({"MapReduce"})
   @InterfaceStability.Unstable
-  protected class TaskReporter 
+  public class TaskReporter 
       extends org.apache.hadoop.mapreduce.StatusReporter
       implements Runnable, Reporter {
     private TaskUmbilicalProtocol umbilical;
@@ -1466,9 +1466,9 @@ abstract public class Task implements Writable, Configurable {
     return reducerContext;
   }
 
-  @InterfaceAudience.Private
+  @InterfaceAudience.LimitedPrivate({"MapReduce"})
   @InterfaceStability.Unstable
-  protected static abstract class CombinerRunner<K,V> {
+  public static abstract class CombinerRunner<K,V> {
     protected final Counters.Counter inputCounter;
     protected final JobConf job;
     protected final TaskReporter reporter;
@@ -1486,13 +1486,13 @@ abstract public class Task implements Writable, Configurable {
      * @param iterator the key/value pairs to use as input
      * @param collector the output collector
      */
-    abstract void combine(RawKeyValueIterator iterator, 
+    public abstract void combine(RawKeyValueIterator iterator, 
                           OutputCollector<K,V> collector
                          ) throws IOException, InterruptedException, 
                                   ClassNotFoundException;
 
     @SuppressWarnings("unchecked")
-    static <K,V> 
+    public static <K,V> 
     CombinerRunner<K,V> create(JobConf job,
                                TaskAttemptID taskId,
                                Counters.Counter inputCounter,
@@ -1542,7 +1542,7 @@ abstract public class Task implements Writable, Configurable {
     }
 
     @SuppressWarnings("unchecked")
-    protected void combine(RawKeyValueIterator kvIter,
+    public void combine(RawKeyValueIterator kvIter,
                            OutputCollector<K,V> combineCollector
                            ) throws IOException {
       Reducer<K,V,K,V> combiner = 
@@ -1611,7 +1611,7 @@ abstract public class Task implements Writable, Configurable {
 
     @SuppressWarnings("unchecked")
     @Override
-    void combine(RawKeyValueIterator iterator, 
+    public void combine(RawKeyValueIterator iterator, 
                  OutputCollector<K,V> collector
                  ) throws IOException, InterruptedException,
                           ClassNotFoundException {

+ 7 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ExceptionReporter.java

@@ -17,9 +17,15 @@
  */
 package org.apache.hadoop.mapreduce.task.reduce;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 /**
  * An interface for reporting exceptions to other threads
  */
-interface ExceptionReporter {
+
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
+public interface ExceptionReporter { 
   void reportException(Throwable t);
 }

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapHost.java

@@ -20,9 +20,13 @@ package org.apache.hadoop.mapreduce.task.reduce;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
-class MapHost {
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
+public class MapHost {
   
   public static enum State {
     IDLE,               // No map outputs available

+ 6 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MapOutput.java

@@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+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.LocalDirAllocator;
@@ -33,7 +35,9 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
-class MapOutput<K,V> {
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
+public abstract class MapOutput<K,V> {
   private static final Log LOG = LogFactory.getLog(MapOutput.class);
   private static AtomicInteger ID = new AtomicInteger(0);
   
@@ -62,7 +66,7 @@ class MapOutput<K,V> {
   
   private final boolean primaryMapOutput;
   
-  MapOutput(TaskAttemptID mapId, MergeManager<K,V> merger, long size, 
+  MapOutput(caskAttemptID mapId, MergeManager<K,V> merger, long size, 
             JobConf conf, LocalDirAllocator localDirAllocator,
             int fetcher, boolean primaryMapOutput, MapOutputFile mapOutputFile)
          throws IOException {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java

@@ -59,7 +59,7 @@ import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 
 @SuppressWarnings(value={"unchecked", "deprecation"})
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
 @InterfaceStability.Unstable
 public class MergeManager<K, V> {
   

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Shuffle.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.util.Progress;
 
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
 @InterfaceStability.Unstable
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class Shuffle<K, V> implements ExceptionReporter {

+ 5 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/ShuffleClientMetrics.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.mapreduce.task.reduce;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -25,7 +27,9 @@ import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.Updater;
 
-class ShuffleClientMetrics implements Updater {
+@InterfaceAudience.LimitedPrivate({"MapReduce"})
+@InterfaceStability.Unstable
+public class ShuffleClientMetrics implements Updater {
 
   private MetricsRecord shuffleMetrics = null;
   private int numFailedFetches = 0;