MAPREDUCE-2365. Add counters to track bytes (read,written) via File(Input,Output)Format. Contributed by Siddharth Seth.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1146515 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2011-07-13 23:36:02 +00:00
parent 8327e70be8
commit 4796e1adcb
14 changed files with 403 additions and 91 deletions

View File

@ -40,6 +40,9 @@ Trunk (unreleased changes)
IMPROVEMENTS
MAPREDUCE-2365. Add counters to track bytes (read,written) via
File(Input,Output)Format. (Siddharth Seth via acmurthy)
MAPREDUCE-2680. Display queue name in job client CLI. (acmurthy)
MAPREDUCE-2679. Minor changes to sync trunk with MR-279 branch. (acmurthy)

View File

@ -38,7 +38,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.util.StringUtils;
/**
@ -457,11 +457,10 @@ public synchronized Counter findCounter(Enum key) {
*/
public synchronized Counter findCounter(String group, String name) {
if (name.equals("MAP_INPUT_BYTES")) {
group = FileInputFormat.COUNTER_GROUP;
name = FileInputFormat.BYTES_READ;
LOG.warn("Counter name MAP_INPUT_BYTES is deprecated. " +
"Use FileInputFormatCounters as group name and " +
" BYTES_READ as counter name instead");
return findCounter(FileInputFormatCounter.BYTES_READ);
}
return getGroup(group).getCounterForName(name);
}

View File

