MAPREDUCE-6526. Remove usage of metrics v1 from hadoop-mapreduce. (aajisaka)

This commit is contained in:
Akira Ajisaka 2016-05-03 10:46:11 +09:00
parent 3ff0510ffd
commit 4ee4e5ca2b
4 changed files with 70 additions and 119 deletions

View File

@ -768,7 +768,7 @@ public class LocalJobRunner implements ClientProtocol {
public LocalJobRunner(JobConf conf) throws IOException { public LocalJobRunner(JobConf conf) throws IOException {
this.fs = FileSystem.getLocal(conf); this.fs = FileSystem.getLocal(conf);
this.conf = conf; this.conf = conf;
myMetrics = new LocalJobRunnerMetrics(new JobConf(conf)); myMetrics = LocalJobRunnerMetrics.create();
} }
// JobSubmissionProtocol methods // JobSubmissionProtocol methods

View File

@ -17,82 +17,50 @@
*/ */
package org.apache.hadoop.mapred; package org.apache.hadoop.mapred;
import org.apache.hadoop.metrics.MetricsContext; import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics.MetricsRecord; import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics.MetricsUtil; import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics.Updater; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics.jvm.JvmMetrics; import org.apache.hadoop.metrics2.lib.MutableCounterInt;
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
@SuppressWarnings("deprecation") import java.util.concurrent.ThreadLocalRandom;
class LocalJobRunnerMetrics implements Updater {
private final MetricsRecord metricsRecord;
private int numMapTasksLaunched = 0; @Metrics(name="LocalJobRunnerMetrics", context="mapred")
private int numMapTasksCompleted = 0; final class LocalJobRunnerMetrics {
private int numReduceTasksLaunched = 0;
private int numReduceTasksCompleted = 0; @Metric
private int numWaitingMaps = 0; private MutableCounterInt numMapTasksLaunched;
private int numWaitingReduces = 0; @Metric
private MutableCounterInt numMapTasksCompleted;
public LocalJobRunnerMetrics(JobConf conf) { @Metric
String sessionId = conf.getSessionId(); private MutableCounterInt numReduceTasksLaunched;
// Initiate JVM Metrics @Metric
JvmMetrics.init("JobTracker", sessionId); private MutableGaugeInt numReduceTasksCompleted;
// Create a record for map-reduce metrics
MetricsContext context = MetricsUtil.getContext("mapred"); private LocalJobRunnerMetrics() {
// record name is jobtracker for compatibility
metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
metricsRecord.setTag("sessionId", sessionId);
context.registerUpdater(this);
} }
/**
* Since this object is a registered updater, this method will be called
* periodically, e.g. every 5 seconds.
*/
public void doUpdates(MetricsContext unused) {
synchronized (this) {
metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
numMapTasksLaunched = 0; public static LocalJobRunnerMetrics create() {
numMapTasksCompleted = 0; MetricsSystem ms = DefaultMetricsSystem.initialize("JobTracker");
numReduceTasksLaunched = 0; return ms.register("LocalJobRunnerMetrics-" +
numReduceTasksCompleted = 0; ThreadLocalRandom.current().nextInt(), null,
numWaitingMaps = 0; new LocalJobRunnerMetrics());
numWaitingReduces = 0;
}
metricsRecord.update();
} }
public synchronized void launchMap(TaskAttemptID taskAttemptID) { public synchronized void launchMap(TaskAttemptID taskAttemptID) {
++numMapTasksLaunched; numMapTasksLaunched.incr();
decWaitingMaps(taskAttemptID.getJobID(), 1);
} }
public synchronized void completeMap(TaskAttemptID taskAttemptID) { public void completeMap(TaskAttemptID taskAttemptID) {
++numMapTasksCompleted; numMapTasksCompleted.incr();
} }
public synchronized void launchReduce(TaskAttemptID taskAttemptID) { public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
++numReduceTasksLaunched; numReduceTasksLaunched.incr();
decWaitingReduces(taskAttemptID.getJobID(), 1);
} }
public synchronized void completeReduce(TaskAttemptID taskAttemptID) { public void completeReduce(TaskAttemptID taskAttemptID) {
++numReduceTasksCompleted; numReduceTasksCompleted.incr();
} }
private synchronized void decWaitingMaps(JobID id, int task) {
numWaitingMaps -= task;
}
private synchronized void decWaitingReduces(JobID id, int task){
numWaitingReduces -= task;
}
} }

