@InterfaceAudience.Public @InterfaceStability.Evolving public abstract class SchedulingPolicy extends Object
Every queue has a policy, including parents and children. If a child
queue doesn't specify one, it inherits the parent's policy.
The policy for a child queue must be compatible with the policy of
the parent queue; there are some combinations that aren't allowed.
See isChildPolicyAllowed(SchedulingPolicy)
.
The policy for a queue is specified by setting property
schedulingPolicy in the fair scheduler configuration file.
The default policy is FairSharePolicy
if not specified.
Modifier and Type | Field and Description |
---|---|
static SchedulingPolicy |
DEFAULT_POLICY |
Constructor and Description |
---|
SchedulingPolicy() |
Modifier and Type | Method and Description |
---|---|
abstract boolean |
checkIfUsageOverFairShare(org.apache.hadoop.yarn.api.records.Resource usage,
org.apache.hadoop.yarn.api.records.Resource fairShare)
Check if the resource usage is over the fair share under this policy.
|
abstract void |
computeShares(Collection<? extends Schedulable> schedulables,
org.apache.hadoop.yarn.api.records.Resource totalResources)
Computes and updates the shares of
Schedulable s as per
the SchedulingPolicy , to be used later for scheduling decisions. |
abstract void |
computeSteadyShares(Collection<? extends FSQueue> queues,
org.apache.hadoop.yarn.api.records.Resource totalResources)
Computes and updates the steady shares of
FSQueue s as per the
SchedulingPolicy . |
abstract Comparator<Schedulable> |
getComparator()
The comparator returned by this method is to be used for sorting the
Schedulable s in that queue. |
abstract org.apache.hadoop.yarn.api.records.Resource |
getHeadroom(org.apache.hadoop.yarn.api.records.Resource queueFairShare,
org.apache.hadoop.yarn.api.records.Resource queueUsage,
org.apache.hadoop.yarn.api.records.Resource maxAvailable)
Get headroom by calculating the min of
clusterAvailable and
(queueFairShare - queueUsage ) resources that are
applicable to this policy. |
static SchedulingPolicy |
getInstance(Class<? extends SchedulingPolicy> clazz)
Returns a
SchedulingPolicy instance corresponding
to the passed clazz. |
abstract String |
getName() |
abstract org.apache.hadoop.yarn.util.resource.ResourceCalculator |
getResourceCalculator()
The
ResourceCalculator returned by this method should be used
for any calculations involving resources. |
void |
initialize(FSContext fsContext)
Initialize the scheduling policy with a
FSContext object, which has
a pointer to the cluster resources among other information. |
void |
initialize(org.apache.hadoop.yarn.api.records.Resource clusterCapacity)
Deprecated.
Since it doesn't track cluster resource changes, replaced by
initialize(FSContext) . |
boolean |
isChildPolicyAllowed(SchedulingPolicy childPolicy)
Check whether the policy of a child queue is allowed.
|
static SchedulingPolicy |
parse(String policy)
Returns
SchedulingPolicy instance corresponding to the
SchedulingPolicy passed as a string. |
public static final SchedulingPolicy DEFAULT_POLICY
public static SchedulingPolicy getInstance(Class<? extends SchedulingPolicy> clazz)
SchedulingPolicy
instance corresponding
to the passed clazz.clazz
- a class that extends SchedulingPolicy
SchedulingPolicy
instancepublic static SchedulingPolicy parse(String policy) throws AllocationConfigurationException
SchedulingPolicy
instance corresponding to the
SchedulingPolicy
passed as a string. The policy can be "fair" for
FairSharePolicy, "fifo" for FifoPolicy, or "drf" for
DominantResourceFairnessPolicy. For a custom
SchedulingPolicy
s in the RM classpath, the policy should be
canonical class name of the SchedulingPolicy
.policy
- canonical class name or "drf" or "fair" or "fifo"SchedulingPolicy
instance parsed from given policyAllocationConfigurationException
@Deprecated public void initialize(org.apache.hadoop.yarn.api.records.Resource clusterCapacity)
initialize(FSContext)
.clusterCapacity
- cluster resourcespublic void initialize(FSContext fsContext)
FSContext
object, which has
a pointer to the cluster resources among other information.fsContext
- a FSContext
object which has a pointer to the
cluster resourcespublic abstract org.apache.hadoop.yarn.util.resource.ResourceCalculator getResourceCalculator()
ResourceCalculator
returned by this method should be used
for any calculations involving resources.public abstract String getName()
SchedulingPolicy
public abstract Comparator<Schedulable> getComparator()
Schedulable
s in that queue.public abstract void computeShares(Collection<? extends Schedulable> schedulables, org.apache.hadoop.yarn.api.records.Resource totalResources)
Schedulable
s as per
the SchedulingPolicy
, to be used later for scheduling decisions.
The shares computed are instantaneous and only consider queues with
running applications.schedulables
- Schedulable
s whose shares are to be updatedtotalResources
- Total Resource
s in the clusterpublic abstract void computeSteadyShares(Collection<? extends FSQueue> queues, org.apache.hadoop.yarn.api.records.Resource totalResources)
FSQueue
s as per the
SchedulingPolicy
. The steady share does not differentiate
between queues with and without running applications under them. The
steady share is not used for scheduling, it is displayed on the Web UI
for better visibility.queues
- FSQueue
s whose shares are to be updatedtotalResources
- Total Resource
s in the clusterpublic abstract boolean checkIfUsageOverFairShare(org.apache.hadoop.yarn.api.records.Resource usage, org.apache.hadoop.yarn.api.records.Resource fairShare)
usage
- Resource
the resource usagefairShare
- Resource
the fair sharepublic abstract org.apache.hadoop.yarn.api.records.Resource getHeadroom(org.apache.hadoop.yarn.api.records.Resource queueFairShare, org.apache.hadoop.yarn.api.records.Resource queueUsage, org.apache.hadoop.yarn.api.records.Resource maxAvailable)
clusterAvailable
and
(queueFairShare
- queueUsage
) resources that are
applicable to this policy. For eg if only memory then leave other
resources such as CPU to same as clusterAvailable
.queueFairShare
- fairshare in the queuequeueUsage
- resources used in the queuemaxAvailable
- available resource in cluster for this queuepublic boolean isChildPolicyAllowed(SchedulingPolicy childPolicy)
childPolicy
- the policy of child queueCopyright © 2008–2019 Apache Software Foundation. All rights reserved.