소스 검색

MAPREDUCE-3134. Added documentation the CapacityScheduler.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1178530 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 13 년 전
부모
커밋
43b59b894b

+ 2 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -333,6 +333,8 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2889. Added documentation for writing new YARN applications.
     (Hitesh Shah via acmurthy) 
 
+    MAPREDUCE-3134. Added documentation the CapacityScheduler. (acmurthy) 
+
   OPTIMIZATIONS
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and

+ 35 - 19
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/resources/capacity-scheduler.xml

@@ -3,76 +3,92 @@
   <property>
     <name>yarn.scheduler.capacity.maximum-applications</name>
     <value>10000</value>
-    <description>Maximum number of applications that can be running.
+    <description>
+      Maximum number of applications that can be pending and running.
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
     <value>0.1</value>
+    <description>
+      Maximum percent of resources in the cluster which can be used to run 
+      application masters i.e. controls number of concurrent running
+      applications.
+    </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.queues</name>
     <value>default</value>
-    <description>The queues at the this level (root is the root queue).
+    <description>
+      The queues at the this level (root is the root queue).
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.capacity</name>
     <value>100</value>
-    <description>The total capacity as a percentage out of 100 for this queue.
-    If it has child queues then this includes their capacity as well.
-    The child queues capacity should add up to their parent queue's capacity
-    or less.</description>
-  </property>
-
-  <property>
-    <name>yarn.scheduler.capacity.root.acl_administer_queues</name>
-    <value>*</value>
-    <description>The ACL for who can administer this queue. i.e.
-    change sub queue allocations.</description>
+    <description>
+      The total capacity as a percentage out of 100 for this queue.
+      If it has child queues then this includes their capacity as well.
+      The child queues capacity should add up to their parent queue's capacity
+      or less.
+    </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.capacity</name>
     <value>100</value>
-    <description>default queue target capacity.</description>
+    <description>Default queue target capacity.</description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
     <value>1</value>
-    <description>default queue user limit a percantage from 0.0 to 1.0.
+    <description>
+      Default queue user limit a percentage from 0.0 to 1.0.
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
     <value>-1</value>
-    <description>the maximum capacity of the default queue -1 disables.
+    <description>
+      The maximum capacity of the default queue. A value of -1 disables this.
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.state</name>
     <value>RUNNING</value>
-    <description>The state of the default queue.  can be RUNNING or STOPPED
+    <description>
+      The state of the default queue. State can be one of RUNNING or STOPPED.
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.acl_submit_jobs</name>
     <value>*</value>
-    <description>The ACL of who can submit jobs to the default queue.
+    <description>
+      The ACL of who can submit jobs to the default queue.
     </description>
   </property>
 
   <property>
     <name>yarn.scheduler.capacity.root.default.acl_administer_jobs</name>
     <value>*</value>
-    <description>The ACL of who can administer jobs on the default queue.
+    <description>
+      The ACL of who can administer jobs on the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.acl_administer_queues</name>
+    <value>*</value>
+    <description>
+      The ACL for who can administer this queue i.e. change sub-queue 
+      allocations.
     </description>
   </property>
 

+ 309 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm

@@ -0,0 +1,309 @@
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~   http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License. See accompanying LICENSE file.
+
+  ---
+  Hadoop Map Reduce Next Generation-${project.version} - Capacity Scheduler
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop MapReduce Next Generation - Capacity Scheduler
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+
+* {Purpose} 
+
+  This document describes the <<<CapacityScheduler>>>, a pluggable 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.
+
+* {Overview}
+
+  The <<<CapacityScheduler>>> is designed to run Hadoop applications as a 
+  shared, multi-tenant cluster in an operator-friendly manner while maximizing 
+  the throughput and the utilization of the cluster.
+   
+  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 
+  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. 
+   
+  The <<<CapacityScheduler>>> is designed to allow sharing a large cluster while 
+  giving each organization capacity guarantees. The central idea is 
+  that the available resources in the Hadoop cluster are shared among multiple 
+  organizations who collectively fund the cluster based on their 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.
+   
+  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 
+  application or user or sets thereof. The <<<CapacityScheduler>>> provides a 
+  stringent set of limits to ensure that a single application or user or queue 
+  cannot consume dispropotionate amount of resources in the cluster. Also, the 
+  <<<CapacityScheduler>>> provides limits on initialized/pending applications 
+  from a single user and queue to ensure fairness and stability of the cluster.
+   
+  The primary abstraction provided by the <<<CapacityScheduler>>> is the concept 
+  of <queues>. These queues are typically setup by administrators to reflect the 
+  economics of the shared cluster. 
+  
+  To provide further control and predictability on sharing of resources, the 
+  <<<CapacityScheduler>>> supports <heirarchical queues> to ensure 
+  resources are shared among the sub-queues of an organization before other 
+  queues are allowed to use free resources, there-by providing <affinity> 
+  for sharing free resources among applications of a given organization.
+   
+* {Features}
+
+  The <<<CapacityScheduler>>> supports the following features:
+  
+  * Heirarchical Queues - Heirarchy of queues is supported to ensure resources 
+    are shared among the sub-queues of an organization before other 
+    queues are allowed to use free resources, there-by providing more control
+    and predictability.
+    
+  * Capacity Guarantees - 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 applications 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.
+    
+  * Security - Each queue has strict ACLs which controls which users can submit 
+    applications to individual queues. Also, there are safe-guards to ensure 
+    that users cannot view and/or modify applications from other users.
+    Also, per-queue and system administrator roles are supported.
+    
+  * 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 applications on queues 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.
+    
+  * Multi-tenancy - Comprehensive set of limits are provided to prevent a 
+    single application, user and queue from monpolizing resources of the queue 
+    or the cluster as a whole to ensure that the cluster isn't overwhelmed.
+    
+  * Operability
+  
+    * Runtime Configuration - The queue definitions and properties such as 
+      capacity, ACLs 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. Administrators can also 
+      <add additional queues> at runtime.
+      
+    * Drain applications - Administrators can <stop> queues
+      at runtime to ensure that while existing applications run to completion,
+      no new applications can be submitted. If a queue is in <<<STOPPED>>> 
+      state, new applications cannot be submitted to <itself> or 
+      <any of its child queueus>. Existing applications continue to completion, 
+      thus the queue can be <drained> gracefully.  Administrators can also 
+      <start> the stopped queues. 
+    
+  * Resource-based Scheduling - Support for resource-intensive applications, 
+    where-in a application 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.
+  
+  []
+  
+* {Configuration}
+
+  * Setting up <<<ResourceManager>>> to use <<<CapacityScheduler>>>
+  
+    To configure the <<<ResourceManager>>> to use the <<<CapacityScheduler>>>, set
+    the following property in the <<conf/yarn-site.xml>>:
+  
+*--------------------------------------+--------------------------------------+
+|| Property                            || Value                                |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.resourcemanager.scheduler.class>>> | |
+| | <<<org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.CapacityScheduler>>> |
+*--------------------------------------------+--------------------------------------------+
+
+  * Setting up <queues>
+   
+    <<conf/capacity-scheduler.xml>> is the configuration file for the
+    <<<CapacityScheduler>>>.  
+  
+    The <<<CapacityScheduler>>> has a pre-defined queue called <root>. All 
+    queueus in the system are children of the root queue.
+
+    Further queues can be setup by configuring 
+    <<<yarn.scheduler.capacity.root.queues>>> with a list of comma-separated
+    child queues.
+    
+    The configuration for <<<CapacityScheduler>>> uses a concept called
+    <queue path> to configure the heirarchy of queues. The <queue path> is the
+    full path of the queue's heirarcy, starting at <root>, with . (dot) as the 
+    delimiter.
+    
+    A given queue's children can be defined with the configuration knob:
+    <<<yarn.scheduler.capacity.<queue-path>.queues>>>
+     
+    Here is an example with three top-level child-queues <<<a>>>, <<<b>>> and 
+    <<<c>>> and some sub-queues for <<<a>>> and <<<b>>>:
+     
+----    
+<property>
+  <name>yarn.scheduler.capacity.root.queues</name>
+  <value>a,b,c</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+
+<property>
+  <name>yarn.scheduler.capacity.root.a.queues</name>
+  <value>a1,a2</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+
+<property>
+  <name>yarn.scheduler.capacity.root.b.queues</name>
+  <value>b1,b2,b3</value>
+  <description>The queues at the this level (root is the root queue).
+  </description>
+</property>
+----    
+
+  * Queue Properties
+  
+    * Resource Allocation
+  
+*--------------------------------------+--------------------------------------+
+|| Property                            || Description                         |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.<queue-path>.capacity>>> | |
+| | Queue <capacity> in percentage (%). | 
+| | The sum of capacities for all queues, at each level, should be less than | 
+| | or equal to 100. | 
+| | Applications in the queue may consume more resources than the queue's | 
+| | capacity if there are free resources, providing elasticity. |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.<queue-path>.maximum-capacity>>> |   | 
+| | Maximum queue capacity in percentage (%). |
+| | This limits the <elasticity> for applications in the queue. |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.<queue-path>.minimum-user-limit-percent>>> |   | 
+| | Each queue enforces a limit on the percentage of resources allocated to a | 
+| | user at any given time, if there is demand for resources. The user limit | 
+| | can vary between a minimum and maximum value. The former depends on the | 
+| | number of users who have submitted applications, and the latter is set to | 
+| | this property value. For e.g., suppose the value of this property is 25. | 
+| | If two users have submitted applications to a queue, no single user can |
+| | use more than 50% of the queue resources. If a third user submits an | 
+| | application, 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 queues |
+| | resources. A value of 100 implies no user limits are imposed. |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.<queue-path>.user-limit-factor>>> |   | 
+| | The multiple of the queue capacity which can be configured to allow a | 
+| | single user to acquire more resources. By default this is set to 1 which | 
+| | ensures that a single user can never take more than the queue's configured | 
+| | capacity irrespective of how idle th cluster is. |
+*--------------------------------------+--------------------------------------+
+
+    * Running and Pending Application Limits
+    
+    
+    The <<<CapacityScheduler>>> supports the following parameters to control 
+    the runnign and pending applications:
+    
+
+*--------------------------------------+--------------------------------------+
+|| Property                            || Description                         |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.maximum-applications>>> | |
+| | Maximum number of jobs in the system which can be concurently active |
+| | both running and pending. Limits on each queue are directly proportional |
+| | to their queue capacities. |
+*--------------------------------------+--------------------------------------+
+| yarn.scheduler.capacity.maximum-am-resource-percent | |
+| | Maximum percent of resources in the cluster which can be used to run |
+| | application masters - controls number of concurrent running applications. |
+*--------------------------------------+--------------------------------------+
+
+    * Queue Administration & Permissions
+    
+    The <<<CapacityScheduler>>> supports the following parameters to  
+    the administer the queues:
+    
+    
+*--------------------------------------+--------------------------------------+
+|| Property                            || Description                         |
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.<queue-path>.state>>> | |
+| | The <state> of the queue. Can be one of <<<RUNNING>>> or <<<STOPPED>>>. |
+| | If a queue is in <<<STOPPED>>> state, new applications cannot be |
+| | submitted to <itself> or <any of its child queueus>. | 
+| | Thus, if the <root> queue is <<<STOPPED>>> no applications can be | 
+| | submitted to the entire cluster. |
+| | Existing applications continue to completion, thus the queue can be 
+| | <drained> gracefully. | 
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.root.<queue-path>.acl_submit_jobs>>> | |
+| | The <ACL> which controls who can <submit> jobs to the given queue. |
+| | If the given user/group has necessary ACLs on the given queue or |
+| | <one of the parent queues in the hierarchy> they can submit jobs. | 
+*--------------------------------------+--------------------------------------+
+| <<<yarn.scheduler.capacity.root.<queue-path>.acl_administer_jobs>>> | |
+| | The <ACL> which controls who can <administer> jobs on the given queue. |
+| | If the given user/group has necessary ACLs on the given queue or |
+| | <one of the parent queues in the hierarchy> they can administer jobs. | 
+*--------------------------------------+--------------------------------------+
+    
+    <Note:> An <ACL> is of the form <user1>, <user2><space><group1>, <group2>.
+    The special value of <<*>> implies <anyone>. The special value of <space>
+    implies <no one>.
+     
+    * Reviewing the configuration of the CapacityScheduler
+
+      Once the installation and configuration is completed, you can review it 
+      after starting the YARN cluster from the web-ui.
+
+      * Start the YARN cluster in the normal manner.
+
+      * Open the <<<ResourceManager>>> web UI.
+
+      * The </scheduler> web-page should show the resource usages of individual 
+        queues.
+      
+      []
+      
+* {Changing Queue Configuration}
+
+  Changing queue properties and adding new queues is very simple. You need to
+  edit <<conf/capacity-scheduler.xml>> and run <rmadmin -refreshQueues>.
+  
+----
+$ vi $HADOOP_CONF_DIR/capacity-scheduler.xml
+$ $YARN_HOME/bin/rmadmin -refreshQueues
+----  
+
+  <Note:> Queues cannot be <deleted>, only addition of new queues is supported -
+  the updated queue configuration should be a valid one i.e. queue-capacity at
+  each <level> should be equal to 100%.
+  

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm

@@ -45,3 +45,5 @@ Hadoop MapReduce Next Generation
 
   * {{{./WritingYarnApplications.html}Writing Yarn Applications}}
 
+  * {{{./CapacityScheduler.html}Capacity Scheduler}}
+