Prechádzať zdrojové kódy

MAPREDUCE-2316. Updated CapacityScheduler documentation. (acmurthy)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-203@1097242 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 14 rokov pred
rodič
commit
0dc3968345

+ 2 - 2
CHANGES.txt

@@ -2,10 +2,10 @@ Hadoop Change Log
 
 
 Release 0.20.203.0 - unreleased
 Release 0.20.203.0 - unreleased
 
 
+    MAPREDUCE-2316. Updated CapacityScheduler documentation. (acmurthy) 
+
     HADOOP-7243. Fix contrib unit tests missing dependencies. (omalley)
     HADOOP-7243. Fix contrib unit tests missing dependencies. (omalley)
 
 
-    MAPREDUCE-2355. Add a dampner to out-of-band heartbeats. (acmurthy) 
- 
     HADOOP-7190. Add metrics v1 back for backwards compatibility. (omalley)
     HADOOP-7190. Add metrics v1 back for backwards compatibility. (omalley)
 
 
     MAPREDUCE-2360. Remove stripping of scheme, authority from submit dir in 
     MAPREDUCE-2360. Remove stripping of scheme, authority from submit dir in 

+ 454 - 196
src/docs/src/documentation/content/xdocs/capacity_scheduler.xml

@@ -20,7 +20,7 @@
 <document>
 <document>
   
   
   <header>
   <header>
-    <title>Capacity Scheduler Guide</title>
+    <title>CapacityScheduler Guide</title>
   </header>
   </header>
   
   
   <body>
   <body>
@@ -28,91 +28,125 @@
     <section>
     <section>
       <title>Purpose</title>
       <title>Purpose</title>
       
       
-      <p>This document describes the Capacity Scheduler, a pluggable 
-      MapReduce scheduler for Hadoop which provides a way to share 
-      large clusters.</p>
+      <p>This document describes the CapacityScheduler, a pluggable 
+      MapReduce scheduler for Hadoop which allows for multiple-tenants to 
+      securely share a large cluster such that their applications are allocated
+      resources in a timely manner under constraints of allocated capacities.
+      </p>
+    </section>
+    
+    <section>
+      <title>Overview</title>
+     
+      <p>The CapacityScheduler is designed to run Hadoop Map-Reduce as a 
+      shared, multi-tenant cluster in an operator-friendly manner while 
+      maximizing the throughput and the utilization of the cluster while
+      running Map-Reduce applications. </p>
+     
+      <p>Traditionally each organization has it own private set of compute 
+      resources that have sufficient capacity to meet the organization's SLA 
+      under peak or near peak conditions. This generally leads to poor average 
+      utilization and the overhead of managing multiple independent clusters, 
+      one per each organization. Sharing clusters between organizations is a 
+      cost-effective manner of running large Hadoop installations since this 
+      allows them to reap benefits of economies of scale without creating 
+      private clusters.  However, organizations are concerned about sharing a 
+      cluster because they are worried about others using the resources that 
+      are critical for their SLAs.</p> 
+
+      <p>The CapacityScheduler is designed to allow sharing a large cluster 
+      while giving  each organization a minimum capacity guarantee. The central 
+      idea is that the available resources in the Hadoop Map-Reduce cluster are 
+      partitioned among multiple organizations who collectively fund the 
+      cluster based on computing needs. There is an added benefit that an 
+      organization can access any excess capacity no being used by others. This 
+      provides elasticity for the organizations in a cost-effective manner.</p> 
+
+      <p>Sharing clusters across organizations necessitates strong support for
+      multi-tenancy since each organization must be guaranteed capacity and 
+      safe-guards to ensure the shared cluster is impervious to single rouge 
+      job or user. The CapacityScheduler provides a stringent set of limits to 
+      ensure that a single job or user or queue cannot consume dispropotionate 
+      amount of resources in the cluster. Also, the JobTracker of the cluster,  
+      in particular, is a precious resource and the CapacityScheduler provides 
+      limits on initialized/pending tasks and jobs from a single user and queue 
+      to ensure fairness and stability of the cluster.</p> 
+
+      <p>The primary abstraction provided by the CapacityScheduler is the 
+      concept of <em>queues</em>. These queues are typically setup by administrators
+      to reflect the economics of the shared cluster.</p>
     </section>
     </section>
     
     
     <section>
     <section>
       <title>Features</title>
       <title>Features</title>
       
       
