Selaa lähdekoodia

Clean up javadoc, reduce number of public classes.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@375685 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 vuotta sitten
vanhempi
commit
a1e62cdf65

+ 1 - 1
src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java

@@ -22,7 +22,7 @@ import java.io.*;
  * Protocol that a TaskTracker and the central JobTracker use to communicate.
  * The JobTracker is the Server, which implements this protocol.
  */ 
-public interface InterTrackerProtocol {
+interface InterTrackerProtocol {
   public final static int TRACKERS_OK = 0;
   public final static int UNKNOWN_TASKTRACKER = 1;
 

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -29,7 +29,7 @@ import java.util.logging.*;
 // and its latest JobStatus, plus a set of tables for 
 // doing bookkeeping of its Tasks.
 ///////////////////////////////////////////////////////
-public class JobInProgress {
+class JobInProgress {
     public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobInProgress");
 
     JobProfile profile;

+ 10 - 1
src/java/org/apache/hadoop/mapred/JobProfile.java

@@ -26,7 +26,16 @@ import java.net.*;
  *
  * @author Mike Cafarella
  **************************************************/
-public class JobProfile implements Writable {
+class JobProfile implements Writable {
+
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (JobProfile.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new JobProfile(); }
+         });
+    }
+
     String jobid;
     String jobFile;
     String url;

+ 10 - 1
src/java/org/apache/hadoop/mapred/JobStatus.java

@@ -26,7 +26,16 @@ import java.io.*;
  *
  * @author Mike Cafarella
  **************************************************/
-public class JobStatus implements Writable {
+class JobStatus implements Writable {
+
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (JobStatus.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new JobStatus(); }
+         });
+    }
+
     public static final int RUNNING = 1;
     public static final int SUCCEEDED = 2;
     public static final int FAILED = 3;

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java

@@ -24,7 +24,7 @@ import java.util.*;
  * JobClient can use these methods to submit a Job for execution, and learn about
  * the current system status.
  */ 
-public interface JobSubmissionProtocol {
+interface JobSubmissionProtocol {
     /**
      * Submit a Job for execution.  Returns the latest profile for
      * that job.

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java

@@ -29,7 +29,7 @@ import java.net.*;
  *
  * @author Mike Cafarella
  *******************************************************/
-public class JobTrackerInfoServer {
+class JobTrackerInfoServer {
 
     public static class RedirectHandler extends AbstractHttpHandler {
         public void handle(String pathInContext, String pathParams, HttpRequest request, HttpResponse response) throws HttpException, IOException {

+ 1 - 1
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.LogFormatter;
 
 /** Implements MapReduce locally, in-process, for debugging. */ 
-public class LocalJobRunner implements JobSubmissionProtocol {
+class LocalJobRunner implements JobSubmissionProtocol {
   public static final Logger LOG =
     LogFormatter.getLogger("org.apache.hadoop.mapred.LocalJobRunner");
 

+ 1 - 1
src/java/org/apache/hadoop/mapred/MRConstants.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.mapred;
  * 
  * @author Mike Cafarella
  *******************************/
-public interface MRConstants {
+interface MRConstants {
     //
     // Timeouts, constants
     //

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.conf.Configuration;
 
+/** An {@link OutputFormat} that writes {@link MapFile}s. */
 public class MapFileOutputFormat implements OutputFormat {
 
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,

+ 10 - 1
src/java/org/apache/hadoop/mapred/MapOutputFile.java

@@ -24,7 +24,16 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 
 /** A local file to be transferred via the {@link MapOutputProtocol}. */ 
-public class MapOutputFile implements Writable, Configurable {
+class MapOutputFile implements Writable, Configurable {
+
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (MapOutputFile.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new MapOutputFile(); }
+         });
+    }
+
   private String mapTaskId;
   private String reduceTaskId;
   private int partition;

+ 10 - 1
src/java/org/apache/hadoop/mapred/MapOutputLocation.java

@@ -23,7 +23,16 @@ import org.apache.hadoop.io.*;
 
 /** The location of a map output file, as passed to a reduce task via the
  * {@link InterTrackerProtocol}. */ 
-public class MapOutputLocation implements Writable {
+class MapOutputLocation implements Writable {
+
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (MapOutputLocation.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new MapOutputLocation(); }
+         });
+    }
+
   private String mapTaskId;
   private String host;
   private int port;

+ 1 - 1
src/java/org/apache/hadoop/mapred/MapOutputProtocol.java

@@ -22,7 +22,7 @@ import org.apache.hadoop.io.*;
 
 /** Protocol that a reduce task uses to retrieve output data from a map task's
  * tracker. */ 
-public interface MapOutputProtocol {
+interface MapOutputProtocol {
 
   /** Returns the output from the named map task destined for this partition.*/
   MapOutputFile getFile(String mapTaskId, String reduceTaskId,

+ 10 - 1
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -23,7 +23,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
 /** A Map task. */
-public class MapTask extends Task {
+class MapTask extends Task {
+
+  static {                                        // register a ctor
+    WritableFactories.setFactory
+      (MapTask.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new MapTask(); }
+       });
+  }
+
   private FileSplit split;
   private MapOutputFile mapOutputFile;
   private Configuration conf;

+ 10 - 1
src/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -26,7 +26,16 @@ import java.util.*;
 import java.text.*;
 
 /** A Reduce task. */
-public class ReduceTask extends Task {
+class ReduceTask extends Task {
+
+  static {                                        // register a ctor
+    WritableFactories.setFactory
+      (ReduceTask.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new ReduceTask(); }
+       });
+  }
+
   private String[][] mapTaskIds;
   private int partition;
   private boolean sortComplete;

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

@@ -27,6 +27,7 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.conf.Configuration;
 
+/** An {@link OutputFormat} that writes {@link SequenceFile}s. */
 public class SequenceFileOutputFormat implements OutputFormat {
 
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,

+ 1 - 1
src/java/org/apache/hadoop/mapred/Task.java

@@ -23,7 +23,7 @@ import org.apache.hadoop.util.*;
 import java.io.*;
 
 /** Base class for tasks. */
-public abstract class Task implements Writable, Configurable {
+abstract class Task implements Writable, Configurable {
   ////////////////////////////////////////////
   // Fields
   ////////////////////////////////////////////

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskStatus.java

@@ -25,7 +25,7 @@ import java.io.*;
  *
  * @author Mike Cafarella
  **************************************************/
-public class TaskStatus implements Writable {
+class TaskStatus implements Writable {
     public static final int RUNNING = 0;
     public static final int SUCCEEDED = 1;
     public static final int FAILED = 2;

+ 10 - 1
src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java

@@ -28,7 +28,16 @@ import java.util.*;
  *
  * @author Mike Cafarella
  **************************************************/
-public class TaskTrackerStatus implements Writable {
+class TaskTrackerStatus implements Writable {
+
+    static {                                        // register a ctor
+      WritableFactories.setFactory
+        (TaskTrackerStatus.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new TaskTrackerStatus(); }
+         });
+    }
+
     String trackerName;
     String host;
     int port;

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java

@@ -22,7 +22,7 @@ import java.io.IOException;
  * parent is a daemon which which polls the central master for a new map or
  * reduce task and runs it as a child process.  All communication between child
  * and parent is via this protocol. */ 
-public interface TaskUmbilicalProtocol {
+interface TaskUmbilicalProtocol {
 
   /** Called when a child task process starts, to get its task.*/
   Task getTask(String taskid) throws IOException;

+ 1 - 0
src/java/org/apache/hadoop/mapred/TextOutputFormat.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
 
+/** An {@link OutputFormat} that writes plain text files. */
 public class TextOutputFormat implements OutputFormat {
 
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,