You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ma...@apache.org on 2009/11/05 19:38:42 UTC

svn commit: r833117 - in /hadoop/mapreduce/trunk: ./ src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/ src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/ src/docs/src/documentation/content/xdocs/

Author: matei
Date: Thu Nov  5 18:38:41 2009
New Revision: 833117

URL: http://svn.apache.org/viewvc?rev=833117&view=rev
Log:
MAPREDUCE-707. Provide a jobconf property for explicitly assigning a job to 
a pool in the Fair Scheduler. Contributed by Alan Heirich.


Modified:
    hadoop/mapreduce/trunk/CHANGES.txt
    hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
    hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
    hadoop/mapreduce/trunk/src/docs/src/documentation/content/xdocs/fair_scheduler.xml

Modified: hadoop/mapreduce/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/CHANGES.txt?rev=833117&r1=833116&r2=833117&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/CHANGES.txt (original)
+++ hadoop/mapreduce/trunk/CHANGES.txt Thu Nov  5 18:38:41 2009
@@ -11,6 +11,9 @@
 
   IMPROVEMENTS
 
+    MAPREDUCE-707. Provide a jobconf property for explicitly assigning a job to 
+    a pool in the Fair Scheduler. (Alan Heirich via matei)
+
     MAPREDUCE-999. Improve Sqoop test speed and refactor tests.
     (Aaron Kimball via tomwhite)
 

Modified: hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java?rev=833117&r1=833116&r2=833117&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java (original)
+++ hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java Thu Nov  5 18:38:41 2009
@@ -61,6 +61,8 @@
    */
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000; 
 
+  public static final String EXPLICIT_POOL_PROPERTY = "mapred.fairscheduler.pool";
+
   private final FairScheduler scheduler;
   
   // Map and reduce minimum allocations for each pool
@@ -391,7 +393,7 @@
    */
   public synchronized void setPool(JobInProgress job, String pool) {
     removeJob(job);
-    job.getJobConf().set(poolNameProperty, pool);
+    job.getJobConf().set(EXPLICIT_POOL_PROPERTY, pool);
     addJob(job);
   }
 