-      <p>The Capacity Scheduler supports the following features:</p> 
+      <p>The CapacityScheduler supports the following features:</p> 
       <ul>
       <ul>
         <li>
         <li>
-          Support for multiple queues, where a job is submitted to a queue.
+          Capacity Guarantees - Support for multiple queues, where a job is 
+          submitted to a queue.Queues are allocated a fraction of the capacity 
+          of the grid in the sense that a certain capacity of resources will be 
+          at their disposal. All jobs submitted to a queue will have access to 
+          the capacity allocated to the queue. Adminstrators can configure soft 
+          limits and optional hard limits on the capacity allocated to each queue. 
         </li>
         </li>
         <li>
         <li>
-          Queues are allocated a fraction of the capacity of the grid in the 
-          sense that a certain capacity of resources will be at their 
-          disposal. All jobs submitted to a queue will have access to the 
-          capacity allocated to the queue.
+          Security - Each queue has strict ACLs which controls which users can 
+          submit jobs to individual queues. Also, there are safe-guards to 
+          ensure that users cannot view and/or modify jobs from other users if
+          so desired. Also, per-queue and system administrator roles are 
+          supported.
         </li>
         </li>
         <li>
         <li>
-          Free resources can be allocated to any queue beyond it's capacity. 
-          When there is demand for these resources from queues running below 
-          capacity at a future point in time, as tasks scheduled on these 
+          Elasticity - Free resources can be allocated to any queue beyond it's 
+          capacity. When there is demand for these resources from queues running 
+          below capacity at a future point in time, as tasks scheduled on these 
           resources complete, they will be assigned to jobs on queues 
           resources complete, they will be assigned to jobs on queues 
-          running below the capacity.
+          running below the capacity. This ensures that resources are available 
+          in a predictable and elastic manner to queues, thus preventing 
+          artifical silos of resources in the cluster which helps utilization.
         </li>
         </li>
         <li>
         <li>
-          Queues optionally support job priorities (disabled by default).
+          Multi-tenancy - Comprehensive set of limits are provided to prevent 
+          a single job, user and queue from monpolizing resources of the queue 
+          or the cluster as a whole to ensure that the system, particularly the 
+          JobTracker, isn't overwhelmed by too many tasks or jobs. 
         </li>
         </li>
         <li>
         <li>
-          Within a queue, jobs with higher priority will have access to the 
-          queue's resources before jobs with lower priority. However, once a 
-          job is running, it will not be preempted for a higher priority job,
-          though new tasks from the higher priority job will be 
-          preferentially scheduled.
+          Operability - The queue definitions and properties can be changed, 
+          at runtime, by administrators in a secure manner to minimize 
+          disruption to users. Also, a console is provided for users and 
+          administrators to view current allocation of resources to various 
+          queues in the system.
         </li>
         </li>
         <li>
         <li>
-          In order to prevent one or more users from monopolizing its 
-          resources, each queue enforces a limit on the percentage of 
-          resources allocated to a user at any given time, if there is 
-          competition for them.  
+          Resource-based Scheduling - Support for resource-intensive jobs, 
+          wherein a job can optionally specify higher resource-requirements than 
+          the default, there-by accomodating applications with differing resource
+          requirements. Currently, memory is the the resource requirement 
+          supported.
         </li>
         </li>
         <li>
         <li>
-          Support for memory-intensive jobs, wherein a job can optionally 
-          specify higher memory-requirements than the default, and the tasks 
-          of the job will only be run on TaskTrackers that have enough memory 
-          to spare.
+          Job Priorities - Queues optionally support job priorities (disabled 
+          by default). Within a queue, jobs with higher priority will have 
+          access to the queue's resources before jobs with lower priority. 
+          However, once a job is running, it will not be preempted for a higher 
+          priority job, <em>premption</em> is on the roadmap is currently not 
+          supported.
         </li>
         </li>
       </ul>
       </ul>
     </section>
     </section>
     
     
