|
@@ -141,7 +141,7 @@ class MapTask extends Task {
|
|
|
private TaskReporter reporter;
|
|
|
private long bytesInPrev = -1;
|
|
|
private long bytesInCurr = -1;
|
|
|
- private final Statistics fsStats;
|
|
|
+ private final List<Statistics> fsStats;
|
|
|
|
|
|
TrackedRecordReader(TaskReporter reporter, JobConf job)
|
|
|
throws IOException{
|
|
@@ -149,7 +149,7 @@ class MapTask extends Task {
|
|
|
fileInputByteCounter = reporter.getCounter(FileInputFormatCounter.BYTES_READ);
|
|
|
this.reporter = reporter;
|
|
|
|
|
|
- Statistics matchedStats = null;
|
|
|
+ List<Statistics> matchedStats = null;
|
|
|
if (this.reporter.getInputSplit() instanceof FileSplit) {
|
|
|
matchedStats = getFsStatistics(((FileSplit) this.reporter
|
|
|
.getInputSplit()).getPath(), job);
|
|
@@ -210,8 +210,13 @@ class MapTask extends Task {
|
|
|
return reporter;
|
|
|
}
|
|
|
|
|
|
- private long getInputBytes(Statistics stats) {
|
|
|
- return stats == null ? 0 : stats.getBytesRead();
|
|
|
+ private long getInputBytes(List<Statistics> stats) {
|
|
|
+ if (stats == null) return 0;
|
|
|
+ long bytesRead = 0;
|
|
|
+ for (Statistics stat: stats) {
|
|
|
+ bytesRead = bytesRead + stat.getBytesRead();
|
|
|
+ }
|
|
|
+ return bytesRead;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -426,7 +431,7 @@ class MapTask extends Task {
|
|
|
private final org.apache.hadoop.mapreduce.Counter inputRecordCounter;
|
|
|
private final org.apache.hadoop.mapreduce.Counter fileInputByteCounter;
|
|
|
private final TaskReporter reporter;
|
|
|
- private final Statistics fsStats;
|
|
|
+ private final List<Statistics> fsStats;
|
|
|
|
|
|
NewTrackingRecordReader(org.apache.hadoop.mapreduce.InputSplit split,
|
|
|
org.apache.hadoop.mapreduce.InputFormat<K, V> inputFormat,
|
|
@@ -439,7 +444,7 @@ class MapTask extends Task {
|
|
|
this.fileInputByteCounter = reporter
|
|
|
.getCounter(FileInputFormatCounter.BYTES_READ);
|
|
|
|
|
|
- Statistics matchedStats = null;
|
|
|
+ List <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());
|
|
@@ -498,8 +503,13 @@ class MapTask extends Task {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
- private long getInputBytes(Statistics stats) {
|
|
|
- return stats == null ? 0 : stats.getBytesRead();
|
|
|
+ private long getInputBytes(List<Statistics> stats) {
|
|
|
+ if (stats == null) return 0;
|
|
|
+ long bytesRead = 0;
|
|
|
+ for (Statistics stat: stats) {
|
|
|
+ bytesRead = bytesRead + stat.getBytesRead();
|
|
|
+ }
|
|
|
+ return bytesRead;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -554,7 +564,7 @@ class MapTask extends Task {
|
|
|
|
|
|
private final Counters.Counter mapOutputRecordCounter;
|
|
|
private final Counters.Counter fileOutputByteCounter;
|
|
|
- private final Statistics fsStats;
|
|
|
+ private final List<Statistics> fsStats;
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
NewDirectOutputCollector(MRJobConfig jobContext,
|
|
@@ -566,7 +576,7 @@ class MapTask extends Task {
|
|
|
fileOutputByteCounter = reporter
|
|
|
.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
|
|
|
|
|
|
- Statistics matchedStats = null;
|
|
|
+ List<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());
|
|
@@ -603,8 +613,13 @@ class MapTask extends Task {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private long getOutputBytes(Statistics stats) {
|
|
|
- return stats == null ? 0 : stats.getBytesWritten();
|
|
|
+ private long getOutputBytes(List<Statistics> stats) {
|
|
|
+ if (stats == null) return 0;
|
|
|
+ long bytesWritten = 0;
|
|
|
+ for (Statistics stat: stats) {
|
|
|
+ bytesWritten = bytesWritten + stat.getBytesWritten();
|
|
|
+ }
|
|
|
+ return bytesWritten;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -735,7 +750,7 @@ class MapTask extends Task {
|
|
|
|
|
|
private final Counters.Counter mapOutputRecordCounter;
|
|
|
private final Counters.Counter fileOutputByteCounter;
|
|
|
- private final Statistics fsStats;
|
|
|
+ private final List<Statistics> fsStats;
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public DirectMapOutputCollector(TaskUmbilicalProtocol umbilical,
|
|
@@ -750,7 +765,7 @@ class MapTask extends Task {
|
|
|
fileOutputByteCounter = reporter
|
|
|
.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
|
|
|
|
|
|
- Statistics matchedStats = null;
|
|
|
+ List<Statistics> matchedStats = null;
|
|
|
if (outputFormat instanceof FileOutputFormat) {
|
|
|
matchedStats = getFsStatistics(FileOutputFormat.getOutputPath(job), job);
|
|
|
}
|
|
@@ -785,8 +800,13 @@ class MapTask extends Task {
|
|
|
mapOutputRecordCounter.increment(1);
|
|
|
}
|
|
|
|
|
|
- private long getOutputBytes(Statistics stats) {
|
|
|
- return stats == null ? 0 : stats.getBytesWritten();
|
|
|
+ private long getOutputBytes(List<Statistics> stats) {
|
|
|
+ if (stats == null) return 0;
|
|
|
+ long bytesWritten = 0;
|
|
|
+ for (Statistics stat: stats) {
|
|
|
+ bytesWritten = bytesWritten + stat.getBytesWritten();
|
|
|
+ }
|
|
|
+ return bytesWritten;
|
|
|
}
|
|
|
}
|
|
|
|