|
@@ -23,6 +23,7 @@ import java.io.IOException;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.mapred.TaskAttemptID;
|
|
@@ -41,14 +42,19 @@ import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer;
|
|
|
* Java Record Reader + Java Mapper + Native Collector
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked")
|
|
|
+@InterfaceAudience.Private
|
|
|
public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Closeable {
|
|
|
|
|
|
- public static String NAME = "NativeTask.MCollectorOutputHandler";
|
|
|
+ public static final String NAME = "NativeTask.MCollectorOutputHandler";
|
|
|
private static Log LOG = LogFactory.getLog(NativeCollectorOnlyHandler.class);
|
|
|
- public static Command GET_OUTPUT_PATH = new Command(100, "GET_OUTPUT_PATH");
|
|
|
- public static Command GET_OUTPUT_INDEX_PATH = new Command(101, "GET_OUTPUT_INDEX_PATH");
|
|
|
- public static Command GET_SPILL_PATH = new Command(102, "GET_SPILL_PATH");
|
|
|
- public static Command GET_COMBINE_HANDLER = new Command(103, "GET_COMBINE_HANDLER");
|
|
|
+ public static final Command GET_OUTPUT_PATH =
|
|
|
+ new Command(100, "GET_OUTPUT_PATH");
|
|
|
+ public static final Command GET_OUTPUT_INDEX_PATH =
|
|
|
+ new Command(101, "GET_OUTPUT_INDEX_PATH");
|
|
|
+ public static final Command GET_SPILL_PATH =
|
|
|
+ new Command(102, "GET_SPILL_PATH");
|
|
|
+ public static final Command GET_COMBINE_HANDLER =
|
|
|
+ new Command(103, "GET_COMBINE_HANDLER");
|
|
|
|
|
|
private NativeTaskOutput output;
|
|
|
private int spillNumber = 0;
|
|
@@ -63,7 +69,7 @@ public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Clos
|
|
|
ICombineHandler combinerHandler = null;
|
|
|
try {
|
|
|
final TaskContext combineContext = context.copyOf();
|
|
|
- combineContext.setInputKeyClass(context.getOuputKeyClass());
|
|
|
+ combineContext.setInputKeyClass(context.getOutputKeyClass());
|
|
|
combineContext.setInputValueClass(context.getOutputValueClass());
|
|
|
|
|
|
combinerHandler = CombinerHandler.create(combineContext);
|
|
@@ -76,7 +82,9 @@ public class NativeCollectorOnlyHandler<K, V> implements CommandDispatcher, Clos
|
|
|
}
|
|
|
|
|
|
final INativeHandler nativeHandler = NativeBatchProcessor.create(NAME, context.getConf(), DataChannel.OUT);
|
|
|
- final BufferPusher<K, V> kvPusher = new BufferPusher<K, V>(context.getOuputKeyClass(), context.getOutputValueClass(),
|
|
|
+ final BufferPusher<K, V> kvPusher = new BufferPusher<K, V>(
|
|
|
+ (Class<K>)context.getOutputKeyClass(),
|
|
|
+ (Class<V>)context.getOutputValueClass(),
|
|
|
nativeHandler);
|
|
|
|
|
|
return new NativeCollectorOnlyHandler<K, V>(context, nativeHandler, kvPusher, combinerHandler);
|