-    <section>
-      <title>Picking a task to run</title>
-      
-      <p>Note that many of these steps can be, and will be, enhanced over time
-      to provide better algorithms.</p>
-      
-      <p>Whenever a TaskTracker is free, the Capacity Scheduler picks 
-      a queue which has most free space (whose ratio of # of running slots to 
-      capacity is the lowest).</p>
-      
-      <p>Once a queue is selected, the Scheduler picks a job in the queue. Jobs
-      are sorted based on when they're submitted and their priorities (if the 
-      queue supports priorities). Jobs are considered in order, and a job is 
-      selected if its user is within the user-quota for the queue, i.e., the 
-      user is not already using queue resources above his/her limit. The 
-      Scheduler also makes sure that there is enough free memory in the 
-      TaskTracker to tun the job's task, in case the job has special memory
-      requirements.</p>
-      
-      <p>Once a job is selected, the Scheduler picks a task to run. This logic 
-      to pick a task remains unchanged from earlier versions.</p> 
-      
-    </section>
-    
     <section>
     <section>
       <title>Installation</title>
       <title>Installation</title>
       
       
-        <p>The Capacity Scheduler is available as a JAR file in the Hadoop
+        <p>The CapacityScheduler is available as a JAR file in the Hadoop
         tarball under the <em>contrib/capacity-scheduler</em> directory. The name of 
         tarball under the <em>contrib/capacity-scheduler</em> directory. The name of 
         the JAR file would be on the lines of hadoop-*-capacity-scheduler.jar.</p>
         the JAR file would be on the lines of hadoop-*-capacity-scheduler.jar.</p>
         <p>You can also build the Scheduler from source by executing
         <p>You can also build the Scheduler from source by executing
         <em>ant package</em>, in which case it would be available under
         <em>ant package</em>, in which case it would be available under
         <em>build/contrib/capacity-scheduler</em>.</p>
         <em>build/contrib/capacity-scheduler</em>.</p>
-        <p>To run the Capacity Scheduler in your Hadoop installation, you need 
+        <p>To run the CapacityScheduler in your Hadoop installation, you need 
         to put it on the <em>CLASSPATH</em>. The easiest way is to copy the 
         to put it on the <em>CLASSPATH</em>. The easiest way is to copy the 
         <code>hadoop-*-capacity-scheduler.jar</code> from 
         <code>hadoop-*-capacity-scheduler.jar</code> from 
         to <code>HADOOP_HOME/lib</code>. Alternatively, you can modify 
         to <code>HADOOP_HOME/lib</code>. Alternatively, you can modify 
@@ -124,9 +158,9 @@
       <title>Configuration</title>
       <title>Configuration</title>
 
 
       <section>
       <section>
-        <title>Using the Capacity Scheduler</title>
+        <title>Using the CapacityScheduler</title>
         <p>
         <p>
-          To make the Hadoop framework use the Capacity Scheduler, set up
+          To make the Hadoop framework use the CapacityScheduler, set up
           the following property in the site configuration:</p>
           the following property in the site configuration:</p>
           <table>
           <table>
             <tr>
             <tr>
@@ -144,14 +178,22 @@
         <title>Setting up queues</title>
         <title>Setting up queues</title>
         <p>
         <p>
           You can define multiple queues to which users can submit jobs with
           You can define multiple queues to which users can submit jobs with
-          the Capacity Scheduler. To define multiple queues, you should edit
-          the site configuration for Hadoop and modify the
-          <em>mapred.queue.names</em> property.
+          the CapacityScheduler. To define multiple queues, you should use the  
+          <em>mapred.queue.names</em> property in 
+          <code>conf/hadoop-site.xml</code>.
         </p>
         </p>
+        
+        <p>
+          The CapacityScheduler can be configured with several properties
+          for each queue that control the behavior of the Scheduler. This
+          configuration is in the <em>conf/capacity-scheduler.xml</em>.
+        </p>
+        
         <p>
         <p>
           You can also configure ACLs for controlling which users or groups
           You can also configure ACLs for controlling which users or groups
-          have access to the queues.
+          have access to the queues in <code>conf/mapred-queue-acls.xml</code>.
         </p>
         </p>
+        
         <p>
         <p>
           For more details, refer to
           For more details, refer to
           <a href="cluster_setup.html#Configuring+the+Hadoop+Daemons">Cluster 
           <a href="cluster_setup.html#Configuring+the+Hadoop+Daemons">Cluster 
@@ -160,25 +202,12 @@
       </section>
       </section>
   
   
       <section>
       <section>