View File

@ -68,7 +68,7 @@ public class Shuffle<K, V> implements ShuffleConsumerPlugin<K, V>, ExceptionRepo
this.jobConf = context.getJobConf(); this.jobConf = context.getJobConf();
this.umbilical = context.getUmbilical(); this.umbilical = context.getUmbilical();
this.reporter = context.getReporter(); this.reporter = context.getReporter();
this.metrics = new ShuffleClientMetrics(reduceId, jobConf); this.metrics = ShuffleClientMetrics.create();
this.copyPhase = context.getCopyPhase(); this.copyPhase = context.getCopyPhase();
this.taskStatus = context.getStatus(); this.taskStatus = context.getStatus();
this.reduceTask = context.getReduceTask(); this.reduceTask = context.getReduceTask();

View File

@ -20,70 +20,53 @@ package org.apache.hadoop.mapreduce.task.reduce;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics.MetricsContext; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics.MetricsRecord; import org.apache.hadoop.metrics2.lib.MutableCounterInt;
import org.apache.hadoop.metrics.MetricsUtil; import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics.Updater; import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
import java.util.concurrent.ThreadLocalRandom;
@InterfaceAudience.LimitedPrivate({"MapReduce"}) @InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable @InterfaceStability.Unstable
public class ShuffleClientMetrics implements Updater { @Metrics(name="ShuffleClientMetrics", context="mapred")
public class ShuffleClientMetrics {
private MetricsRecord shuffleMetrics = null; @Metric
private int numFailedFetches = 0; private MutableCounterInt numFailedFetches;
private int numSuccessFetches = 0; @Metric
private long numBytes = 0; private MutableCounterInt numSuccessFetches;
private int numThreadsBusy = 0; @Metric
private final int numCopiers; private MutableCounterLong numBytes;
@Metric
ShuffleClientMetrics(TaskAttemptID reduceId, JobConf jobConf) { private MutableGaugeInt numThreadsBusy;
this.numCopiers = jobConf.getInt(MRJobConfig.SHUFFLE_PARALLEL_COPIES, 5);
MetricsContext metricsContext = MetricsUtil.getContext("mapred"); private ShuffleClientMetrics() {
this.shuffleMetrics =
MetricsUtil.createRecord(metricsContext, "shuffleInput");
this.shuffleMetrics.setTag("user", jobConf.getUser());
this.shuffleMetrics.setTag("jobName", jobConf.getJobName());
this.shuffleMetrics.setTag("jobId", reduceId.getJobID().toString());
this.shuffleMetrics.setTag("taskId", reduceId.toString());
this.shuffleMetrics.setTag("sessionId", jobConf.getSessionId());
metricsContext.registerUpdater(this);
} }
public synchronized void inputBytes(long numBytes) {
this.numBytes += numBytes; public static ShuffleClientMetrics create() {
MetricsSystem ms = DefaultMetricsSystem.initialize("JobTracker");
return ms.register("ShuffleClientMetrics-" +
ThreadLocalRandom.current().nextInt(), null,
new ShuffleClientMetrics());
} }
public synchronized void failedFetch() {
++numFailedFetches; public void inputBytes(long bytes) {
numBytes.incr(bytes);
} }
public synchronized void successFetch() { public void failedFetch() {
++numSuccessFetches; numFailedFetches.incr();
} }
public synchronized void threadBusy() { public void successFetch() {
++numThreadsBusy; numSuccessFetches.incr();
} }
public synchronized void threadFree() { public void threadBusy() {
--numThreadsBusy; numThreadsBusy.incr();
} }
public void doUpdates(MetricsContext unused) { public void threadFree() {
synchronized (this) { numThreadsBusy.decr();
shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
shuffleMetrics.incrMetric("shuffle_failed_fetches",
numFailedFetches);
shuffleMetrics.incrMetric("shuffle_success_fetches",
numSuccessFetches);
if (numCopiers != 0) {
shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
100*((float)numThreadsBusy/numCopiers));
} else {
shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
}
numBytes = 0;
numSuccessFetches = 0;
numFailedFetches = 0;
}
shuffleMetrics.update();
} }
} }