@@ -403,13 +405,16 @@
   }
   
   /**
-   * Get the pool name for a JobInProgress from its configuration. This uses
-   * the "project" property in the jobconf by default, or the property set with
-   * "mapred.fairscheduler.poolnameproperty".
+   * Get the pool name for a JobInProgress from its configuration.  This uses
+   * the value of mapred.fairscheduler.pool if specified, otherwise the value 
+   * of the property named in mapred.fairscheduler.poolnameproperty if that is
+   * specified.  Otherwise if neither is specified it uses the "user.name" property 
+   * in the jobconf by default.
    */
   public String getPoolName(JobInProgress job) {
     Configuration conf = job.getJobConf();
-    return conf.get(poolNameProperty, Pool.DEFAULT_POOL_NAME).trim();
+    return conf.get(EXPLICIT_POOL_PROPERTY,
+      conf.get(poolNameProperty, Pool.DEFAULT_POOL_NAME)).trim();
   }
 
   /**

Modified: hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java?rev=833117&r1=833116&r2=833117&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java (original)
+++ hadoop/mapreduce/trunk/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairScheduler.java Thu Nov  5 18:38:41 2009
@@ -52,6 +52,7 @@
       "test-pools").getAbsolutePath();
   
   private static final String POOL_PROPERTY = "pool";
+  private static final String EXPLICIT_POOL_PROPERTY = "mapred.fairscheduler.pool";
   
   private static int jobCounter;
   
@@ -2471,6 +2472,87 @@
     checkAssignment("tt2", "attempt_test_0001_r_000001_0 on tt2");
     checkAssignment("tt2", "attempt_test_0004_r_000000_0 on tt2");
   }
+
+  /**
+   * This test uses the mapred.fairscheduler.pool property to assign jobs to pools.
+   */
+  public void testPoolAssignment() 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());
+
+    // Submit a job to the default pool and move it to poolA using setPool.
+    JobInProgress job2 = submitJob(JobStatus.RUNNING, 5, 7);
+
+    assertEquals(6,    defaultPool.getMapSchedulable().getDemand());
+    assertEquals(10,   defaultPool.getReduceSchedulable().getDemand());
+    assertEquals(0,    poolA.getMapSchedulable().getDemand());
+    assertEquals(0,    poolA.getReduceSchedulable().getDemand());
+
+    scheduler.getPoolManager().setPool(job2, "poolA");
+    assertEquals("poolA", scheduler.getPoolManager().getPoolName(job2));
+
+    defaultPool.getMapSchedulable().updateDemand();
+    defaultPool.getReduceSchedulable().updateDemand();
+    poolA.getMapSchedulable().updateDemand();
+    poolA.getReduceSchedulable().updateDemand();
+
+    assertEquals(1,    defaultPool.getMapSchedulable().getDemand());
+    assertEquals(3,    defaultPool.getReduceSchedulable().getDemand());
+    assertEquals(5,    poolA.getMapSchedulable().getDemand());
+    assertEquals(7,    poolA.getReduceSchedulable().getDemand());
+
+    // Submit a job to poolA by specifying mapred.fairscheduler.pool
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setNumMapTasks(11);
+    jobConf.setNumReduceTasks(13);
+    jobConf.set(POOL_PROPERTY, "nonsense"); // test that this is overridden
+    jobConf.set(EXPLICIT_POOL_PROPERTY, "poolA");
+    JobInProgress job3 = new FakeJobInProgress(jobConf, taskTrackerManager,
+        null, UtilsForTests.getJobTracker());
+    job3.getStatus().setRunState(JobStatus.RUNNING);
+    taskTrackerManager.submitJob(job3);
+
+    assertEquals(1,    defaultPool.getMapSchedulable().getDemand());
+    assertEquals(3,    defaultPool.getReduceSchedulable().getDemand());
+    assertEquals(16,   poolA.getMapSchedulable().getDemand());
+    assertEquals(20,   poolA.getReduceSchedulable().getDemand());
+
+    // Submit a job to poolA by specifying pool and not mapred.fairscheduler.pool
+    JobConf jobConf2 = new JobConf(conf);
+    jobConf2.setNumMapTasks(17);
+    jobConf2.setNumReduceTasks(19);
+    jobConf2.set(POOL_PROPERTY, "poolA");
+    JobInProgress job4 = new FakeJobInProgress(jobConf2, taskTrackerManager,
+        null, UtilsForTests.getJobTracker());
+    job4.getStatus().setRunState(JobStatus.RUNNING);
+    taskTrackerManager.submitJob(job4);
+
+    assertEquals(1,    defaultPool.getMapSchedulable().getDemand());
+    assertEquals(3,    defaultPool.getReduceSchedulable().getDemand());
+    assertEquals(33,   poolA.getMapSchedulable().getDemand());
+    assertEquals(39,   poolA.getReduceSchedulable().getDemand());
+  }
   
   private void advanceTime(long time) {
     clock.advance(time);

Modified: hadoop/mapreduce/trunk/src/docs/src/documentation/content/xdocs/fair_scheduler.xml
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/docs/src/documentation/content/xdocs/fair_scheduler.xml?rev=833117&r1=833116&r2=833117&view=diff
==============================================================================
--- hadoop/mapreduce/trunk/src/docs/src/documentation/content/xdocs/fair_scheduler.xml (original)
+++ hadoop/mapreduce/trunk/src/docs/src/documentation/content/xdocs/fair_scheduler.xml Thu Nov  5 18:38:41 2009
@@ -163,6 +163,15 @@
           </tr>
           <tr>
           <td>
+            mapred.fairscheduler.pool
+          </td>
+          <td>
+            Specify the pool that a job belongs in.  
+            If this is specified then mapred.fairscheduler.poolnameproperty is ignored.
+          </td>
+          </tr>
+          <tr>
+          <td>
             mapred.fairscheduler.poolnameproperty
           </td>
           <td>
@@ -171,17 +180,8 @@
             (i.e. one pool for each user). 
             Another useful value is <em>group.name</em> to create a
             pool per Unix group.
-            Finally, a common setting is to use a non-standard property
-            such as <em>pool.name</em> as the pool name property, and make it
-            default to <em>mapreduce.job.mapreduce.job.user.name</em> through the following setting:<br/>
-            <code>&lt;property&gt;</code><br/> 
-            <code>&nbsp;&nbsp;&lt;name&gt;pool.name&lt;/name&gt;</code><br/>
-            <code>&nbsp;&nbsp;&lt;value&gt;${mapreduce.job.mapreduce.job.user.name}&lt;/value&gt;</code><br/>
-            <code>&lt;/property&gt;</code><br/>
-            This allows you to specify the pool name explicitly for some jobs
-            through the jobconf (e.g. passing <em>-Dpool.name=&lt;name&gt;</em>
-            to <em>bin/hadoop jar</em>, while having the default be the user's
-            pool.
+            mapred.fairscheduler. poolnameproperty is used only for jobs in which 
+            mapred.fairscheduler.pool is not explicitly set.
           </td>
           </tr>
           <tr>