-        <title>Configuring properties for queues</title>
+        <title>Queue properties</title>
 
 
-        <p>The Capacity Scheduler can be configured with several properties
-        for each queue that control the behavior of the Scheduler. This
-        configuration is in the <em>conf/capacity-scheduler.xml</em>. By
-        default, the configuration is set up for one queue, named 
-        <em>default</em>.</p>
-        <p>To specify a property for a queue that is defined in the site
-        configuration, you should use the property name as
-        <em>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.&lt;property-name&gt;</em>.
-        </p>
-        <p>For example, to define the property <em>capacity</em>
-        for queue named <em>research</em>, you should specify the property
-        name as 
-        <em>mapred.capacity-scheduler.queue.research.capacity</em>.
-        </p>
-
-        <p>The properties defined for queues and their descriptions are
-        listed in the table below:</p>
+        <section>
+        <title>Resource allocation</title>
+        <p>The properties defined for resource allocations to queues and their 
+        descriptions are listed in below:</p>
 
 
         <table>
         <table>
           <tr><th>Name</th><th>Description</th></tr>
           <tr><th>Name</th><th>Description</th></tr>
@@ -187,25 +216,8 @@
             to be available for jobs in this queue. The sum of capacities 
             to be available for jobs in this queue. The sum of capacities 
             for all queues should be less than or equal 100.</td>
             for all queues should be less than or equal 100.</td>
           </tr>
           </tr>
-          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.supports-priority</td>
-          	<td>If true, priorities of jobs will be taken into account in scheduling 
-          	decisions.</td>
-          </tr>
-          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.minimum-user-limit-percent</td>
-          	<td>Each queue enforces a limit on the percentage of resources 
-          	allocated to a user at any given time, if there is competition 
-          	for them. This user limit can vary between a minimum and maximum 
-          	value. The former depends on the number of users who have submitted
-          	jobs, and the latter is set to this property value. For example, 
-          	suppose the value of this property is 25. If two users have 
-          	submitted jobs to a queue, no single user can use more than 50% 
-          	of the queue resources. If a third user submits a job, no single 
-          	user can use more than 33% of the queue resources. With 4 or more 
-          	users, no user can use more than 25% of the queue's resources. A 
-          	value of 100 implies no user limits are imposed.</td>
-          </tr>
           <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-capacity</td>
           <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-capacity</td>
-          	<td>
+            <td>
                   maximum-capacity defines a limit beyond which a queue cannot
                   maximum-capacity defines a limit beyond which a queue cannot
                   use the capacity of the cluster.This provides a means to limit
                   use the capacity of the cluster.This provides a means to limit
                   how much excess capacity a queue can use. By default, there
                   how much excess capacity a queue can use. By default, there
@@ -228,137 +240,175 @@
                   absolute terms would increase accordingly.
                   absolute terms would increase accordingly.
                 </td>
                 </td>
           </tr>
           </tr>