@ -35,7 +35,7 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputBuffer;
@ -51,12 +51,12 @@
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.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
import org.apache.hadoop.util.IndexedSortable;
@ -141,20 +141,31 @@ public void readFields(DataInput in) throws IOException {
class TrackedRecordReader<K, V>
implements RecordReader<K,V> {
private RecordReader<K,V> rawIn;
private Counters.Counter inputByteCounter;
private Counters.Counter fileInputByteCounter;
private Counters.Counter inputRecordCounter;
private TaskReporter reporter;
private long beforePos = -1;
private long afterPos = -1;
private long bytesInPrev = -1;
private long bytesInCurr = -1;
private final Statistics fsStats;
TrackedRecordReader(RecordReader<K,V> raw, TaskReporter reporter)
TrackedRecordReader(TaskReporter reporter, JobConf job)
throws IOException{
rawIn = raw;
inputRecordCounter = reporter.getCounter(TaskCounter.MAP_INPUT_RECORDS);
inputByteCounter = reporter.getCounter(
FileInputFormat.COUNTER_GROUP,
FileInputFormat.BYTES_READ);
fileInputByteCounter = reporter.getCounter(FileInputFormatCounter.BYTES_READ);
this.reporter = reporter;
Statistics matchedStats = null;
if (this.reporter.getInputSplit() instanceof FileSplit) {
matchedStats = getFsStatistics(((FileSplit) this.reporter
.getInputSplit()).getPath(), job);
}
fsStats = matchedStats;
bytesInPrev = getInputBytes(fsStats);
rawIn = job.getInputFormat().getRecordReader(reporter.getInputSplit(),
job, reporter);
bytesInCurr = getInputBytes(fsStats);
fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
}
public K createKey() {
@ -176,26 +187,37 @@ public synchronized boolean next(K key, V value)
protected void incrCounters() {
inputRecordCounter.increment(1);
inputByteCounter.increment(afterPos - beforePos);
}
protected synchronized boolean moveToNext(K key, V value)
throws IOException {
beforePos = getPos();
bytesInPrev = getInputBytes(fsStats);
boolean ret = rawIn.next(key, value);
afterPos = getPos();
bytesInCurr = getInputBytes(fsStats);
fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
reporter.setProgress(getProgress());
return ret;
}
public long getPos() throws IOException { return rawIn.getPos(); }
public void close() throws IOException { rawIn.close(); }
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();
}
}
/**
@ -210,9 +232,9 @@ class SkippingRecordReader<K, V> extends TrackedRecordReader<K,V> {
private Counters.Counter skipRecCounter;
private long recIndex = -1;
SkippingRecordReader(RecordReader<K,V> raw, TaskUmbilicalProtocol umbilical,
TaskReporter reporter) throws IOException{
super(raw, reporter);
SkippingRecordReader(TaskUmbilicalProtocol umbilical,
TaskReporter reporter, JobConf job) throws IOException{
super(reporter, job);
this.umbilical = umbilical;
this.skipRecCounter = reporter.getCounter(TaskCounter.MAP_SKIPPED_RECORDS);
this.toWriteSkipRecs = toWriteSkipRecs() &&
@ -356,11 +378,9 @@ void runOldMapper(final JobConf job,
updateJobWithSplit(job, inputSplit);
reporter.setInputSplit(inputSplit);
RecordReader<INKEY,INVALUE> rawIn = // open input
job.getInputFormat().getRecordReader(inputSplit, job, reporter);
RecordReader<INKEY,INVALUE> in = isSkipping() ?
new SkippingRecordReader<INKEY,INVALUE>(rawIn, umbilical, reporter) :
new TrackedRecordReader<INKEY,INVALUE>(rawIn, reporter);
new SkippingRecordReader<INKEY,INVALUE>(umbilical, reporter, job) :
new TrackedRecordReader<INKEY,INVALUE>(reporter, job);
job.setBoolean(JobContext.SKIP_RECORDS, isSkipping());
@ -409,18 +429,40 @@ static 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 final Statistics fsStats;
NewTrackingRecordReader(org.apache.hadoop.mapreduce.RecordReader<K,V> real,
TaskReporter reporter) {
this.real = real;
NewTrackingRecordReader(org.apache.hadoop.mapreduce.InputSplit split,
org.apache.hadoop.mapreduce.InputFormat<K, V> inputFormat,
TaskReporter reporter,
org.apache.hadoop.mapreduce.TaskAttemptContext taskContext)
throws InterruptedException, IOException {
this.reporter = reporter;
this.inputRecordCounter = reporter.getCounter(TaskCounter.MAP_INPUT_RECORDS);
this.inputRecordCounter = reporter
.getCounter(TaskCounter.MAP_INPUT_RECORDS);
this.fileInputByteCounter = reporter
.getCounter(FileInputFormatCounter.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(), taskContext.getConfiguration());
}
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
@ -442,18 +484,28 @@ public float getProgress() throws IOException, InterruptedException {
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 {
long bytesInPrev = getInputBytes(fsStats);
boolean result = real.nextKeyValue();
long bytesInCurr = getInputBytes(fsStats);
if (result) {
inputRecordCounter.increment(1);
}
fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
reporter.setProgress(getProgress());
return result;
}
private long getInputBytes(Statistics stats) {
return stats == null ? 0 : stats.getBytesRead();
}
}
/**
@ -506,15 +558,30 @@ private class NewDirectOutputCollector<K,V>
private final TaskReporter reporter;
private final Counters.Counter mapOutputRecordCounter;
private final Counters.Counter fileOutputByteCounter;
private final Statistics fsStats;
@SuppressWarnings("unchecked")
NewDirectOutputCollector(MRJobConfig jobContext,
JobConf job, TaskUmbilicalProtocol umbilical, TaskReporter reporter)
throws IOException, ClassNotFoundException, InterruptedException {
this.reporter = reporter;
mapOutputRecordCounter = reporter
.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
fileOutputByteCounter = reporter
.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
Statistics matchedStats = null;
if (outputFormat instanceof org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
matchedStats = getFsStatistics(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
.getOutputPath(taskContext), taskContext.getConfiguration());
}
fsStats = matchedStats;
long bytesOutPrev = getOutputBytes(fsStats);
out = outputFormat.getRecordWriter(taskContext);
mapOutputRecordCounter =
reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
}
@Override
@ -522,7 +589,10 @@ private class NewDirectOutputCollector<K,V>
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);
}
@ -531,9 +601,16 @@ 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>
@ -609,7 +686,7 @@ void runNewMapper(final JobConf job,
org.apache.hadoop.mapreduce.RecordReader<INKEY,INVALUE> input =
new NewTrackingRecordReader<INKEY,INVALUE>
(inputFormat.createRecordReader(split, taskContext), reporter);
(split, inputFormat, reporter, taskContext);
job.setBoolean(JobContext.SKIP_RECORDS, isSkipping());
org.apache.hadoop.mapreduce.RecordWriter output = null;
@ -662,6 +739,8 @@ class DirectMapOutputCollector<K, V>
private TaskReporter reporter = null;
private final Counters.Counter mapOutputRecordCounter;
private final Counters.Counter fileOutputByteCounter;
private final Statistics fsStats;
@SuppressWarnings("unchecked")
public DirectMapOutputCollector(TaskUmbilicalProtocol umbilical,
@ -670,14 +749,30 @@ public DirectMapOutputCollector(TaskUmbilicalProtocol umbilical,
String finalName = getOutputName(getPartition());
FileSystem fs = FileSystem.get(job);
out = job.getOutputFormat().getRecordWriter(fs, job, finalName, reporter);
OutputFormat<K, V> outputFormat = job.getOutputFormat();
mapOutputRecordCounter = reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
fileOutputByteCounter = reporter
.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
Statistics matchedStats = null;
if (outputFormat instanceof FileOutputFormat) {
matchedStats = getFsStatistics(FileOutputFormat.getOutputPath(job), job);
}
fsStats = matchedStats;
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);
}
}
@ -688,10 +783,16 @@ public void flush() throws IOException, InterruptedException,
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();
}
}
private class MapOutputBuffer<K extends Object, V extends Object>
@ -757,6 +858,7 @@ private class MapOutputBuffer<K extends Object, V extends Object>
// Counters
final Counters.Counter mapOutputByteCounter;
final Counters.Counter mapOutputRecordCounter;
final Counters.Counter fileOutputByteCounter;
final ArrayList<SpillRecord> indexCacheList =
new ArrayList<SpillRecord>();
@ -823,6 +925,8 @@ public MapOutputBuffer(TaskUmbilicalProtocol umbilical, JobConf job,
mapOutputByteCounter = reporter.getCounter(TaskCounter.MAP_OUTPUT_BYTES);
mapOutputRecordCounter =
reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
fileOutputByteCounter = reporter
.getCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
// compression
if (job.getCompressMapOutput()) {
@ -1317,6 +1421,8 @@ public void flush() throws IOException, ClassNotFoundException,
// release sort buffer before the merge
kvbuffer = null;
mergeParts();
Path outputPath = mapOutputFile.getOutputFile();
fileOutputByteCounter.increment(rfs.getFileStatus(outputPath).getLen());
}
public void close() { }

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.RawComparator;
@ -48,6 +49,7 @@
import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
import org.apache.hadoop.mapreduce.task.reduce.Shuffle;
import org.apache.hadoop.util.Progress;
@ -95,6 +97,8 @@ public class ReduceTask extends Task {
getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
private Counters.Counter reduceCombineOutputCounter =
getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
private Counters.Counter fileOutputByteCounter =
getCounters().findCounter(FileOutputFormatCounter.BYTES_WRITTEN);
// A custom comparator for map output files. Here the ordering is determined
// by the file's size and path. In case of files with same size and different
@ -407,17 +411,14 @@ void runOldReducer(JobConf job,
// make output collector
String finalName = getOutputName(getPartition());
FileSystem fs = FileSystem.get(job);
final RecordWriter<OUTKEY, OUTVALUE> out = new OldTrackingRecordWriter<OUTKEY, OUTVALUE>(
this, job, reporter, finalName);
final RecordWriter<OUTKEY,OUTVALUE> out =
job.getOutputFormat().getRecordWriter(fs, job, finalName, reporter);
OutputCollector<OUTKEY,OUTVALUE> collector =
new OutputCollector<OUTKEY,OUTVALUE>() {
public void collect(OUTKEY key, OUTVALUE value)
throws IOException {
out.write(key, value);
reduceOutputCounter.increment(1);
// indicate that progress update needs to be sent
reporter.progress();
}
@ -465,28 +466,104 @@ public void collect(OUTKEY key, OUTVALUE value)
}
}
static class OldTrackingRecordWriter<K, V> implements RecordWriter<K, V> {
private final RecordWriter<K, V> real;
private final org.apache.hadoop.mapred.Counters.Counter reduceOutputCounter;
private final org.apache.hadoop.mapred.Counters.Counter fileOutputByteCounter;
private final Statistics fsStats;
@SuppressWarnings({ "deprecation", "unchecked" })
public OldTrackingRecordWriter(ReduceTask reduce, JobConf job,
TaskReporter reporter, String finalName) throws IOException {
this.reduceOutputCounter = reduce.reduceOutputCounter;
this.fileOutputByteCounter = reduce.fileOutputByteCounter;
Statistics matchedStats = null;
if (job.getOutputFormat() instanceof FileOutputFormat) {
matchedStats = getFsStatistics(FileOutputFormat.getOutputPath(job), job);
}
fsStats = matchedStats;
FileSystem fs = FileSystem.get(job);
long bytesOutPrev = getOutputBytes(fsStats);
this.real = job.getOutputFormat().getRecordWriter(fs, job, finalName,
reporter);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
}
@Override
public void write(K key, V value) throws IOException {
long bytesOutPrev = getOutputBytes(fsStats);
real.write(key, value);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
reduceOutputCounter.increment(1);
}
@Override
public void close(Reporter reporter) throws IOException {
long bytesOutPrev = getOutputBytes(fsStats);
real.close(reporter);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
}
private long getOutputBytes(Statistics stats) {
return stats == null ? 0 : stats.getBytesWritten();
}
}
static class NewTrackingRecordWriter<K,V>
extends org.apache.hadoop.mapreduce.RecordWriter<K,V> {
private final org.apache.hadoop.mapreduce.RecordWriter<K,V> real;
private final org.apache.hadoop.mapreduce.Counter outputRecordCounter;
NewTrackingRecordWriter(org.apache.hadoop.mapreduce.RecordWriter<K,V> real,
org.apache.hadoop.mapreduce.Counter recordCounter) {
this.real = real;
this.outputRecordCounter = recordCounter;
private final org.apache.hadoop.mapreduce.Counter fileOutputByteCounter;
private final Statistics fsStats;
@SuppressWarnings("unchecked")
NewTrackingRecordWriter(ReduceTask reduce,
org.apache.hadoop.mapreduce.TaskAttemptContext taskContext)
throws InterruptedException, IOException {
this.outputRecordCounter = reduce.reduceOutputCounter;
this.fileOutputByteCounter = reduce.fileOutputByteCounter;
Statistics matchedStats = null;
if (reduce.outputFormat instanceof org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
matchedStats = getFsStatistics(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
.getOutputPath(taskContext), taskContext.getConfiguration());
}
fsStats = matchedStats;
long bytesOutPrev = getOutputBytes(fsStats);
this.real = (org.apache.hadoop.mapreduce.RecordWriter<K, V>) reduce.outputFormat
.getRecordWriter(taskContext);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
}
@Override
public void close(TaskAttemptContext context) throws IOException,
InterruptedException {
long bytesOutPrev = getOutputBytes(fsStats);
real.close(context);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
}
@Override
public void write(K key, V value) throws IOException, InterruptedException {
long bytesOutPrev = getOutputBytes(fsStats);
real.write(key,value);
long bytesOutCurr = getOutputBytes(fsStats);
fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
outputRecordCounter.increment(1);
}
private long getOutputBytes(Statistics stats) {
return stats == null ? 0 : stats.getBytesWritten();
}
}
@SuppressWarnings("unchecked")
@ -529,11 +606,8 @@ public boolean next() throws IOException {
org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE> reducer =
(org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>)
ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output =
(org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE>)
outputFormat.getRecordWriter(taskContext);
org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> trackedRW =
new NewTrackingRecordWriter<OUTKEY, OUTVALUE>(output, reduceOutputCounter);
new NewTrackingRecordWriter<OUTKEY, OUTVALUE>(this, taskContext);
job.setBoolean("mapred.skip.on", isSkipping());
job.setBoolean(JobContext.SKIP_RECORDS, isSkipping());
org.apache.hadoop.mapreduce.Reducer.Context
@ -545,6 +619,6 @@ public boolean next() throws IOException {
reporter, comparator, keyClass,
valueClass);
reducer.run(reducerContext);
output.close(reducerContext);
trackedRW.close(reducerContext);
}
}

View File

@ -289,6 +289,28 @@ protected void reportFatalError(TaskAttemptID id, Throwable throwable,
}
}
/**
* Gets a handle to the Statistics instance based on the scheme associated
* with path.
*
* @param path the path.
* @param conf the configuration to extract the scheme from if not part of
* the path.
* @return a Statistics instance, or null if none is found for the scheme.
*/
protected static Statistics getFsStatistics(Path path, Configuration conf) throws IOException {
Statistics matchedStats = null;
path = path.getFileSystem(conf).makeQualified(path);
String scheme = path.toUri().getScheme();
for (Statistics stats : FileSystem.getAllStatistics()) {
if (stats.getScheme().equals(scheme)) {
matchedStats = stats;
break;
}
}
return matchedStats;
}
/**
* Get skipRanges.
*/

View File

@ -29,6 +29,7 @@ public enum TaskCounter {
MAP_OUTPUT_RECORDS,
MAP_SKIPPED_RECORDS,
MAP_OUTPUT_BYTES,
MAP_OUTPUT_MATERIALIZED_BYTES,
SPLIT_RAW_BYTES,
COMBINE_INPUT_RECORDS,
COMBINE_OUTPUT_RECORDS,

View File

@ -17,6 +17,7 @@ CounterGroupName= Map-Reduce Framework
MAP_INPUT_RECORDS.name= Map input records
MAP_OUTPUT_RECORDS.name= Map output records
MAP_OUTPUT_BYTES.name= Map output bytes
MAP_OUTPUT_MATERIALIZED_BYTES.name= Map output materialized bytes
MAP_SKIPPED_RECORDS.name= Map skipped records
COMBINE_INPUT_RECORDS.name= Combine input records
COMBINE_OUTPUT_RECORDS.name= Combine output records

View File

@ -54,9 +54,6 @@
@InterfaceAudience.Public
@InterfaceStability.Stable
public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
public static final String COUNTER_GROUP =
"FileInputFormatCounters";
public static final String BYTES_READ = "BYTES_READ";
public static final String INPUT_DIR =
"mapreduce.input.fileinputformat.inputdir";
public static final String SPLIT_MAXSIZE =

View File

@ -35,7 +35,6 @@
import org.apache.hadoop.io.compress.SplittableCompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
@ -63,7 +62,6 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
private int maxLineLength;
private LongWritable key = null;
private Text value = null;
private Counter inputByteCounter;
private CompressionCodec codec;
private Decompressor decompressor;
private byte[] recordDelimiterBytes;
@ -78,8 +76,6 @@ public LineRecordReader(byte[] recordDelimiter) {
public void initialize(InputSplit genericSplit,
TaskAttemptContext context) throws IOException {
FileSplit split = (FileSplit) genericSplit;
inputByteCounter = context.getCounter(
FileInputFormat.COUNTER_GROUP, FileInputFormat.BYTES_READ);
Configuration job = context.getConfiguration();
this.maxLineLength = job.getInt(MAX_LINE_LENGTH, Integer.MAX_VALUE);
start = split.getStart();
@ -174,7 +170,6 @@ public boolean nextKeyValue() throws IOException {
break;
}
pos += newSize;
inputByteCounter.increment(newSize);
if (newSize < maxLineLength) {
break;
}

View File

@ -27,9 +27,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.*;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.MapContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
@ -44,16 +42,12 @@ public class SequenceFileRecordReader<K, V> extends RecordReader<K, V> {
private K key = null;
private V value = null;
protected Configuration conf;
private Counter inputByteCounter;
private long pos;
@Override
public void initialize(InputSplit split,
TaskAttemptContext context
) throws IOException, InterruptedException {
FileSplit fileSplit = (FileSplit) split;
inputByteCounter = ((MapContext)context).getCounter(
FileInputFormat.COUNTER_GROUP, FileInputFormat.BYTES_READ);
conf = context.getConfiguration();
Path path = fileSplit.getPath();
FileSystem fs = path.getFileSystem(conf);
@ -74,8 +68,7 @@ public boolean nextKeyValue() throws IOException, InterruptedException {
if (!more) {
return false;
}
inputByteCounter.increment(in.getPosition()-pos);
pos = in.getPosition();
long pos = in.getPosition();
key = (K) in.next(key);
if (key == null || (pos >= end && in.syncSeen())) {
more = false;

View File

@ -150,7 +150,9 @@ public void checkOutputSpecs(JobContext job
* @param outputDir the {@link Path} of the output directory for
* the map-reduce job.
*/
public static void setOutputPath(Job job, Path outputDir) {
public static void setOutputPath(Job job, Path outputDir) throws IOException {
outputDir = outputDir.getFileSystem(job.getConfiguration()).makeQualified(
outputDir);
job.getConfiguration().set(FileOutputFormat.OUTDIR, outputDir.toString());
}

View File

@ -42,9 +42,10 @@
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
/**
* This is an wordcount application that tests the count of records
@ -58,6 +59,26 @@
*/
public class TestJobCounters {
private void validateFileCounters(Counters counter, long fileBytesRead,
long fileBytesWritten, long mapOutputBytes,
long mapOutputMaterializedBytes) {
assertTrue(counter.findCounter(FileInputFormatCounter.BYTES_READ)
.getValue() != 0);
assertEquals(fileBytesRead,
counter.findCounter(FileInputFormatCounter.BYTES_READ).getValue());
assertTrue(counter.findCounter(FileOutputFormatCounter.BYTES_WRITTEN)
.getValue() != 0);
if (mapOutputBytes >= 0) {
assertTrue(counter.findCounter(TaskCounter.MAP_OUTPUT_BYTES).getValue() != 0);
}
if (mapOutputMaterializedBytes >= 0) {
assertTrue(counter.findCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES)
.getValue() != 0);
}
}
private void validateCounters(Counters counter, long spillRecCnt,
long mapInputRecords, long mapOutputRecords) {
// Check if the numer of Spilled Records is same as expected
@ -108,6 +129,19 @@ private static void createWordsFile(Path inpFile, Configuration conf)
private static Path OUT_DIR = null;
private static Path testdir = null;
private static Path[] inFiles = new Path[5];
private static long getFileSize(Path path) throws IOException {
FileSystem fs = FileSystem.getLocal(new Configuration());
long len = 0;
len += fs.getFileStatus(path).getLen();
Path crcPath = new Path(path.getParent(), "." + path.getName() + ".crc");
if (fs.exists(crcPath)) {
len += fs.getFileStatus(crcPath).getLen();
}
return len;
}
@BeforeClass
public static void initPaths() throws IOException {
final Configuration conf = new Configuration();
@ -125,11 +159,15 @@ public static void initPaths() throws IOException {
if (!fs.mkdirs(IN_DIR)) {
throw new IOException("Mkdirs failed to create " + IN_DIR);
}
// create 3 input files each with 5*2k words
createWordsFile(new Path(IN_DIR, "input5_2k_1"), conf);
createWordsFile(new Path(IN_DIR, "input5_2k_2"), conf);
createWordsFile(new Path(IN_DIR, "input5_2k_3"), conf);
for (int i = 0; i < inFiles.length; i++) {
inFiles[i] = new Path(IN_DIR, "input5_2k_" + i);
}
// create 3 input files each with 5*2k words
createWordsFile(inFiles[0], conf);
createWordsFile(inFiles[1], conf);
createWordsFile(inFiles[2], conf);
}
@AfterClass
@ -181,8 +219,12 @@ public void testOldCounterA() throws Exception {
JobConf conf = createConfiguration();
conf.setNumMapTasks(3);
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
removeWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
removeWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
removeWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
FileInputFormat.setInputPaths(conf, IN_DIR);
FileOutputFormat.setOutputPath(conf, new Path(OUT_DIR, "outputO0"));
@ -211,6 +253,7 @@ public void testOldCounterA() throws Exception {
// 3 files, 5120 = 5 * 1024 rec/file = 15360 input records
// 4 records/line = 61440 output records
validateCounters(c1, 90112, 15360, 61440);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@ -218,8 +261,13 @@ public void testOldCounterA() throws Exception {
public void testOldCounterB() throws Exception {
JobConf conf = createConfiguration();
createWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
removeWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
createWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
inputSize += getFileSize(inFiles[3]);
conf.setNumMapTasks(4);
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
FileInputFormat.setInputPaths(conf, IN_DIR);
@ -239,13 +287,20 @@ public void testOldCounterB() throws Exception {
// 4 files, 5120 = 5 * 1024 rec/file = 15360 input records
// 4 records/line = 81920 output records
validateCounters(c1, 131072, 20480, 81920);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@Test
public void testOldCounterC() throws Exception {
JobConf conf = createConfiguration();
createWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
createWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
createWordsFile(inFiles[3], conf);
createWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
inputSize += getFileSize(inFiles[3]);
inputSize += getFileSize(inFiles[4]);
conf.setNumMapTasks(4);
conf.setInt(JobContext.IO_SORT_FACTOR, 3);
FileInputFormat.setInputPaths(conf, IN_DIR);
@ -260,6 +315,31 @@ public void testOldCounterC() throws Exception {
// 5 files, 5120 = 5 * 1024 rec/file = 15360 input records
// 4 records/line = 102400 output records
validateCounters(c1, 147456, 25600, 102400);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@Test
public void testOldCounterD() throws Exception {
JobConf conf = createConfiguration();
conf.setNumMapTasks(3);
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
conf.setNumReduceTasks(0);
removeWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
FileInputFormat.setInputPaths(conf, IN_DIR);
FileOutputFormat.setOutputPath(conf, new Path(OUT_DIR, "outputO3"));
RunningJob myJob = JobClient.runJob(conf);
Counters c1 = myJob.getCounters();
// No Reduces. Will go through the direct output collector. Spills=0
validateCounters(c1, 0, 15360, 61440);
validateFileCounters(c1, inputSize, 0, -1, -1);
}
@Test
@ -267,8 +347,12 @@ public void testNewCounterA() throws Exception {
final Job job = createJob();
final Configuration conf = job.getConfiguration();
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
removeWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
removeWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
removeWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(
job, IN_DIR);
org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath(
@ -276,6 +360,7 @@ public void testNewCounterA() throws Exception {
assertTrue(job.waitForCompletion(true));
final Counters c1 = Counters.downgrade(job.getCounters());
validateCounters(c1, 90112, 15360, 61440);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@Test
@ -283,8 +368,13 @@ public void testNewCounterB() throws Exception {
final Job job = createJob();
final Configuration conf = job.getConfiguration();
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
createWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
removeWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
createWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
inputSize += getFileSize(inFiles[3]);
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(
job, IN_DIR);
org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath(
@ -292,6 +382,7 @@ public void testNewCounterB() throws Exception {
assertTrue(job.waitForCompletion(true));
final Counters c1 = Counters.downgrade(job.getCounters());
validateCounters(c1, 131072, 20480, 81920);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@Test
@ -299,8 +390,14 @@ public void testNewCounterC() throws Exception {
final Job job = createJob();
final Configuration conf = job.getConfiguration();
conf.setInt(JobContext.IO_SORT_FACTOR, 3);
createWordsFile(new Path(IN_DIR, "input5_2k_4"), conf);
createWordsFile(new Path(IN_DIR, "input5_2k_5"), conf);
createWordsFile(inFiles[3], conf);
createWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
inputSize += getFileSize(inFiles[3]);
inputSize += getFileSize(inFiles[4]);
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(
job, IN_DIR);
org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath(
@ -308,6 +405,29 @@ public void testNewCounterC() throws Exception {
assertTrue(job.waitForCompletion(true));
final Counters c1 = Counters.downgrade(job.getCounters());
validateCounters(c1, 147456, 25600, 102400);
validateFileCounters(c1, inputSize, 0, 0, 0);
}
@Test
public void testNewCounterD() throws Exception {
final Job job = createJob();
final Configuration conf = job.getConfiguration();
conf.setInt(JobContext.IO_SORT_FACTOR, 2);
job.setNumReduceTasks(0);
removeWordsFile(inFiles[3], conf);
removeWordsFile(inFiles[4], conf);
long inputSize = 0;
inputSize += getFileSize(inFiles[0]);
inputSize += getFileSize(inFiles[1]);
inputSize += getFileSize(inFiles[2]);
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job,
IN_DIR);
org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.setOutputPath(job,
new Path(OUT_DIR, "outputN3"));
assertTrue(job.waitForCompletion(true));
final Counters c1 = Counters.downgrade(job.getCounters());
validateCounters(c1, 0, 15360, 61440);
validateFileCounters(c1, inputSize, 0, -1, -1);
}
/**

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.mapred.lib.IdentityMapper;
import org.apache.hadoop.mapred.lib.IdentityReducer;
import org.apache.hadoop.mapred.lib.NullOutputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.ToolRunner;
@ -101,10 +102,8 @@ private static void runSort(JobConf job, Path sortInput, Path sortOutput)
Sort sort = new Sort();
assertEquals(ToolRunner.run(job, sort, sortArgs), 0);
org.apache.hadoop.mapreduce.Counters counters = sort.getResult().getCounters();
long mapInput = counters.findCounter(
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.COUNTER_GROUP,
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.BYTES_READ).
getValue();
long mapInput = counters.findCounter(FileInputFormatCounter.BYTES_READ)
.getValue();
long hdfsRead = counters.findCounter(Task.FILESYSTEM_COUNTER_GROUP,
"HDFS_BYTES_READ").getValue();
// the hdfs read should be between 100% and 110% of the map input bytes

View File

@ -38,6 +38,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
@ -147,8 +148,7 @@ private void runWordCount(Configuration conf
out);
Counters ctrs = job.getCounters();
System.out.println("Counters: " + ctrs);
long mapIn = ctrs.findCounter(FileInputFormat.COUNTER_GROUP,
FileInputFormat.BYTES_READ).getValue();
long mapIn = ctrs.findCounter(FileInputFormatCounter.BYTES_READ).getValue();
assertTrue(mapIn != 0);
long combineIn = ctrs.findCounter(COUNTER_GROUP,
"COMBINE_INPUT_RECORDS").getValue();