/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.hadoop.mapred; import static org.apache.hadoop.mapred.Task.Counter.COMBINE_INPUT_RECORDS; import static org.apache.hadoop.mapred.Task.Counter.COMBINE_OUTPUT_RECORDS; import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_BYTES; import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS; import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES; import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_MATERIALIZED_BYTES; import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_RECORDS; import java.io.DataInput; import java.io.DataOutput; import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; 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.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.mapred.IFile.Writer; import org.apache.hadoop.mapred.Merger.Segment; import org.apache.hadoop.mapred.SortedRanges.SkipRangeIterator; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapreduce.split.JobSplit; import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo; import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex; import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.IndexedSortable; import org.apache.hadoop.util.IndexedSorter; import org.apache.hadoop.util.Progress; import org.apache.hadoop.util.QuickSort; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; /** A Map task. */ class MapTask extends Task { /** * The size of each record in the index file for the map-outputs. */ public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24; private TaskSplitIndex splitMetaInfo = new TaskSplitIndex(); private final static int APPROX_HEADER_LENGTH = 150; private static final Log LOG = LogFactory.getLog(MapTask.class.getName()); { // set phase for this task setPhase(TaskStatus.Phase.MAP); } public MapTask() { super(); } public MapTask(String jobFile, TaskAttemptID taskId, int partition, TaskSplitIndex splitIndex, int numSlotsRequired) { super(jobFile, taskId, partition, numSlotsRequired); this.splitMetaInfo = splitIndex; } @Override public boolean isMapTask() { return true; } @Override public void localizeConfiguration(JobConf conf) throws IOException { super.localizeConfiguration(conf); // split.info file is used only by IsolationRunner. // Write the split file to the local disk if it is a normal map task (not a // job-setup or a job-cleanup task) and if the user wishes to run // IsolationRunner either by setting keep.failed.tasks.files to true or by // using keep.tasks.files.pattern if (supportIsolationRunner(conf) && isMapOrReduce()) { // localize the split meta-information Path localSplitMeta = new LocalDirAllocator("mapred.local.dir").getLocalPathForWrite( TaskTracker.getLocalSplitFile(conf.getUser(), getJobID() .toString(), getTaskID().toString()), conf); LOG.debug("Writing local split to " + localSplitMeta); DataOutputStream out = FileSystem.getLocal(conf).create(localSplitMeta); splitMetaInfo.write(out); out.close(); } } @Override public TaskRunner createRunner(TaskTracker tracker, TaskTracker.TaskInProgress tip, TaskTracker.RunningJob rjob ) throws IOException { return new JobTaskRunner(tip, tracker, this.conf, rjob); } @Override public void write(DataOutput out) throws IOException { super.write(out); if (isMapOrReduce()) { if (splitMetaInfo != null) { splitMetaInfo.write(out); } else { new TaskSplitIndex().write(out); } //TODO do we really need to set this to null? splitMetaInfo = null; } } @Override public void readFields(DataInput in) throws IOException { super.readFields(in); if (isMapOrReduce()) { splitMetaInfo.readFields(in); } } /** * This class wraps the user's record reader to update the counters and * progress as records are read. * @param <K> * @param <V> */ class TrackedRecordReader<K, V> implements RecordReader<K,V> { protected RecordReader<K,V> rawIn; private Counters.Counter inputByteCounter; private Counters.Counter inputRecordCounter; private Counters.Counter fileInputByteCounter; private InputSplit split; private TaskReporter reporter; private long beforePos = -1; private long afterPos = -1; private long bytesInPrev = -1; private long bytesInCurr = -1; private final Statistics fsStats; TrackedRecordReader(InputSplit split, JobConf job, TaskReporter reporter) throws IOException { inputRecordCounter = reporter.getCounter(MAP_INPUT_RECORDS); inputByteCounter = reporter.getCounter(MAP_INPUT_BYTES); fileInputByteCounter = reporter .getCounter(FileInputFormat.Counter.BYTES_READ); Statistics matchedStats = null; if (split instanceof FileSplit) { matchedStats = getFsStatistics(((FileSplit) split).getPath(), job); } fsStats = matchedStats; bytesInPrev = getInputBytes(fsStats); rawIn = job.getInputFormat().getRecordReader(split, job, reporter); bytesInCurr = getInputBytes(fsStats); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); this.reporter = reporter; this.split = split; conf = job; } public K createKey() { return rawIn.createKey(); } public V createValue() { return rawIn.createValue(); } public synchronized boolean next(K key, V value) throws IOException { if (isKilled) { rawIn.close(); LOG.info("TrackedRecordReader killed"); return false; } boolean ret = moveToNext(key, value); if (ret) { incrCounters(); } return ret; } protected void incrCounters() { inputRecordCounter.increment(1); inputByteCounter.increment(afterPos - beforePos); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } protected synchronized boolean moveToNext(K key, V value) throws IOException { boolean ret = false; try { reporter.setProgress(getProgress()); beforePos = getPos(); bytesInPrev = getInputBytes(fsStats); ret = rawIn.next(key, value); afterPos = getPos(); bytesInCurr = getInputBytes(fsStats); } catch (IOException ioe) { if (split instanceof FileSplit) { LOG.error("IO error in map input file " + conf.get("map.input.file")); throw new IOException("IO error in map input file " + conf.get("map.input.file"), ioe); } throw ioe; } return ret; } public long getPos() throws IOException { return rawIn.getPos(); } public void close() throws IOException { bytesInPrev = getInputBytes(fsStats); rawIn.close(); bytesInCurr = getInputBytes(fsStats); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } public float getProgress() throws IOException { return rawIn.getProgress(); } TaskReporter getTaskReporter() { return reporter; } private long getInputBytes(Statistics stats) { return stats == null ? 0 : stats.getBytesRead(); } } /** * This class skips the records based on the failed ranges from previous * attempts. */ class SkippingRecordReader<K, V> extends TrackedRecordReader<K,V> { private SkipRangeIterator skipIt; private SequenceFile.Writer skipWriter; private boolean toWriteSkipRecs; private TaskUmbilicalProtocol umbilical; private Counters.Counter skipRecCounter; private long recIndex = -1; SkippingRecordReader(InputSplit split, TaskUmbilicalProtocol umbilical, TaskReporter reporter) throws IOException { super(split, conf, reporter); this.umbilical = umbilical; this.skipRecCounter = reporter.getCounter(Counter.MAP_SKIPPED_RECORDS); this.toWriteSkipRecs = toWriteSkipRecs() && SkipBadRecords.getSkipOutputPath(conf)!=null; skipIt = getSkipRanges().skipRangeIterator(); } public synchronized boolean next(K key, V value) throws IOException { if (isKilled) { rawIn.close(); LOG.info("SkippingRecordReader killed"); return false; } if(!skipIt.hasNext()) { LOG.warn("Further records got skipped."); return false; } boolean ret = moveToNext(key, value); long nextRecIndex = skipIt.next(); long skip = 0; while(recIndex<nextRecIndex && ret) { if(toWriteSkipRecs) { writeSkippedRec(key, value); } ret = moveToNext(key, value); skip++; } //close the skip writer once all the ranges are skipped if(skip>0 && skipIt.skippedAllRanges() && skipWriter!=null) { skipWriter.close(); } skipRecCounter.increment(skip); reportNextRecordRange(umbilical, recIndex); if (ret) { incrCounters(); } return ret; } protected synchronized boolean moveToNext(K key, V value) throws IOException { recIndex++; return super.moveToNext(key, value); } @SuppressWarnings("unchecked") private void writeSkippedRec(K key, V value) throws IOException{ if(skipWriter==null) { Path skipDir = SkipBadRecords.getSkipOutputPath(conf); Path skipFile = new Path(skipDir, getTaskID().toString()); skipWriter = SequenceFile.createWriter( skipFile.getFileSystem(conf), conf, skipFile, (Class<K>) createKey().getClass(), (Class<V>) createValue().getClass(), CompressionType.BLOCK, getTaskReporter()); } skipWriter.append(key, value); } } @Override public void run(final JobConf job, final TaskUmbilicalProtocol umbilical) throws IOException, ClassNotFoundException, InterruptedException { this.umbilical = umbilical; // start thread that will handle communication with parent TaskReporter reporter = new TaskReporter(getProgress(), umbilical, jvmContext); reporter.startCommunicationThread(); boolean useNewApi = job.getUseNewMapper(); initialize(job, getJobID(), reporter, useNewApi); // check if it is a cleanupJobTask if (jobCleanup) { runJobCleanupTask(umbilical, reporter); return; } if (jobSetup) { runJobSetupTask(umbilical, reporter); return; } if (taskCleanup) { runTaskCleanupTask(umbilical, reporter); return; } if (useNewApi) { runNewMapper(job, splitMetaInfo, umbilical, reporter); } else { runOldMapper(job, splitMetaInfo, umbilical, reporter); } if (isKilled) { reporter.stopCommunicationThread(); return; } done(umbilical, reporter); } @SuppressWarnings("unchecked") private <T> T getSplitDetails(Path file, long offset) throws IOException { FileSystem fs = file.getFileSystem(conf); FSDataInputStream inFile = fs.open(file); inFile.seek(offset); String className = Text.readString(inFile); Class<T> cls; try { cls = (Class<T>) conf.getClassByName(className); } catch (ClassNotFoundException ce) { IOException wrap = new IOException("Split class " + className + " not found"); wrap.initCause(ce); throw wrap; } SerializationFactory factory = new SerializationFactory(conf); Deserializer<T> deserializer = (Deserializer<T>) factory.getDeserializer(cls); deserializer.open(inFile); T split = deserializer.deserialize(null); long pos = inFile.getPos(); getCounters().findCounter( Task.Counter.SPLIT_RAW_BYTES).increment(pos - offset); inFile.close(); return split; } @SuppressWarnings("unchecked") private <INKEY,INVALUE,OUTKEY,OUTVALUE> void runOldMapper(final JobConf job, final TaskSplitIndex splitIndex, final TaskUmbilicalProtocol umbilical, TaskReporter reporter ) throws IOException, InterruptedException, ClassNotFoundException { InputSplit inputSplit = getSplitDetails(new Path(splitIndex.getSplitLocation()), splitIndex.getStartOffset()); updateJobWithSplit(job, inputSplit); reporter.setInputSplit(inputSplit); RecordReader<INKEY,INVALUE> in = isSkipping() ? new SkippingRecordReader<INKEY,INVALUE>(inputSplit, umbilical, reporter) : new TrackedRecordReader<INKEY,INVALUE>(inputSplit, job, reporter); job.setBoolean("mapred.skip.on", isSkipping()); int numReduceTasks = conf.getNumReduceTasks(); LOG.info("numReduceTasks: " + numReduceTasks); MapOutputCollector collector = null; if (numReduceTasks > 0) { collector = new MapOutputBuffer(umbilical, job, reporter); } else { collector = new DirectMapOutputCollector(umbilical, job, reporter); } MapRunnable<INKEY,INVALUE,OUTKEY,OUTVALUE> runner = ReflectionUtils.newInstance(job.getMapRunnerClass(), job); try { runner.run(in, new OldOutputCollector(collector, conf), reporter); collector.flush(); } finally { //close in.close(); // close input collector.close(); } } /** * Update the job with details about the file split * @param job the job configuration to update * @param inputSplit the file split */ private void updateJobWithSplit(final JobConf job, InputSplit inputSplit) { if (inputSplit instanceof FileSplit) { FileSplit fileSplit = (FileSplit) inputSplit; job.set("map.input.file", fileSplit.getPath().toString()); job.setLong("map.input.start", fileSplit.getStart()); job.setLong("map.input.length", fileSplit.getLength()); } } class NewTrackingRecordReader<K,V> extends org.apache.hadoop.mapreduce.RecordReader<K,V> { private final org.apache.hadoop.mapreduce.RecordReader<K,V> real; private final org.apache.hadoop.mapreduce.Counter inputRecordCounter; private final org.apache.hadoop.mapreduce.Counter fileInputByteCounter; private final TaskReporter reporter; private org.apache.hadoop.mapreduce.InputSplit inputSplit; private final JobConf job; private final Statistics fsStats; NewTrackingRecordReader(org.apache.hadoop.mapreduce.InputSplit split, org.apache.hadoop.mapreduce.InputFormat inputFormat, TaskReporter reporter, JobConf job, org.apache.hadoop.mapreduce.TaskAttemptContext taskContext) throws IOException, InterruptedException { this.reporter = reporter; this.inputSplit = split; this.job = job; this.inputRecordCounter = reporter.getCounter(MAP_INPUT_RECORDS); this.fileInputByteCounter = reporter .getCounter(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.Counter.BYTES_READ); Statistics matchedStats = null; if (split instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) { matchedStats = getFsStatistics(((org.apache.hadoop.mapreduce.lib.input.FileSplit) split) .getPath(), job); } fsStats = matchedStats; long bytesInPrev = getInputBytes(fsStats); this.real = inputFormat.createRecordReader(split, taskContext); long bytesInCurr = getInputBytes(fsStats); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } @Override public void close() throws IOException { long bytesInPrev = getInputBytes(fsStats); real.close(); long bytesInCurr = getInputBytes(fsStats); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } @Override public K getCurrentKey() throws IOException, InterruptedException { return real.getCurrentKey(); } @Override public V getCurrentValue() throws IOException, InterruptedException { return real.getCurrentValue(); } @Override public float getProgress() throws IOException, InterruptedException { return real.getProgress(); } @Override public void initialize(org.apache.hadoop.mapreduce.InputSplit split, org.apache.hadoop.mapreduce.TaskAttemptContext context ) throws IOException, InterruptedException { long bytesInPrev = getInputBytes(fsStats); real.initialize(split, context); long bytesInCurr = getInputBytes(fsStats); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (isKilled) { real.close(); LOG.info("NewTrackingRecordReader<K, V> nextKeyValue() killed"); return false; } boolean result = false; try { long bytesInPrev = getInputBytes(fsStats); result = real.nextKeyValue(); long bytesInCurr = getInputBytes(fsStats); if (result) { inputRecordCounter.increment(1); fileInputByteCounter.increment(bytesInCurr - bytesInPrev); } reporter.setProgress(getProgress()); } catch (IOException ioe) { if (inputSplit instanceof FileSplit) { FileSplit fileSplit = (FileSplit) inputSplit; LOG.error("IO error in map input file " + fileSplit.getPath().toString()); throw new IOException("IO error in map input file " + fileSplit.getPath().toString(), ioe); } throw ioe; } return result; } private long getInputBytes(Statistics stats) { return stats == null ? 0 : stats.getBytesRead(); } } /** * Since the mapred and mapreduce Partitioners don't share a common interface * (JobConfigurable is deprecated and a subtype of mapred.Partitioner), the * partitioner lives in Old/NewOutputCollector. Note that, for map-only jobs, * the configured partitioner should not be called. It's common for * partitioners to compute a result mod numReduces, which causes a div0 error */ private static class OldOutputCollector<K,V> implements OutputCollector<K,V> { private final Partitioner<K,V> partitioner; private final MapOutputCollector<K,V> collector; private final int numPartitions; @SuppressWarnings("unchecked") OldOutputCollector(MapOutputCollector<K,V> collector, JobConf conf) { numPartitions = conf.getNumReduceTasks(); if (numPartitions > 0) { partitioner = (Partitioner<K,V>) ReflectionUtils.newInstance(conf.getPartitionerClass(), conf); } else { partitioner = new Partitioner<K,V>() { @Override public void configure(JobConf job) { } @Override public int getPartition(K key, V value, int numPartitions) { return -1; } }; } this.collector = collector; } @Override public void collect(K key, V value) throws IOException { try { collector.collect(key, value, partitioner.getPartition(key, value, numPartitions)); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new IOException("interrupt exception", ie); } } } private class NewDirectOutputCollector<K,V> extends org.apache.hadoop.mapreduce.RecordWriter<K,V> { private final org.apache.hadoop.mapreduce.RecordWriter out; private final TaskReporter reporter; private final Counters.Counter mapOutputRecordCounter; private final Counters.Counter fileOutputByteCounter; private final Statistics fsStats; @SuppressWarnings("unchecked") NewDirectOutputCollector(org.apache.hadoop.mapreduce.JobContext jobContext, JobConf job, TaskUmbilicalProtocol umbilical, TaskReporter reporter) throws IOException, ClassNotFoundException, InterruptedException { this.reporter = reporter; Statistics matchedStats = null; if (outputFormat instanceof org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) { matchedStats = getFsStatistics(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat .getOutputPath(jobContext), job); } fsStats = matchedStats; mapOutputRecordCounter = reporter.getCounter(MAP_OUTPUT_RECORDS); fileOutputByteCounter = reporter .getCounter(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.Counter.BYTES_WRITTEN); long bytesOutPrev = getOutputBytes(fsStats); out = outputFormat.getRecordWriter(taskContext); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); } @Override @SuppressWarnings("unchecked") public void write(K key, V value) throws IOException, InterruptedException { reporter.progress(); long bytesOutPrev = getOutputBytes(fsStats); out.write(key, value); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); mapOutputRecordCounter.increment(1); } @Override public void close(TaskAttemptContext context) throws IOException,InterruptedException { reporter.progress(); if (out != null) { long bytesOutPrev = getOutputBytes(fsStats); out.close(context); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); } } private long getOutputBytes(Statistics stats) { return stats == null ? 0 : stats.getBytesWritten(); } } private class NewOutputCollector<K,V> extends org.apache.hadoop.mapreduce.RecordWriter<K,V> { private final MapOutputCollector<K,V> collector; private final org.apache.hadoop.mapreduce.Partitioner<K,V> partitioner; private final int partitions; @SuppressWarnings("unchecked") NewOutputCollector(org.apache.hadoop.mapreduce.JobContext jobContext, JobConf job, TaskUmbilicalProtocol umbilical, TaskReporter reporter ) throws IOException, ClassNotFoundException, InterruptedException { collector = new MapOutputBuffer<K,V>(umbilical, job, reporter); partitions = jobContext.getNumReduceTasks(); if (partitions > 0) { partitioner = (org.apache.hadoop.mapreduce.Partitioner<K,V>) ReflectionUtils.newInstance(jobContext.getPartitionerClass(), job); } else { partitioner = new org.apache.hadoop.mapreduce.Partitioner<K,V>() { @Override public int getPartition(K key, V value, int numPartitions) { return -1; } }; } } @Override public void write(K key, V value) throws IOException, InterruptedException { collector.collect(key, value, partitioner.getPartition(key, value, partitions)); } @Override public void close(TaskAttemptContext context ) throws IOException,InterruptedException { try { collector.flush(); } catch (ClassNotFoundException cnf) { throw new IOException("can't find class ", cnf); } collector.close(); } } @SuppressWarnings("unchecked") private <INKEY,INVALUE,OUTKEY,OUTVALUE> void runNewMapper(final JobConf job, final TaskSplitIndex splitIndex, final TaskUmbilicalProtocol umbilical, TaskReporter reporter ) throws IOException, ClassNotFoundException, InterruptedException { // make a task context so we can get the classes org.apache.hadoop.mapreduce.TaskAttemptContext taskContext = new org.apache.hadoop.mapreduce.TaskAttemptContext(job, getTaskID()); // make a mapper org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE> mapper = (org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>) ReflectionUtils.newInstance(taskContext.getMapperClass(), job); // make the input format org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE> inputFormat = (org.apache.hadoop.mapreduce.InputFormat<INKEY,INVALUE>) ReflectionUtils.newInstance(taskContext.getInputFormatClass(), job); // rebuild the input split org.apache.hadoop.mapreduce.InputSplit split = null; split = getSplitDetails(new Path(splitIndex.getSplitLocation()), splitIndex.getStartOffset()); org.apache.hadoop.mapreduce.RecordReader<INKEY,INVALUE> input = new NewTrackingRecordReader<INKEY,INVALUE> (split, inputFormat, reporter, job, taskContext); job.setBoolean("mapred.skip.on", isSkipping()); org.apache.hadoop.mapreduce.RecordWriter output = null; org.apache.hadoop.mapreduce.Mapper<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context mapperContext = null; try { Constructor<org.apache.hadoop.mapreduce.Mapper.Context> contextConstructor = org.apache.hadoop.mapreduce.Mapper.Context.class.getConstructor (new Class[]{org.apache.hadoop.mapreduce.Mapper.class, Configuration.class, org.apache.hadoop.mapreduce.TaskAttemptID.class, org.apache.hadoop.mapreduce.RecordReader.class, org.apache.hadoop.mapreduce.RecordWriter.class, org.apache.hadoop.mapreduce.OutputCommitter.class, org.apache.hadoop.mapreduce.StatusReporter.class, org.apache.hadoop.mapreduce.InputSplit.class}); // get an output object if (job.getNumReduceTasks() == 0) { output = new NewDirectOutputCollector(taskContext, job, umbilical, reporter); } else { output = new NewOutputCollector(taskContext, job, umbilical, reporter); } mapperContext = contextConstructor.newInstance(mapper, job, getTaskID(), input, output, committer, reporter, split); input.initialize(split, mapperContext); mapper.run(mapperContext); input.close(); output.close(mapperContext); } catch (NoSuchMethodException e) { throw new IOException("Can't find Context constructor", e); } catch (InstantiationException e) { throw new IOException("Can't create Context", e); } catch (InvocationTargetException e) { throw new IOException("Can't invoke Context constructor", e); } catch (IllegalAccessException e) { throw new IOException("Can't invoke Context constructor", e); } } interface MapOutputCollector<K, V> { public void collect(K key, V value, int partition ) throws IOException, InterruptedException; public void close() throws IOException, InterruptedException; public void flush() throws IOException, InterruptedException, ClassNotFoundException; } class DirectMapOutputCollector<K, V> implements MapOutputCollector<K, V> { private RecordWriter<K, V> out = null; private TaskReporter reporter = null; private final Counters.Counter mapOutputRecordCounter; private final Counters.Counter fileOutputByteCounter; private final Statistics fsStats; @SuppressWarnings("unchecked") public DirectMapOutputCollector(TaskUmbilicalProtocol umbilical, JobConf job, TaskReporter reporter) throws IOException { this.reporter = reporter; String finalName = getOutputName(getPartition()); FileSystem fs = FileSystem.get(job); OutputFormat<K, V> outputFormat = job.getOutputFormat(); Statistics matchedStats = null; if (outputFormat instanceof FileOutputFormat) { matchedStats = getFsStatistics(FileOutputFormat.getOutputPath(job), job); } fsStats = matchedStats; mapOutputRecordCounter = reporter.getCounter(MAP_OUTPUT_RECORDS); fileOutputByteCounter = reporter .getCounter(FileOutputFormat.Counter.BYTES_WRITTEN); long bytesOutPrev = getOutputBytes(fsStats); out = job.getOutputFormat().getRecordWriter(fs, job, finalName, reporter); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); } public void close() throws IOException { if (this.out != null) { long bytesOutPrev = getOutputBytes(fsStats); out.close(this.reporter); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); } } public void flush() throws IOException, InterruptedException, ClassNotFoundException { } public void collect(K key, V value, int partition) throws IOException { reporter.progress(); long bytesOutPrev = getOutputBytes(fsStats); out.write(key, value); long bytesOutCurr = getOutputBytes(fsStats); fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev); mapOutputRecordCounter.increment(1); } private long getOutputBytes(Statistics stats) { return stats == null ? 0 : stats.getBytesWritten(); } } class MapOutputBuffer<K extends Object, V extends Object> implements MapOutputCollector<K, V> { //------------------------------ MemoryElement memElement = null; //MemoryElement bigMemElement = null; //------------------------------ private final TaskReporter reporter; private final Class<K> keyClass; private final Class<V> valClass; private final JobConf jobConf; private final Counters.Counter mapOutputRecordCounter; @SuppressWarnings("unchecked") public MapOutputBuffer(TaskUmbilicalProtocol umbilical, JobConf job, TaskReporter reporter ) throws IOException, ClassNotFoundException, InterruptedException { this.reporter = reporter; this.jobConf = job; keyClass = (Class<K>)job.getMapOutputKeyClass(); valClass = (Class<V>)job.getMapOutputValueClass(); mapOutputRecordCounter = reporter.getCounter(MAP_OUTPUT_RECORDS); this.getMemoryElement(false); jvmMemManager.registerSpiller(job, getTaskID(), reporter); } private void getMemoryElement(boolean big) throws ClassNotFoundException, IOException, InterruptedException { if (big) { this.memElement = jvmMemManager.getBigMemoryElement(getTaskID()); this.memElement.initialize(jobConf, reporter, getTaskID()); } else { this.memElement = jvmMemManager.getRegMemoryElement(); this.memElement.initialize(jobConf, reporter, getTaskID()); } } public synchronized void collect(K key, V value, int partition ) throws IOException, InterruptedException { if (isKilled) { jvmMemManager.returnMemElement(memElement, true); memElement = null; } reporter.progress(); if (key.getClass() != keyClass) { throw new IOException("Type mismatch in key from map: expected " + keyClass.getName() + ", recieved " + key.getClass().getName()); } if (value.getClass() != valClass) { throw new IOException("Type mismatch in value from map: expected " + valClass.getName() + ", recieved " + value.getClass().getName()); } try { this.memElement.collect(key, value, partition); } catch (MemoryElement.MemoryElementFullException e) { // LOG.info("MemoryElementFullException!!"); jvmMemManager.returnMemElement(memElement, false); try { this.getMemoryElement(false); } catch (ClassNotFoundException e1) { // TODO Auto-generated catch block e1.printStackTrace(); } collect(key, value, partition); } catch(MemoryElement.MapBufferTooSmallException e) { if (!memElement.isBig()) { jvmMemManager.returnMemElement(memElement, true); } else { LOG.info("Record too large for big memory element buffer: " + e.getMessage()); jvmMemManager.spillSingleRecord(getTaskID(), key, value, partition); mapOutputRecordCounter.increment(1); return; } try { this.getMemoryElement(true); } catch (ClassNotFoundException e1) { // TODO Auto-generated catch block e1.printStackTrace(); } if (memElement != null) { collect(key, value, partition); } else { LOG.info("Record too large for regular memory element buffer: " + e.getMessage()); jvmMemManager.spillSingleRecord(getTaskID(), key, value, partition); mapOutputRecordCounter.increment(1); try { this.getMemoryElement(false); } catch (ClassNotFoundException e1) { // TODO Auto-generated catch block e1.printStackTrace(); } } } } public synchronized void flush() throws IOException, ClassNotFoundException, InterruptedException { LOG.info("Starting flush of map output"); try { jvmMemManager.returnMemElement(memElement, memElement.getSize() == 0); if (isKilled) { return; } jvmMemManager.flush(getTaskID()); } catch (InterruptedException e) { throw (IOException)new IOException( "Buffer interrupted while waiting for the writer" ).initCause(e); } } public void close() { } } }