-        </table>
-      </section>
-      
-      <section>
-        <title>Memory management</title>
-      
-        <p>The Capacity Scheduler supports scheduling of tasks on a
-        <code>TaskTracker</code>(TT) based on a job's memory requirements
-        and the availability of RAM and Virtual Memory (VMEM) on the TT node.
-        See the <a href="mapred_tutorial.html#Memory+monitoring"> 
-        MapReduce Tutorial</a> for details on how the TT monitors
-        memory usage.</p>
-        <p>Currently the memory based scheduling is only supported
-        in Linux platform.</p>
-        <p>Memory-based scheduling works as follows:</p>
-        <ol>
-          <li>The absence of any one or more of three config parameters 
-          or -1 being set as value of any of the parameters, 
-          <code>mapred.tasktracker.vmem.reserved</code>, 
-          <code>mapred.task.default.maxvmem</code>, or
-          <code>mapred.task.limit.maxvmem</code>, disables memory-based
-          scheduling, just as it disables memory monitoring for a TT. These
-          config parameters are described in the 
-          <a href="mapred_tutorial.html#Memory+monitoring">MapReduce 
-          Tutorial</a>. The value of  
-          <code>mapred.tasktracker.vmem.reserved</code> is 
-          obtained from the TT via its heartbeat. 
-          </li>
-          <li>If all the three mandatory parameters are set, the Scheduler 
-          enables VMEM-based scheduling. First, the Scheduler computes the free
-          VMEM on the TT. This is the difference between the available VMEM on the
-          TT (the node's total VMEM minus the offset, both of which are sent by 
-          the TT on each heartbeat)and the sum of VMs already allocated to 
-          running tasks (i.e., sum of the VMEM task-limits). Next, the Scheduler
-          looks at the VMEM requirements for the job that's first in line to 
-          run. If the job's VMEM requirements are less than the available VMEM on 
-          the node, the job's task can be scheduled. If not, the Scheduler 
-          ensures that the TT does not get a task to run (provided the job 
-          has tasks to run). This way, the Scheduler ensures that jobs with 
-          high memory requirements are not starved, as eventually, the TT 
-          will have enough VMEM available. If the high-mem job does not have 
-          any task to run, the Scheduler moves on to the next job. 
-          </li>
-          <li>In addition to VMEM, the Capacity Scheduler can also consider 
-          RAM on the TT node. RAM is considered the same way as VMEM. TTs report
-          the total RAM available on their node, and an offset. If both are
-          set, the Scheduler computes the available RAM on the node. Next, 
-          the Scheduler figures out the RAM requirements of the job, if any. 
-          As with VMEM, users can optionally specify a RAM limit for their job
-          (<code>mapred.task.maxpmem</code>, described in the MapReduce 
-          Tutorial). The Scheduler also maintains a limit for this value 
-          (<code>mapred.capacity-scheduler.task.default-pmem-percentage-in-vmem</code>, 
-          described below). All these three values must be set for the 
-          Scheduler to schedule tasks based on RAM constraints.
-          </li>
-          <li>The Scheduler ensures that jobs cannot ask for RAM or VMEM higher
-          than configured limits. If this happens, the job is failed when it
-          is submitted. 
-          </li>
-        </ol>
-        
-        <p>As described above, the additional scheduler-based config 
-        parameters are as follows:</p>
-
-        <table>
-          <tr><th>Name</th><th>Description</th></tr>
-          <tr><td>mapred.capacity-scheduler.task.default-pmem-percentage-in-vmem</td>
-          	<td>A percentage of the default VMEM limit for jobs
-          	(<code>mapred.task.default.maxvmem</code>). This is the default 
-          	RAM task-limit associated with a task. Unless overridden by a 
-          	job's setting, this number defines the RAM task-limit.</td>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.minimum-user-limit-percent</td>
+          	<td>Each queue enforces a limit on the percentage of resources 
+          	allocated to a user at any given time, if there is competition 
+          	for them. This user limit can vary between a minimum and maximum 
+          	value. The former depends on the number of users who have submitted
+          	jobs, and the latter is set to this property value. For example, 
+          	suppose the value of this property is 25. If two users have 
+          	submitted jobs to a queue, no single user can use more than 50% 
+          	of the queue resources. If a third user submits a job, no single 
+          	user can use more than 33% of the queue resources. With 4 or more 
+          	users, no user can use more than 25% of the queue's resources. A 
+          	value of 100 implies no user limits are imposed.</td>
+          </tr>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.user-limit-factor</td>
+            <td>The multiple of the queue capacity which can be configured to 
+              allow a single user to acquire more slots. By default this is set 
+              to 1 which ensure that a single user can never take more than the 
+              queue's configured capacity irrespective of how idle th cluster 
+              is.</td>
           </tr>
           </tr>
-          <tr><td>mapred.capacity-scheduler.task.limit.maxpmem</td>
-          <td>Configuration which provides an upper limit to maximum physical
-           memory which can be specified by a job. If a job requires more 
-           physical memory than what is specified in this limit then the same
-           is rejected.</td>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.supports-priority</td>
+            <td>If true, priorities of jobs will be taken into account in scheduling 
+            decisions.</td>
           </tr>
           </tr>
         </table>
         </table>
-      </section>
+   </section>
    <section>
    <section>
-        <title>Job Initialization Parameters</title>
+        <title>Job initialization</title>
         <p>Capacity scheduler lazily initializes the jobs before they are
         <p>Capacity scheduler lazily initializes the jobs before they are
         scheduled, for reducing the memory footprint on jobtracker. 
         scheduled, for reducing the memory footprint on jobtracker. 
-        Following are the parameters, by which you can control the laziness
-        of the job initialization. The following parameters can be 
-        configured in capacity-scheduler.xml
+        Following are the parameters, by which you can control the
+        initialization of jobs per-queue.
         </p>
         </p>
         
         
         <table>
         <table>
           <tr><th>Name</th><th>Description</th></tr>
           <tr><th>Name</th><th>Description</th></tr>
           <tr>
           <tr>
             <td>
             <td>
