|
@@ -47,6 +47,11 @@ import org.apache.hadoop.mapreduce.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
|
|
|
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
|
|
|
import org.apache.hadoop.mapreduce.split.JobSplit;
|
|
|
+import org.apache.hadoop.metrics.ContextFactory;
|
|
|
+import org.apache.hadoop.metrics.MetricsContext;
|
|
|
+import org.apache.hadoop.metrics.MetricsUtil;
|
|
|
+import org.apache.hadoop.metrics.spi.NoEmitMetricsContext;
|
|
|
+import org.apache.hadoop.metrics.spi.OutputRecord;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
import org.mortbay.log.Log;
|
|
|
|
|
@@ -516,7 +521,10 @@ public class TestFairScheduler extends TestCase {
|
|
|
}
|
|
|
|
|
|
private void setUpCluster(int numRacks, int numNodesPerRack,
|
|
|
- boolean assignMultiple) {
|
|
|
+ boolean assignMultiple) throws IOException {
|
|
|
+
|
|
|
+ resetMetrics();
|
|
|
+
|
|
|
conf = new JobConf();
|
|
|
conf.set("mapred.fairscheduler.allocation.file", ALLOC_FILE);
|
|
|
conf.set("mapred.fairscheduler.poolnameproperty", POOL_PROPERTY);
|
|
@@ -534,6 +542,20 @@ public class TestFairScheduler extends TestCase {
|
|
|
scheduler.start();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Set up a metrics context that doesn't emit anywhere but stores the data
|
|
|
+ * so we can verify it. Also clears it of any data so that different test
|
|
|
+ * cases don't pollute each other.
|
|
|
+ */
|
|
|
+ private void resetMetrics() throws IOException {
|
|
|
+ ContextFactory factory = ContextFactory.getFactory();
|
|
|
+ factory.setAttribute("fairscheduler.class",
|
|
|
+ NoEmitMetricsContext.class.getName());
|
|
|
+
|
|
|
+ MetricsUtil.getContext("fairscheduler").createRecord("jobs").remove();
|
|
|
+ MetricsUtil.getContext("fairscheduler").createRecord("pools").remove();
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected void tearDown() throws Exception {
|
|
|
if (scheduler != null) {
|
|
@@ -689,7 +711,8 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(1, info1.reduceSchedulable.getDemand());
|
|
|
assertEquals(2.0, info1.mapSchedulable.getFairShare());
|
|
|
assertEquals(1.0, info1.reduceSchedulable.getFairShare());
|
|
|
-
|
|
|
+ verifyMetrics();
|
|
|
+
|
|
|
// Advance time before submitting another job j2, to make j1 run before j2
|
|
|
// deterministically.
|
|
|
advanceTime(100);
|
|
@@ -709,6 +732,7 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(2, info2.reduceSchedulable.getDemand());
|
|
|
assertEquals(1.0, info2.mapSchedulable.getFairShare());
|
|
|
assertEquals(2.0, info2.reduceSchedulable.getFairShare());
|
|
|
+ verifyMetrics();
|
|
|
|
|
|
// Assign tasks and check that jobs alternate in filling slots
|
|
|
checkAssignment("tt1", "attempt_test_0001_m_000000_0 on tt1");
|
|
@@ -729,8 +753,8 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(2, info2.reduceSchedulable.getRunningTasks());
|
|
|
assertEquals(1, info2.mapSchedulable.getDemand());
|
|
|
assertEquals(2, info2.reduceSchedulable.getDemand());
|
|
|
+ verifyMetrics();
|
|
|
}
|
|
|
-
|
|
|
/**
|
|
|
* This test is identical to testSmallJobs but sets assignMultiple to
|
|
|
* true so that multiple tasks can be assigned per heartbeat.
|
|
@@ -748,6 +772,7 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(1, info1.reduceSchedulable.getDemand());
|
|
|
assertEquals(2.0, info1.mapSchedulable.getFairShare());
|
|
|
assertEquals(1.0, info1.reduceSchedulable.getFairShare());
|
|
|
+ verifyMetrics();
|
|
|
|
|
|
// Advance time before submitting another job j2, to make j1 run before j2
|
|
|
// deterministically.
|
|
@@ -768,6 +793,7 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(2, info2.reduceSchedulable.getDemand());
|
|
|
assertEquals(1.0, info2.mapSchedulable.getFairShare());
|
|
|
assertEquals(2.0, info2.reduceSchedulable.getFairShare());
|
|
|
+ verifyMetrics();
|
|
|
|
|
|
// Assign tasks and check that jobs alternate in filling slots
|
|
|
checkAssignment("tt1", "attempt_test_0001_m_000000_0 on tt1",
|
|
@@ -788,6 +814,7 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(2, info2.reduceSchedulable.getRunningTasks());
|
|
|
assertEquals(1, info2.mapSchedulable.getDemand());
|
|
|
assertEquals(2, info2.reduceSchedulable.getDemand());
|
|
|
+ verifyMetrics();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1632,6 +1659,7 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(0.28, info3.reduceSchedulable.getFairShare(), 0.01);
|
|
|
assertEquals(0.28, info4.mapSchedulable.getFairShare(), 0.01);
|
|
|
assertEquals(0.28, info4.reduceSchedulable.getFairShare(), 0.01);
|
|
|
+ verifyMetrics();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -2730,6 +2758,65 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertEquals(33, poolA.getMapSchedulable().getDemand());
|
|
|
assertEquals(39, poolA.getReduceSchedulable().getDemand());
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test switching a job from one pool to another, then back to the original
|
|
|
+ * one. This is a regression test for a bug seen during development of
|
|
|
+ * MAPREDUCE-2323 (fair scheduler metrics).
|
|
|
+ */
|
|
|
+ public void testSetPoolTwice() throws Exception {
|
|
|
+ // Set up pools file
|
|
|
+ PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
+ out.println("<?xml version=\"1.0\"?>");
|
|
|
+ out.println("<allocations>");
|
|
|
+ out.println("<pool name=\"default\">");
|
|
|
+ out.println("<schedulingMode>fair</schedulingMode>");
|
|
|
+ out.println("</pool>");
|
|
|
+ out.println("<pool name=\"poolA\">");
|
|
|
+ out.println("<schedulingMode>fair</schedulingMode>");
|
|
|
+ out.println("</pool>");
|
|
|
+ out.println("</allocations>");
|
|
|
+ out.close();
|
|
|
+ scheduler.getPoolManager().reloadAllocs();
|
|
|
+ Pool defaultPool = scheduler.getPoolManager().getPool("default");
|
|
|
+ Pool poolA = scheduler.getPoolManager().getPool("poolA");
|
|
|
+
|
|
|
+ // Submit a job to the default pool. All specifications take default values.
|
|
|
+ JobInProgress job1 = submitJob(JobStatus.RUNNING, 1, 3);
|
|
|
+ assertEquals(1, defaultPool.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(3, defaultPool.getReduceSchedulable().getDemand());
|
|
|
+ assertEquals(0, poolA.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(0, poolA.getReduceSchedulable().getDemand());
|
|
|
+
|
|
|
+ // Move job to poolA and make sure demand moves with it
|
|
|
+ scheduler.getPoolManager().setPool(job1, "poolA");
|
|
|
+ assertEquals("poolA", scheduler.getPoolManager().getPoolName(job1));
|
|
|
+
|
|
|
+ defaultPool.getMapSchedulable().updateDemand();
|
|
|
+ defaultPool.getReduceSchedulable().updateDemand();
|
|
|
+ poolA.getMapSchedulable().updateDemand();
|
|
|
+ poolA.getReduceSchedulable().updateDemand();
|
|
|
+
|
|
|
+ assertEquals(0, defaultPool.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(0, defaultPool.getReduceSchedulable().getDemand());
|
|
|
+ assertEquals(1, poolA.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(3, poolA.getReduceSchedulable().getDemand());
|
|
|
+
|
|
|
+ // Move back to default pool and make sure demand goes back
|
|
|
+ scheduler.getPoolManager().setPool(job1, "default");
|
|
|
+ assertEquals("default", scheduler.getPoolManager().getPoolName(job1));
|
|
|
+
|
|
|
+ defaultPool.getMapSchedulable().updateDemand();
|
|
|
+ defaultPool.getReduceSchedulable().updateDemand();
|
|
|
+ poolA.getMapSchedulable().updateDemand();
|
|
|
+ poolA.getReduceSchedulable().updateDemand();
|
|
|
+
|
|
|
+ assertEquals(1, defaultPool.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(3, defaultPool.getReduceSchedulable().getDemand());
|
|
|
+ assertEquals(0, poolA.getMapSchedulable().getDemand());
|
|
|
+ assertEquals(0, poolA.getReduceSchedulable().getDemand());
|
|
|
+ }
|
|
|
|
|
|
private void advanceTime(long time) {
|
|
|
clock.advance(time);
|
|
@@ -2828,4 +2915,108 @@ public class TestFairScheduler extends TestCase {
|
|
|
assertNull(scheduler.assignTasks(tracker("tt1")));
|
|
|
assertNull(scheduler.assignTasks(tracker("tt2")));
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Ask scheduler to update metrics and then verify that they're all
|
|
|
+ * correctly published to the metrics context
|
|
|
+ */
|
|
|
+ private void verifyMetrics() {
|
|
|
+ scheduler.updateMetrics();
|
|
|
+ verifyPoolMetrics();
|
|
|
+ verifyJobMetrics();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify that pool-level metrics match internal data
|
|
|
+ */
|
|
|
+ private void verifyPoolMetrics() {
|
|
|
+ MetricsContext ctx = MetricsUtil.getContext("fairscheduler");
|
|
|
+ Collection<OutputRecord> records = ctx.getAllRecords().get("pools");
|
|
|
+
|
|
|
+ try {
|
|
|
+ assertEquals(scheduler.getPoolSchedulables(TaskType.MAP).size() * 2,
|
|
|
+ records.size());
|
|
|
+ } catch (Error e) {
|
|
|
+ for (OutputRecord rec : records) {
|
|
|
+ System.err.println("record:");
|
|
|
+ System.err.println(" name: " + rec.getTag("name"));
|
|
|
+ System.err.println(" type: " + rec.getTag("type"));
|
|
|
+ }
|
|
|
+
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+
|
|
|
+ Map<String, OutputRecord> byPoolAndType =
|
|
|
+ new HashMap<String, OutputRecord>();
|
|
|
+ for (OutputRecord rec : records) {
|
|
|
+ String pool = (String)rec.getTag("name");
|
|
|
+ String type = (String)rec.getTag("taskType");
|
|
|
+ assertNotNull(pool);
|
|
|
+ assertNotNull(type);
|
|
|
+ byPoolAndType.put(pool + "_" + type, rec);
|
|
|
+ }
|
|
|
+
|
|
|
+ List<PoolSchedulable> poolScheds = new ArrayList<PoolSchedulable>();
|
|
|
+ poolScheds.addAll(scheduler.getPoolSchedulables(TaskType.MAP));
|
|
|
+ poolScheds.addAll(scheduler.getPoolSchedulables(TaskType.REDUCE));
|
|
|
+
|
|
|
+ for (PoolSchedulable pool : poolScheds) {
|
|
|
+ String poolName = pool.getName();
|
|
|
+ OutputRecord metrics = byPoolAndType.get(
|
|
|
+ poolName + "_" + pool.getTaskType().toString());
|
|
|
+ assertNotNull("Need metrics for " + pool, metrics);
|
|
|
+
|
|
|
+ verifySchedulableMetrics(pool, metrics);
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify that the job-level metrics match internal data
|
|
|
+ */
|
|
|
+ private void verifyJobMetrics() {
|
|
|
+ MetricsContext ctx = MetricsUtil.getContext("fairscheduler");
|
|
|
+ Collection<OutputRecord> records = ctx.getAllRecords().get("jobs");
|
|
|
+
|
|
|
+ System.out.println("Checking job metrics...");
|
|
|
+ Map<String, OutputRecord> byJobIdAndType =
|
|
|
+ new HashMap<String, OutputRecord>();
|
|
|
+ for (OutputRecord rec : records) {
|
|
|
+ String jobId = (String)rec.getTag("name");
|
|
|
+ String type = (String)rec.getTag("taskType");
|
|
|
+ assertNotNull(jobId);
|
|
|
+ assertNotNull(type);
|
|
|
+ byJobIdAndType.put(jobId + "_" + type, rec);
|
|
|
+ System.out.println("Got " + type + " metrics for job: " + jobId);
|
|
|
+ }
|
|
|
+ assertEquals(scheduler.infos.size() * 2, byJobIdAndType.size());
|
|
|
+
|
|
|
+ for (Map.Entry<JobInProgress, JobInfo> entry :
|
|
|
+ scheduler.infos.entrySet()) {
|
|
|
+ JobInfo info = entry.getValue();
|
|
|
+ String jobId = entry.getKey().getJobID().toString();
|
|
|
+
|
|
|
+ OutputRecord mapMetrics = byJobIdAndType.get(jobId + "_MAP");
|
|
|
+ assertNotNull("Job " + jobId + " should have map metrics", mapMetrics);
|
|
|
+ verifySchedulableMetrics(info.mapSchedulable, mapMetrics);
|
|
|
+
|
|
|
+ OutputRecord reduceMetrics = byJobIdAndType.get(jobId + "_REDUCE");
|
|
|
+ assertNotNull("Job " + jobId + " should have reduce metrics", reduceMetrics);
|
|
|
+ verifySchedulableMetrics(info.reduceSchedulable, reduceMetrics);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify that the metrics for a given Schedulable are correct
|
|
|
+ */
|
|
|
+ private void verifySchedulableMetrics(
|
|
|
+ Schedulable sched, OutputRecord metrics) {
|
|
|
+ assertEquals(sched.getRunningTasks(), metrics.getMetric("runningTasks"));
|
|
|
+ assertEquals(sched.getDemand(), metrics.getMetric("demand"));
|
|
|
+ assertEquals(sched.getFairShare(),
|
|
|
+ metrics.getMetric("fairShare").doubleValue(), .001);
|
|
|
+ assertEquals(sched.getWeight(),
|
|
|
+ metrics.getMetric("weight").doubleValue(), .001);
|
|
|
+ }
|
|
|
}
|