-              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-jobs-per-user
+              mapred.capacity-scheduler.maximum-system-jobs
             </td>
             </td>
             <td>
             <td>
-              Maximum number of jobs which are allowed to be pre-initialized for
-              a particular user in the queue. Once a job is scheduled, i.e. 
-              it starts running, then that job is not considered
-              while scheduler computes the maximum job a user is allowed to
-              initialize. 
+              Maximum number of jobs in the system which can be initialized,
+              concurrently, by the CapacityScheduler.
+              
+              Individual queue limits on initialized jobs are directly 
+              proportional to their queue capacities.
             </td>
             </td>
           </tr>
           </tr>
           <tr>
           <tr>
             <td>
             <td>
-              mapred.capacity-scheduler.init-poll-interval
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-active-tasks
             </td>
             </td>
             <td>
             <td>
-              Amount of time in miliseconds which is used to poll the scheduler
-              job queue to look for jobs to be initialized.
+              The maximum number of tasks, across all jobs in the queue, 
+              which can be initialized concurrently. Once the queue's jobs 
+              exceed this limit they will be queued on disk.             
             </td>
             </td>
           </tr>
           </tr>
           <tr>
           <tr>
             <td>
             <td>
-              mapred.capacity-scheduler.init-worker-threads
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-active-tasks-per-user
             </td>
             </td>
             <td>
             <td>
-              Number of worker threads which would be used by Initialization
-              poller to initialize jobs in a set of queue. If number mentioned 
-              in property is equal to number of job queues then a thread is 
-              assigned jobs from one queue. If the number configured is lesser than
-              number of queues, then a thread can get jobs from more than one queue
-              which it initializes in a round robin fashion. If the number configured
-              is greater than number of queues, then number of threads spawned
-              would be equal to number of job queues.
+              The maximum number of tasks per-user, across all the of the
+              user's jobs in the queue, which can be initialized concurrently. 
+              Once the user's jobs exceed this limit they will be queued on disk.
             </td>
             </td>
           </tr>
           </tr>
+          <tr>
+            <td> 
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.init-accept-jobs-factor
+            </td>
+            <td>
+              The multipe of (maximum-system-jobs * queue-capacity) used to
+              determine the number of jobs which are accepted by the scheduler. 
+              The default value is 10. If number of jobs submitted to the queue
+              exceeds this limit, job submission are rejected. 
+            </td>
+          </tr> 
         </table>
         </table>
       </section>   
       </section>   
+      </section>
+      
       <section>
       <section>
-        <title>Reviewing the configuration of the Capacity Scheduler</title>
+        <title>Resource based scheduling</title>
+      
+        <p>The CapacityScheduler supports scheduling of tasks on a
+        <code>TaskTracker</code>(TT) based on a job's memory requirements
+        in terms of RAM and Virtual Memory (VMEM) on the TT node.
+        A TT is conceptually composed of a fixed number of map and reduce
+        slots with fixed slot size across the cluster. A job can ask for one
+        or more slots for each of its component map and/or reduce slots. If a
+        task consumes more memory than configured the TT forcibly kills the task.
+        </p>
+
+        <p>Currently the memory based scheduling is only supported
+        in Linux platform.</p>
+        
+        <p>Additional scheduler-based config 
+        parameters are as follows:</p>
+
+        <table>
+          <tr><th>Name</th><th>Description</th></tr>
+          <tr>
+            <td>mapred.cluster.map.memory.mb</td>
+          	 <td>The size, in terms of virtual memory, of a single map slot
+             in the Map-Reduce framework, used by the scheduler.
+             A job can ask for multiple slots for a single map task via
+             <code>mapred.job.map.memory.mb</code>, upto the limit specified by
+             <code>mapred.cluster.max.map.memory.mb</code>, if the scheduler 
+             supports the feature.
+             The value of -1 indicates that this feature is turned off.
+          	 </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.reduce.memory.mb</td>
+             <td>The size, in terms of virtual memory, of a single reduce slot
+             in the Map-Reduce framework, used by the scheduler.
+             A job can ask for multiple slots for a single reduce task via
+             <code>mapred.job.reduce.memory.mb</code>, upto the limit specified by
+             <code>mapred.cluster.max.reduce.memory.mb</code>, if the scheduler supports the 
+             feature.The value of -1 indicates that this feature is turned off.
+             </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.max.map.memory.mb</td>
+            <td>The maximum size, in terms of virtual memory, of a single map
+            task launched by the Map-Reduce framework, used by the scheduler.
+            A job can ask for multiple slots for a single map task via
+            <code>mapred.job.map.memory.mb</code>, upto the limit specified by
+            <code>mapred.cluster.max.map.memory.mb</code>, if the scheduler supports the 
+            feature. The value of -1 indicates that this feature is turned off.
+            </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.max.reduce.memory.mb</td>
+            <td>The maximum size, in terms of virtual memory, of a single reduce
+            task launched by the Map-Reduce framework, used by the scheduler.
+            A job can ask for multiple slots for a single reduce task via
+            <code>mapred.job.reduce.memory.mb</code>, upto the limit specified by
+            <code>mapred.cluster.max.reduce.memory.mb</code>, if the scheduler supports the 
+            feature. The value of -1 indicates that this feature is turned off.
+            </td>
+          </tr>
+          <tr>
+            <td>mapred.job.map.memory.mb</td>
+            <td>The size, in terms of virtual memory, of a single map task
+            for the job. A job can ask for multiple slots for a single map task, 
+            rounded up to the next multiple of <code>mapred.cluster.map.memory.mb</code> and 
+            upto the limit specified by <code>mapred.cluster.max.map.memory.mb</code>, 
+            if the scheduler supports the feature. The value of -1 indicates 
+            that this feature is turned off iff <code>mapred.cluster.map.memory.mb</code> is 
+            also turned off (-1).</td>
+          </tr>
+          <tr>
+            <td>mapred.job.reduce.memory.mb</td>
+            <td>The size, in terms of virtual memory, of a single reduce task
+            for the job. A job can ask for multiple slots for a single reduce task, 
+            rounded up to the next multiple of <code>mapred.cluster.reduce.memory.mb</code> and 
+            upto the limit specified by <code>mapred.cluster.max.reduce.memory.mb</code>, 
+            if the scheduler supports the feature. The value of -1 indicates 
+            that this feature is turned off iff <code>mapred.cluster.reduce.memory.mb</code> is 
+            also turned off (-1).</td>
+          </tr>
+        </table>
+      </section>
+      
+      <section>
+        <title>Reviewing the configuration of the CapacityScheduler</title>
         <p>
         <p>
           Once the installation and configuration is completed, you can review
           Once the installation and configuration is completed, you can review
           it after starting the MapReduce cluster from the admin UI.
           it after starting the MapReduce cluster from the admin UI.
@@ -370,10 +420,218 @@
               Information</em> section of the page.</li>
               Information</em> section of the page.</li>
           <li>The properties for the queues should be visible in the <em>Scheduling
           <li>The properties for the queues should be visible in the <em>Scheduling
               Information</em> column against each queue.</li>
               Information</em> column against each queue.</li>
+          <li>The /scheduler web-page should show the resource usages of 
+              individual queues.</li>
         </ul>
         </ul>
       </section>
       </section>
       
       
    </section>
    </section>
+
+  <section>
+    <title>Example</title>
+    <p>Here is a practical example for using CapacityScheduler:</p>
+    <table>
+    <tr>
+    <td>
+<code>&lt;?xml version="1.0"?&gt;</code><br/>
+<br/>
+<code>&lt;configuration&gt;</code><br/>
+<br/>
+<code>  &lt;!-- system limit, across all queues --&gt;</code><br/>
+<br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.maximum-system-jobs&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;3000&lt;/value&gt;</code><br/>
+<code>    &lt;description&gt;Maximum number of jobs in the system which can be initialized,</code><br/>
+<code>     concurrently, by the CapacityScheduler.</code><br/>
+<code>    &lt;/description&gt;    </code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code> </code><br/>
+<code>&lt;!-- queue: queueA --&gt;</code><br/>
+<code> &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;8&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueB --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;2&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueC --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;30&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueD --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueE --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;31&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueF --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;28&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;/configuration&gt;</code><br/>
+    </td>
+    </tr>
+    </table>
+  </section>
   </body>
   </body>
   
   
 </document>
 </document>