YARN-6040. Introduce api independent PendingAsk to replace usage of ResourceRequest within Scheduler classes. (Wangda Tan via asuresh)
This commit is contained in:
parent
4a659ff40f
commit
2977bc6a14
|
@ -34,16 +34,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalitySchedulingPlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalitySchedulingPlacementSet;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -582,16 +584,10 @@ public class AppSchedulingInfo {
|
||||||
return schedulerKeys.keySet();
|
return schedulerKeys.keySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
/**
|
||||||
public Map<String, ResourceRequest> getResourceRequests(
|
* Used by REST API to fetch ResourceRequest
|
||||||
SchedulerRequestKey schedulerKey) {
|
* @return All pending ResourceRequests.
|
||||||
SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(schedulerKey);
|
*/
|
||||||
if (null != ps) {
|
|
||||||
return ps.getResourceRequests();
|
|
||||||
}
|
|
||||||
return Collections.emptyMap();
|
|
||||||
}
|
|
||||||
|
|
||||||
public List<ResourceRequest> getAllResourceRequests() {
|
public List<ResourceRequest> getAllResourceRequests() {
|
||||||
List<ResourceRequest> ret = new ArrayList<>();
|
List<ResourceRequest> ret = new ArrayList<>();
|
||||||
try {
|
try {
|
||||||
|
@ -605,53 +601,51 @@ public class AppSchedulingInfo {
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ResourceRequest getResourceRequest(SchedulerRequestKey schedulerKey,
|
public SchedulingPlacementSet getFirstSchedulingPlacementSet() {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
for (SchedulerRequestKey key : schedulerKeys.keySet()) {
|
||||||
|
SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(key);
|
||||||
|
if (null != ps) {
|
||||||
|
return ps;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public PendingAsk getNextPendingAsk() {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
SchedulerRequestKey firstRequestKey = schedulerKeys.firstKey();
|
||||||
|
return getPendingAsk(firstRequestKey, ResourceRequest.ANY);
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public PendingAsk getPendingAsk(SchedulerRequestKey schedulerKey) {
|
||||||
|
return getPendingAsk(schedulerKey, ResourceRequest.ANY);
|
||||||
|
}
|
||||||
|
|
||||||
|
public PendingAsk getPendingAsk(SchedulerRequestKey schedulerKey,
|
||||||
String resourceName) {
|
String resourceName) {
|
||||||
try {
|
try {
|
||||||
this.readLock.lock();
|
this.readLock.lock();
|
||||||
SchedulingPlacementSet ps =
|
SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(schedulerKey);
|
||||||
schedulerKeyToPlacementSets.get(schedulerKey);
|
return (ps == null) ? PendingAsk.ZERO : ps.getPendingAsk(resourceName);
|
||||||
return (ps == null) ? null : ps.getResourceRequest(resourceName);
|
|
||||||
} finally {
|
} finally {
|
||||||
this.readLock.unlock();
|
this.readLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Resource getResource(SchedulerRequestKey schedulerKey) {
|
|
||||||
try {
|
|
||||||
this.readLock.lock();
|
|
||||||
ResourceRequest request =
|
|
||||||
getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
|
||||||
return (request == null) ? null : request.getCapability();
|
|
||||||
} finally {
|
|
||||||
this.readLock.unlock();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Method to return the next resource request to be serviced.
|
|
||||||
*
|
|
||||||
* In the initial implementation, we just pick any {@link ResourceRequest}
|
|
||||||
* corresponding to the highest priority.
|
|
||||||
*
|
|
||||||
* @return next {@link ResourceRequest} to allocate resources for.
|
|
||||||
*/
|
|
||||||
@Unstable
|
|
||||||
public synchronized ResourceRequest getNextResourceRequest() {
|
|
||||||
SchedulingPlacementSet<SchedulerNode> ps = schedulerKeyToPlacementSets.get(
|
|
||||||
schedulerKeys.firstKey());
|
|
||||||
if (null != ps) {
|
|
||||||
for (ResourceRequest rr : ps.getResourceRequests().values()) {
|
|
||||||
return rr;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns if the place (node/rack today) is either blacklisted by the
|
* Returns if the place (node/rack today) is either blacklisted by the
|
||||||
* application (user) or the system
|
* application (user) or the system.
|
||||||
*
|
*
|
||||||
* @param resourceName
|
* @param resourceName
|
||||||
* the resourcename
|
* the resourcename
|
||||||
|
@ -724,7 +718,6 @@ public class AppSchedulingInfo {
|
||||||
|
|
||||||
public List<ResourceRequest> allocate(NodeType type,
|
public List<ResourceRequest> allocate(NodeType type,
|
||||||
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest request,
|
|
||||||
Container containerAllocated) {
|
Container containerAllocated) {
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
@ -733,19 +726,13 @@ public class AppSchedulingInfo {
|
||||||
updateMetricsForAllocatedContainer(type, containerAllocated);
|
updateMetricsForAllocatedContainer(type, containerAllocated);
|
||||||
}
|
}
|
||||||
|
|
||||||
return schedulerKeyToPlacementSets.get(schedulerKey)
|
return schedulerKeyToPlacementSets.get(schedulerKey).allocate(
|
||||||
.allocate(schedulerKey, type, node, request);
|
schedulerKey, type, node);
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<ResourceRequest> allocate(NodeType type,
|
|
||||||
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
|
||||||
Container containerAllocated) {
|
|
||||||
return allocate(type, node, schedulerKey, null, containerAllocated);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void checkForDeactivation() {
|
public void checkForDeactivation() {
|
||||||
if (schedulerKeys.isEmpty()) {
|
if (schedulerKeys.isEmpty()) {
|
||||||
activeUsersManager.deactivateApplication(user, applicationId);
|
activeUsersManager.deactivateApplication(user, applicationId);
|
||||||
|
@ -758,18 +745,20 @@ public class AppSchedulingInfo {
|
||||||
QueueMetrics oldMetrics = queue.getMetrics();
|
QueueMetrics oldMetrics = queue.getMetrics();
|
||||||
QueueMetrics newMetrics = newQueue.getMetrics();
|
QueueMetrics newMetrics = newQueue.getMetrics();
|
||||||
for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) {
|
for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) {
|
||||||
ResourceRequest request = ps.getResourceRequest(ResourceRequest.ANY);
|
PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY);
|
||||||
if (request != null && request.getNumContainers() > 0) {
|
if (ask.getCount() > 0) {
|
||||||
oldMetrics.decrPendingResources(user, request.getNumContainers(),
|
oldMetrics.decrPendingResources(user, ask.getCount(),
|
||||||
request.getCapability());
|
ask.getPerAllocationResource());
|
||||||
newMetrics.incrPendingResources(user, request.getNumContainers(),
|
newMetrics.incrPendingResources(user, ask.getCount(),
|
||||||
request.getCapability());
|
ask.getPerAllocationResource());
|
||||||
|
|
||||||
Resource delta = Resources.multiply(request.getCapability(),
|
Resource delta = Resources.multiply(ask.getPerAllocationResource(),
|
||||||
request.getNumContainers());
|
ask.getCount());
|
||||||
// Update Queue
|
// Update Queue
|
||||||
queue.decPendingResource(request.getNodeLabelExpression(), delta);
|
queue.decPendingResource(
|
||||||
newQueue.incPendingResource(request.getNodeLabelExpression(), delta);
|
ps.getPrimaryRequestedNodePartition(), delta);
|
||||||
|
newQueue.incPendingResource(
|
||||||
|
ps.getPrimaryRequestedNodePartition(), delta);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
oldMetrics.moveAppFrom(this);
|
oldMetrics.moveAppFrom(this);
|
||||||
|
@ -789,16 +778,16 @@ public class AppSchedulingInfo {
|
||||||
this.writeLock.lock();
|
this.writeLock.lock();
|
||||||
QueueMetrics metrics = queue.getMetrics();
|
QueueMetrics metrics = queue.getMetrics();
|
||||||
for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) {
|
for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) {
|
||||||
ResourceRequest request = ps.getResourceRequest(ResourceRequest.ANY);
|
PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY);
|
||||||
if (request != null && request.getNumContainers() > 0) {
|
if (ask.getCount() > 0) {
|
||||||
metrics.decrPendingResources(user, request.getNumContainers(),
|
metrics.decrPendingResources(user, ask.getCount(),
|
||||||
request.getCapability());
|
ask.getPerAllocationResource());
|
||||||
|
|
||||||
// Update Queue
|
// Update Queue
|
||||||
queue.decPendingResource(
|
queue.decPendingResource(
|
||||||
request.getNodeLabelExpression(),
|
ps.getPrimaryRequestedNodePartition(),
|
||||||
Resources.multiply(request.getCapability(),
|
Resources.multiply(ask.getPerAllocationResource(),
|
||||||
request.getNumContainers()));
|
ask.getCount()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
metrics.finishAppAttempt(applicationId, pending, user);
|
metrics.finishAppAttempt(applicationId, pending, user);
|
||||||
|
@ -906,4 +895,38 @@ public class AppSchedulingInfo {
|
||||||
return (SchedulingPlacementSet<N>) schedulerKeyToPlacementSets.get(
|
return (SchedulingPlacementSet<N>) schedulerKeyToPlacementSets.get(
|
||||||
schedulerkey);
|
schedulerkey);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Can delay to next?.
|
||||||
|
*
|
||||||
|
* @param schedulerKey schedulerKey
|
||||||
|
* @param resourceName resourceName
|
||||||
|
*
|
||||||
|
* @return If request exists, return {relaxLocality}
|
||||||
|
* Otherwise, return true.
|
||||||
|
*/
|
||||||
|
public boolean canDelayTo(
|
||||||
|
SchedulerRequestKey schedulerKey, String resourceName) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
|
SchedulingPlacementSet ps =
|
||||||
|
schedulerKeyToPlacementSets.get(schedulerKey);
|
||||||
|
return (ps == null) || ps.canDelayTo(resourceName);
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean acceptNodePartition(SchedulerRequestKey schedulerKey,
|
||||||
|
String nodePartition, SchedulingMode schedulingMode) {
|
||||||
|
try {
|
||||||
|
this.readLock.lock();
|
||||||
|
SchedulingPlacementSet ps =
|
||||||
|
schedulerKeyToPlacementSets.get(schedulerKey);
|
||||||
|
return (ps != null) && ps.acceptNodePartition(nodePartition,
|
||||||
|
schedulingMode);
|
||||||
|
} finally {
|
||||||
|
this.readLock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -78,6 +77,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Scheduli
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
|
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
|
||||||
|
@ -283,11 +283,6 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
return appSchedulingInfo.getUser();
|
return appSchedulingInfo.getUser();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, ResourceRequest> getResourceRequests(
|
|
||||||
SchedulerRequestKey schedulerKey) {
|
|
||||||
return appSchedulingInfo.getResourceRequests(schedulerKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Set<ContainerId> getPendingRelease() {
|
public Set<ContainerId> getPendingRelease() {
|
||||||
return this.pendingRelease;
|
return this.pendingRelease;
|
||||||
}
|
}
|
||||||
|
@ -300,33 +295,27 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
return appSchedulingInfo.getSchedulerKeys();
|
return appSchedulingInfo.getSchedulerKeys();
|
||||||
}
|
}
|
||||||
|
|
||||||
public ResourceRequest getResourceRequest(
|
public PendingAsk getPendingAsk(
|
||||||
SchedulerRequestKey schedulerKey, String resourceName) {
|
SchedulerRequestKey schedulerKey, String resourceName) {
|
||||||
try {
|
try {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
|
return appSchedulingInfo.getPendingAsk(schedulerKey, resourceName);
|
||||||
} finally {
|
|
||||||
readLock.unlock();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getTotalRequiredResources(
|
|
||||||
SchedulerRequestKey schedulerKey) {
|
|
||||||
try {
|
|
||||||
readLock.lock();
|
|
||||||
ResourceRequest request =
|
|
||||||
getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
|
||||||
return request == null ? 0 : request.getNumContainers();
|
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Resource getResource(SchedulerRequestKey schedulerKey) {
|
public int getOutstandingAsksCount(SchedulerRequestKey schedulerKey) {
|
||||||
|
return getOutstandingAsksCount(schedulerKey, ResourceRequest.ANY);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getOutstandingAsksCount(SchedulerRequestKey schedulerKey,
|
||||||
|
String resourceName) {
|
||||||
try {
|
try {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
return appSchedulingInfo.getResource(schedulerKey);
|
SchedulingPlacementSet ps = appSchedulingInfo.getSchedulingPlacementSet(
|
||||||
|
schedulerKey);
|
||||||
|
return ps == null ? 0 : ps.getOutstandingAsksCount(resourceName);
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -625,16 +614,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
||||||
try {
|
try {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
|
for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
|
||||||
Map<String, ResourceRequest> requests = getResourceRequests(
|
SchedulingPlacementSet ps = getSchedulingPlacementSet(schedulerKey);
|
||||||
schedulerKey);
|
if (ps != null &&
|
||||||
if (requests != null) {
|
ps.getOutstandingAsksCount(ResourceRequest.ANY) > 0) {
|
||||||
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
||||||
+ " headRoom=" + getHeadroom() + " currentConsumption="
|
+ " headRoom=" + getHeadroom() + " currentConsumption="
|
||||||
+ attemptResourceUsage.getUsed().getMemorySize());
|
+ attemptResourceUsage.getUsed().getMemorySize());
|
||||||
for (ResourceRequest request : requests.values()) {
|
ps.showRequests();
|
||||||
LOG.debug("showRequests:" + " application=" + getApplicationId()
|
|
||||||
+ " request=" + request);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
|
||||||
|
@ -46,6 +47,7 @@ public abstract class AbstractContainerAllocator {
|
||||||
private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class);
|
private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class);
|
||||||
|
|
||||||
FiCaSchedulerApp application;
|
FiCaSchedulerApp application;
|
||||||
|
AppSchedulingInfo appInfo;
|
||||||
final ResourceCalculator rc;
|
final ResourceCalculator rc;
|
||||||
final RMContext rmContext;
|
final RMContext rmContext;
|
||||||
ActivitiesManager activitiesManager;
|
ActivitiesManager activitiesManager;
|
||||||
|
@ -59,6 +61,8 @@ public abstract class AbstractContainerAllocator {
|
||||||
ResourceCalculator rc, RMContext rmContext,
|
ResourceCalculator rc, RMContext rmContext,
|
||||||
ActivitiesManager activitiesManager) {
|
ActivitiesManager activitiesManager) {
|
||||||
this.application = application;
|
this.application = application;
|
||||||
|
this.appInfo =
|
||||||
|
application == null ? null : application.getAppSchedulingInfo();
|
||||||
this.rc = rc;
|
this.rc = rc;
|
||||||
this.rmContext = rmContext;
|
this.rmContext = rmContext;
|
||||||
this.activitiesManager = activitiesManager;
|
this.activitiesManager = activitiesManager;
|
||||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
||||||
|
@ -54,6 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
@ -65,8 +65,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
public class RegularContainerAllocator extends AbstractContainerAllocator {
|
public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
private static final Log LOG = LogFactory.getLog(RegularContainerAllocator.class);
|
private static final Log LOG = LogFactory.getLog(RegularContainerAllocator.class);
|
||||||
|
|
||||||
private ResourceRequest lastResourceRequest = null;
|
|
||||||
|
|
||||||
public RegularContainerAllocator(FiCaSchedulerApp application,
|
public RegularContainerAllocator(FiCaSchedulerApp application,
|
||||||
ResourceCalculator rc, RMContext rmContext,
|
ResourceCalculator rc, RMContext rmContext,
|
||||||
ActivitiesManager activitiesManager) {
|
ActivitiesManager activitiesManager) {
|
||||||
|
@ -103,9 +101,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
Priority priority = schedulerKey.getPriority();
|
Priority priority = schedulerKey.getPriority();
|
||||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||||
|
|
||||||
ResourceRequest anyRequest =
|
PendingAsk offswitchPendingAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
ResourceRequest.ANY);
|
||||||
if (null == anyRequest) {
|
|
||||||
|
if (offswitchPendingAsk.getCount() <= 0) {
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
activitiesManager, node, application, priority,
|
activitiesManager, node, application, priority,
|
||||||
ActivityDiagnosticConstant.PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST);
|
ActivityDiagnosticConstant.PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST);
|
||||||
|
@ -113,10 +112,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Required resource
|
// Required resource
|
||||||
Resource required = anyRequest.getCapability();
|
Resource required = offswitchPendingAsk.getPerAllocationResource();
|
||||||
|
|
||||||
// Do we need containers at this 'priority'?
|
// Do we need containers at this 'priority'?
|
||||||
if (application.getTotalRequiredResources(schedulerKey) <= 0) {
|
if (application.getOutstandingAsksCount(schedulerKey) <= 0) {
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
activitiesManager, node, application, priority,
|
activitiesManager, node, application, priority,
|
||||||
ActivityDiagnosticConstant.APPLICATION_PRIORITY_DO_NOT_NEED_RESOURCE);
|
ActivityDiagnosticConstant.APPLICATION_PRIORITY_DO_NOT_NEED_RESOURCE);
|
||||||
|
@ -141,11 +140,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Is the node-label-expression of this offswitch resource request
|
// Is the nodePartition of pending request matches the node's partition
|
||||||
// matches the node's label?
|
|
||||||
// If not match, jump to next priority.
|
// If not match, jump to next priority.
|
||||||
if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
|
if (!appInfo.acceptNodePartition(schedulerKey, node.getPartition(),
|
||||||
anyRequest.getNodeLabelExpression(), ps.getPartition(),
|
|
||||||
schedulingMode)) {
|
schedulingMode)) {
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
activitiesManager, node, application, priority,
|
activitiesManager, node, application, priority,
|
||||||
|
@ -182,8 +179,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
// This is to make sure non-partitioned-resource-request will prefer
|
// This is to make sure non-partitioned-resource-request will prefer
|
||||||
// to be allocated to non-partitioned nodes
|
// to be allocated to non-partitioned nodes
|
||||||
int missedNonPartitionedRequestSchedulingOpportunity = 0;
|
int missedNonPartitionedRequestSchedulingOpportunity = 0;
|
||||||
if (anyRequest.getNodeLabelExpression()
|
// Only do this when request associated with given scheduler key accepts
|
||||||
.equals(RMNodeLabelsManager.NO_LABEL)) {
|
// NO_LABEL under RESPECT_EXCLUSIVITY mode
|
||||||
|
if (StringUtils.equals(RMNodeLabelsManager.NO_LABEL,
|
||||||
|
appInfo.getSchedulingPlacementSet(schedulerKey)
|
||||||
|
.getPrimaryRequestedNodePartition())) {
|
||||||
missedNonPartitionedRequestSchedulingOpportunity =
|
missedNonPartitionedRequestSchedulingOpportunity =
|
||||||
application.addMissedNonPartitionedRequestSchedulingOpportunity(
|
application.addMissedNonPartitionedRequestSchedulingOpportunity(
|
||||||
schedulerKey);
|
schedulerKey);
|
||||||
|
@ -264,8 +264,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
public float getLocalityWaitFactor(
|
public float getLocalityWaitFactor(
|
||||||
SchedulerRequestKey schedulerKey, int clusterNodes) {
|
SchedulerRequestKey schedulerKey, int clusterNodes) {
|
||||||
// Estimate: Required unique resources (i.e. hosts + racks)
|
// Estimate: Required unique resources (i.e. hosts + racks)
|
||||||
int requiredResources =
|
int requiredResources = Math.max(
|
||||||
Math.max(application.getResourceRequests(schedulerKey).size() - 1, 0);
|
application.getSchedulingPlacementSet(schedulerKey)
|
||||||
|
.getUniqueLocationAsks() - 1, 0);
|
||||||
|
|
||||||
// waitFactor can't be more than '1'
|
// waitFactor can't be more than '1'
|
||||||
// i.e. no point skipping more than clustersize opportunities
|
// i.e. no point skipping more than clustersize opportunities
|
||||||
|
@ -287,11 +288,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
// 'Delay' off-switch
|
// 'Delay' off-switch
|
||||||
ResourceRequest offSwitchRequest =
|
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
|
||||||
long missedOpportunities =
|
long missedOpportunities =
|
||||||
application.getSchedulingOpportunities(schedulerKey);
|
application.getSchedulingOpportunities(schedulerKey);
|
||||||
long requiredContainers = offSwitchRequest.getNumContainers();
|
long requiredContainers = application.getOutstandingAsksCount(
|
||||||
|
schedulerKey);
|
||||||
|
|
||||||
float localityWaitFactor =
|
float localityWaitFactor =
|
||||||
getLocalityWaitFactor(schedulerKey, rmContext.getScheduler()
|
getLocalityWaitFactor(schedulerKey, rmContext.getScheduler()
|
||||||
|
@ -304,9 +304,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if we need containers on this rack
|
// Check if we need containers on this rack
|
||||||
ResourceRequest rackLocalRequest =
|
if (application.getOutstandingAsksCount(schedulerKey, node.getRackName())
|
||||||
application.getResourceRequest(schedulerKey, node.getRackName());
|
<= 0) {
|
||||||
if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -321,24 +320,21 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
// Check if we need containers on this host
|
// Check if we need containers on this host
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
// Now check if we need containers on this host...
|
// Now check if we need containers on this host...
|
||||||
ResourceRequest nodeLocalRequest =
|
return application.getOutstandingAsksCount(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, node.getNodeName());
|
node.getNodeName()) > 0;
|
||||||
if (nodeLocalRequest != null) {
|
|
||||||
return nodeLocalRequest.getNumContainers() > 0;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerAllocation assignNodeLocalContainers(
|
private ContainerAllocation assignNodeLocalContainers(
|
||||||
Resource clusterResource, ResourceRequest nodeLocalResourceRequest,
|
Resource clusterResource, PendingAsk nodeLocalAsk,
|
||||||
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
||||||
ResourceLimits currentResoureLimits) {
|
ResourceLimits currentResoureLimits) {
|
||||||
if (canAssign(schedulerKey, node, NodeType.NODE_LOCAL, reservedContainer)) {
|
if (canAssign(schedulerKey, node, NodeType.NODE_LOCAL, reservedContainer)) {
|
||||||
return assignContainer(clusterResource, node, schedulerKey,
|
return assignContainer(clusterResource, node, schedulerKey,
|
||||||
nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer,
|
nodeLocalAsk, NodeType.NODE_LOCAL, reservedContainer,
|
||||||
schedulingMode, currentResoureLimits);
|
schedulingMode, currentResoureLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -350,13 +346,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerAllocation assignRackLocalContainers(
|
private ContainerAllocation assignRackLocalContainers(
|
||||||
Resource clusterResource, ResourceRequest rackLocalResourceRequest,
|
Resource clusterResource, PendingAsk rackLocalAsk,
|
||||||
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
||||||
ResourceLimits currentResoureLimits) {
|
ResourceLimits currentResoureLimits) {
|
||||||
if (canAssign(schedulerKey, node, NodeType.RACK_LOCAL, reservedContainer)) {
|
if (canAssign(schedulerKey, node, NodeType.RACK_LOCAL, reservedContainer)) {
|
||||||
return assignContainer(clusterResource, node, schedulerKey,
|
return assignContainer(clusterResource, node, schedulerKey,
|
||||||
rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer,
|
rackLocalAsk, NodeType.RACK_LOCAL, reservedContainer,
|
||||||
schedulingMode, currentResoureLimits);
|
schedulingMode, currentResoureLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -368,13 +364,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerAllocation assignOffSwitchContainers(
|
private ContainerAllocation assignOffSwitchContainers(
|
||||||
Resource clusterResource, ResourceRequest offSwitchResourceRequest,
|
Resource clusterResource, PendingAsk offSwitchAsk,
|
||||||
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
RMContainer reservedContainer, SchedulingMode schedulingMode,
|
||||||
ResourceLimits currentResoureLimits) {
|
ResourceLimits currentResoureLimits) {
|
||||||
if (canAssign(schedulerKey, node, NodeType.OFF_SWITCH, reservedContainer)) {
|
if (canAssign(schedulerKey, node, NodeType.OFF_SWITCH, reservedContainer)) {
|
||||||
return assignContainer(clusterResource, node, schedulerKey,
|
return assignContainer(clusterResource, node, schedulerKey,
|
||||||
offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer,
|
offSwitchAsk, NodeType.OFF_SWITCH, reservedContainer,
|
||||||
schedulingMode, currentResoureLimits);
|
schedulingMode, currentResoureLimits);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -396,12 +392,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
NodeType requestLocalityType = null;
|
NodeType requestLocalityType = null;
|
||||||
|
|
||||||
// Data-local
|
// Data-local
|
||||||
ResourceRequest nodeLocalResourceRequest =
|
PendingAsk nodeLocalAsk =
|
||||||
application.getResourceRequest(schedulerKey, node.getNodeName());
|
application.getPendingAsk(schedulerKey, node.getNodeName());
|
||||||
if (nodeLocalResourceRequest != null) {
|
if (nodeLocalAsk.getCount() > 0) {
|
||||||
requestLocalityType = NodeType.NODE_LOCAL;
|
requestLocalityType = NodeType.NODE_LOCAL;
|
||||||
allocation =
|
allocation =
|
||||||
assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
|
assignNodeLocalContainers(clusterResource, nodeLocalAsk,
|
||||||
node, schedulerKey, reservedContainer, schedulingMode,
|
node, schedulerKey, reservedContainer, schedulingMode,
|
||||||
currentResoureLimits);
|
currentResoureLimits);
|
||||||
if (Resources.greaterThan(rc, clusterResource,
|
if (Resources.greaterThan(rc, clusterResource,
|
||||||
|
@ -412,10 +408,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Rack-local
|
// Rack-local
|
||||||
ResourceRequest rackLocalResourceRequest =
|
PendingAsk rackLocalAsk =
|
||||||
application.getResourceRequest(schedulerKey, node.getRackName());
|
application.getPendingAsk(schedulerKey, node.getRackName());
|
||||||
if (rackLocalResourceRequest != null) {
|
if (rackLocalAsk.getCount() > 0) {
|
||||||
if (!rackLocalResourceRequest.getRelaxLocality()) {
|
if (!appInfo.canDelayTo(schedulerKey, node.getRackName())) {
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
activitiesManager, node, application, priority,
|
activitiesManager, node, application, priority,
|
||||||
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
||||||
|
@ -427,7 +423,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
requestLocalityType;
|
requestLocalityType;
|
||||||
|
|
||||||
allocation =
|
allocation =
|
||||||
assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
|
assignRackLocalContainers(clusterResource, rackLocalAsk,
|
||||||
node, schedulerKey, reservedContainer, schedulingMode,
|
node, schedulerKey, reservedContainer, schedulingMode,
|
||||||
currentResoureLimits);
|
currentResoureLimits);
|
||||||
if (Resources.greaterThan(rc, clusterResource,
|
if (Resources.greaterThan(rc, clusterResource,
|
||||||
|
@ -438,10 +434,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Off-switch
|
// Off-switch
|
||||||
ResourceRequest offSwitchResourceRequest =
|
PendingAsk offSwitchAsk =
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
application.getPendingAsk(schedulerKey, ResourceRequest.ANY);
|
||||||
if (offSwitchResourceRequest != null) {
|
if (offSwitchAsk.getCount() > 0) {
|
||||||
if (!offSwitchResourceRequest.getRelaxLocality()) {
|
if (!appInfo.canDelayTo(schedulerKey, ResourceRequest.ANY)) {
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
activitiesManager, node, application, priority,
|
activitiesManager, node, application, priority,
|
||||||
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
||||||
|
@ -453,7 +449,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
requestLocalityType;
|
requestLocalityType;
|
||||||
|
|
||||||
allocation =
|
allocation =
|
||||||
assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
|
assignOffSwitchContainers(clusterResource, offSwitchAsk,
|
||||||
node, schedulerKey, reservedContainer, schedulingMode,
|
node, schedulerKey, reservedContainer, schedulingMode,
|
||||||
currentResoureLimits);
|
currentResoureLimits);
|
||||||
|
|
||||||
|
@ -474,41 +470,25 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
|
|
||||||
private ContainerAllocation assignContainer(Resource clusterResource,
|
private ContainerAllocation assignContainer(Resource clusterResource,
|
||||||
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest request, NodeType type, RMContainer rmContainer,
|
PendingAsk pendingAsk, NodeType type, RMContainer rmContainer,
|
||||||
SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) {
|
||||||
Priority priority = schedulerKey.getPriority();
|
Priority priority = schedulerKey.getPriority();
|
||||||
lastResourceRequest = request;
|
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("assignContainers: node=" + node.getNodeName()
|
LOG.debug("assignContainers: node=" + node.getNodeName()
|
||||||
+ " application=" + application.getApplicationId()
|
+ " application=" + application.getApplicationId()
|
||||||
+ " priority=" + schedulerKey.getPriority()
|
+ " priority=" + schedulerKey.getPriority()
|
||||||
+ " request=" + request + " type=" + type);
|
+ " pendingAsk=" + pendingAsk + " type=" + type);
|
||||||
}
|
}
|
||||||
|
|
||||||
// check if the resource request can access the label
|
Resource capability = pendingAsk.getPerAllocationResource();
|
||||||
if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
|
|
||||||
request.getNodeLabelExpression(), node.getPartition(),
|
|
||||||
schedulingMode)) {
|
|
||||||
// this is a reserved container, but we cannot allocate it now according
|
|
||||||
// to label not match. This can be caused by node label changed
|
|
||||||
// We should un-reserve this container.
|
|
||||||
ActivitiesLogger.APP.recordAppActivityWithoutAllocation(activitiesManager,
|
|
||||||
node, application, priority,
|
|
||||||
ActivityDiagnosticConstant.REQUEST_CAN_NOT_ACCESS_NODE_LABEL,
|
|
||||||
ActivityState.REJECTED);
|
|
||||||
return new ContainerAllocation(rmContainer, null,
|
|
||||||
AllocationState.LOCALITY_SKIPPED);
|
|
||||||
}
|
|
||||||
|
|
||||||
Resource capability = request.getCapability();
|
|
||||||
Resource available = node.getUnallocatedResource();
|
Resource available = node.getUnallocatedResource();
|
||||||
Resource totalResource = node.getTotalResource();
|
Resource totalResource = node.getTotalResource();
|
||||||
|
|
||||||
if (!Resources.lessThanOrEqual(rc, clusterResource,
|
if (!Resources.lessThanOrEqual(rc, clusterResource,
|
||||||
capability, totalResource)) {
|
capability, totalResource)) {
|
||||||
LOG.warn("Node : " + node.getNodeID()
|
LOG.warn("Node : " + node.getNodeID()
|
||||||
+ " does not have sufficient resource for request : " + request
|
+ " does not have sufficient resource for ask : " + pendingAsk
|
||||||
+ " node total capability : " + node.getTotalResource());
|
+ " node total capability : " + node.getTotalResource());
|
||||||
// Skip this locality request
|
// Skip this locality request
|
||||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||||
|
@ -600,9 +580,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ContainerAllocation result =
|
ContainerAllocation result = new ContainerAllocation(unreservedContainer,
|
||||||
new ContainerAllocation(unreservedContainer, request.getCapability(),
|
pendingAsk.getPerAllocationResource(), AllocationState.ALLOCATED);
|
||||||
AllocationState.ALLOCATED);
|
|
||||||
result.containerNodeType = type;
|
result.containerNodeType = type;
|
||||||
result.setToKillContainers(toKillContainers);
|
result.setToKillContainers(toKillContainers);
|
||||||
return result;
|
return result;
|
||||||
|
@ -626,9 +605,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ContainerAllocation result =
|
ContainerAllocation result = new ContainerAllocation(null,
|
||||||
new ContainerAllocation(null, request.getCapability(),
|
pendingAsk.getPerAllocationResource(), AllocationState.RESERVED);
|
||||||
AllocationState.RESERVED);
|
|
||||||
result.containerNodeType = type;
|
result.containerNodeType = type;
|
||||||
result.setToKillContainers(null);
|
result.setToKillContainers(null);
|
||||||
return result;
|
return result;
|
||||||
|
@ -644,7 +622,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
boolean shouldAllocOrReserveNewContainer(
|
boolean shouldAllocOrReserveNewContainer(
|
||||||
SchedulerRequestKey schedulerKey, Resource required) {
|
SchedulerRequestKey schedulerKey, Resource required) {
|
||||||
int requiredContainers =
|
int requiredContainers =
|
||||||
application.getTotalRequiredResources(schedulerKey);
|
application.getOutstandingAsksCount(schedulerKey);
|
||||||
int reservedContainers = application.getNumReservedContainers(schedulerKey);
|
int reservedContainers = application.getNumReservedContainers(schedulerKey);
|
||||||
int starvation = 0;
|
int starvation = 0;
|
||||||
if (reservedContainers > 0) {
|
if (reservedContainers > 0) {
|
||||||
|
@ -699,7 +677,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
SchedulerRequestKey schedulerKey, Container container) {
|
SchedulerRequestKey schedulerKey, Container container) {
|
||||||
// Inform the application
|
// Inform the application
|
||||||
RMContainer allocatedContainer = application.allocate(node, schedulerKey,
|
RMContainer allocatedContainer = application.allocate(node, schedulerKey,
|
||||||
lastResourceRequest, container);
|
container);
|
||||||
|
|
||||||
allocationResult.updatedContainer = allocatedContainer;
|
allocationResult.updatedContainer = allocatedContainer;
|
||||||
|
|
||||||
|
@ -803,7 +781,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// pre-check when allocating reserved container
|
// pre-check when allocating reserved container
|
||||||
if (application.getTotalRequiredResources(schedulerKey) == 0) {
|
if (application.getOutstandingAsksCount(schedulerKey) == 0) {
|
||||||
// Release
|
// Release
|
||||||
return new ContainerAllocation(reservedContainer, null,
|
return new ContainerAllocation(reservedContainer, null,
|
||||||
AllocationState.QUEUE_SKIPPED);
|
AllocationState.QUEUE_SKIPPED);
|
||||||
|
|
|
@ -0,0 +1,57 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link PendingAsk} is the class to include minimal information of how much
|
||||||
|
* resource to ask under constraints (e.g. on one host / rack / node-attributes)
|
||||||
|
* , etc.
|
||||||
|
*/
|
||||||
|
public class PendingAsk {
|
||||||
|
private final Resource perAllocationResource;
|
||||||
|
private final int count;
|
||||||
|
public final static PendingAsk ZERO = new PendingAsk(Resources.none(), 0);
|
||||||
|
|
||||||
|
public PendingAsk(Resource res, int num) {
|
||||||
|
this.perAllocationResource = res;
|
||||||
|
this.count = num;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Resource getPerAllocationResource() {
|
||||||
|
return perAllocationResource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getCount() {
|
||||||
|
return count;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("<per-allocation-resource=");
|
||||||
|
sb.append(getPerAllocationResource());
|
||||||
|
sb.append(",repeat=");
|
||||||
|
sb.append(getCount());
|
||||||
|
sb.append(">");
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
|
@ -70,6 +70,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
|
@ -206,8 +208,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainer allocate(FiCaSchedulerNode node,
|
public RMContainer allocate(FiCaSchedulerNode node,
|
||||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
SchedulerRequestKey schedulerKey, Container container) {
|
||||||
Container container) {
|
|
||||||
try {
|
try {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
|
|
||||||
|
@ -217,7 +218,16 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
||||||
|
|
||||||
// Required sanity check - AM can call 'allocate' to update resource
|
// Required sanity check - AM can call 'allocate' to update resource
|
||||||
// request without locking the scheduler, hence we need to check
|
// request without locking the scheduler, hence we need to check
|
||||||
if (getTotalRequiredResources(schedulerKey) <= 0) {
|
if (getOutstandingAsksCount(schedulerKey) <= 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
SchedulingPlacementSet<FiCaSchedulerNode> ps =
|
||||||
|
appSchedulingInfo.getSchedulingPlacementSet(schedulerKey);
|
||||||
|
if (null == ps) {
|
||||||
|
LOG.warn("Failed to get " + SchedulingPlacementSet.class.getName()
|
||||||
|
+ " for application=" + getApplicationId() + " schedulerRequestKey="
|
||||||
|
+ schedulerKey);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -225,7 +235,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, schedulerKey,
|
RMContainer rmContainer = new RMContainerImpl(container, schedulerKey,
|
||||||
this.getApplicationAttemptId(), node.getNodeID(),
|
this.getApplicationAttemptId(), node.getNodeID(),
|
||||||
appSchedulingInfo.getUser(), this.rmContext,
|
appSchedulingInfo.getUser(), this.rmContext,
|
||||||
request.getNodeLabelExpression());
|
ps.getPrimaryRequestedNodePartition());
|
||||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||||
|
|
||||||
// FIXME, should set when confirmed
|
// FIXME, should set when confirmed
|
||||||
|
@ -694,21 +704,36 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized Map<String, Resource> getTotalPendingRequestsPerPartition() {
|
public Map<String, Resource> getTotalPendingRequestsPerPartition() {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
|
||||||
Map<String, Resource> ret = new HashMap<String, Resource>();
|
Map<String, Resource> ret = new HashMap<>();
|
||||||
Resource res = null;
|
for (SchedulerRequestKey schedulerKey : appSchedulingInfo
|
||||||
for (SchedulerRequestKey key : appSchedulingInfo.getSchedulerKeys()) {
|
.getSchedulerKeys()) {
|
||||||
ResourceRequest rr = appSchedulingInfo.getResourceRequest(key, "*");
|
SchedulingPlacementSet<FiCaSchedulerNode> ps =
|
||||||
if ((res = ret.get(rr.getNodeLabelExpression())) == null) {
|
appSchedulingInfo.getSchedulingPlacementSet(schedulerKey);
|
||||||
res = Resources.createResource(0, 0);
|
|
||||||
ret.put(rr.getNodeLabelExpression(), res);
|
String nodePartition = ps.getPrimaryRequestedNodePartition();
|
||||||
|
Resource res = ret.get(nodePartition);
|
||||||
|
if (null == res) {
|
||||||
|
res = Resources.createResource(0);
|
||||||
|
ret.put(nodePartition, res);
|
||||||
|
}
|
||||||
|
|
||||||
|
PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY);
|
||||||
|
if (ask.getCount() > 0) {
|
||||||
|
Resources.addTo(res, Resources
|
||||||
|
.multiply(ask.getPerAllocationResource(),
|
||||||
|
ask.getCount()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Resources.addTo(res,
|
return ret;
|
||||||
Resources.multiply(rr.getCapability(), rr.getNumContainers()));
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
}
|
}
|
||||||
return ret;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void markContainerForPreemption(ContainerId cont) {
|
public void markContainerForPreemption(ContainerId cont) {
|
||||||
|
|
|
@ -18,16 +18,6 @@
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
import java.text.DecimalFormat;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -56,11 +46,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
|
import java.text.DecimalFormat;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents an application attempt from the viewpoint of the Fair Scheduler.
|
* Represents an application attempt from the viewpoint of the Fair Scheduler.
|
||||||
*/
|
*/
|
||||||
|
@ -416,7 +417,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainer allocate(NodeType type, FSSchedulerNode node,
|
public RMContainer allocate(NodeType type, FSSchedulerNode node,
|
||||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
SchedulerRequestKey schedulerKey, PendingAsk pendingAsk,
|
||||||
Container reservedContainer) {
|
Container reservedContainer) {
|
||||||
RMContainer rmContainer;
|
RMContainer rmContainer;
|
||||||
Container container;
|
Container container;
|
||||||
|
@ -437,13 +438,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
|
|
||||||
// Required sanity check - AM can call 'allocate' to update resource
|
// Required sanity check - AM can call 'allocate' to update resource
|
||||||
// request without locking the scheduler, hence we need to check
|
// request without locking the scheduler, hence we need to check
|
||||||
if (getTotalRequiredResources(schedulerKey) <= 0) {
|
if (getOutstandingAsksCount(schedulerKey) <= 0) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
container = reservedContainer;
|
container = reservedContainer;
|
||||||
if (container == null) {
|
if (container == null) {
|
||||||
container = createContainer(node, request.getCapability(),
|
container = createContainer(node, pendingAsk.getPerAllocationResource(),
|
||||||
schedulerKey);
|
schedulerKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -459,7 +460,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
|
|
||||||
// Update consumption and track allocations
|
// Update consumption and track allocations
|
||||||
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
||||||
type, node, schedulerKey, request, container);
|
type, node, schedulerKey, container);
|
||||||
this.attemptResourceUsage.incUsed(container.getResource());
|
this.attemptResourceUsage.incUsed(container.getResource());
|
||||||
|
|
||||||
// Update resource requests related to "request" and store in RMContainer
|
// Update resource requests related to "request" and store in RMContainer
|
||||||
|
@ -632,7 +633,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
* in {@link FSSchedulerNode}..
|
* in {@link FSSchedulerNode}..
|
||||||
* return whether reservation was possible with the current threshold limits
|
* return whether reservation was possible with the current threshold limits
|
||||||
*/
|
*/
|
||||||
private boolean reserve(ResourceRequest request, FSSchedulerNode node,
|
private boolean reserve(Resource perAllocationResource, FSSchedulerNode node,
|
||||||
Container reservedContainer, NodeType type,
|
Container reservedContainer, NodeType type,
|
||||||
SchedulerRequestKey schedulerKey) {
|
SchedulerRequestKey schedulerKey) {
|
||||||
|
|
||||||
|
@ -641,7 +642,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
" app_id=" + getApplicationId());
|
" app_id=" + getApplicationId());
|
||||||
if (reservedContainer == null) {
|
if (reservedContainer == null) {
|
||||||
reservedContainer =
|
reservedContainer =
|
||||||
createContainer(node, request.getCapability(),
|
createContainer(node, perAllocationResource,
|
||||||
schedulerKey);
|
schedulerKey);
|
||||||
getMetrics().reserveResource(getUser(),
|
getMetrics().reserveResource(getUser(),
|
||||||
reservedContainer.getResource());
|
reservedContainer.getResource());
|
||||||
|
@ -763,8 +764,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
*
|
*
|
||||||
* @param node
|
* @param node
|
||||||
* The node to try placing the container on.
|
* The node to try placing the container on.
|
||||||
* @param request
|
* @param pendingAsk
|
||||||
* The ResourceRequest we're trying to satisfy.
|
* The {@link PendingAsk} we're trying to satisfy.
|
||||||
* @param type
|
* @param type
|
||||||
* The locality of the assignment.
|
* The locality of the assignment.
|
||||||
* @param reserved
|
* @param reserved
|
||||||
|
@ -776,11 +777,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
* made, returns an empty resource.
|
* made, returns an empty resource.
|
||||||
*/
|
*/
|
||||||
private Resource assignContainer(
|
private Resource assignContainer(
|
||||||
FSSchedulerNode node, ResourceRequest request, NodeType type,
|
FSSchedulerNode node, PendingAsk pendingAsk, NodeType type,
|
||||||
boolean reserved, SchedulerRequestKey schedulerKey) {
|
boolean reserved, SchedulerRequestKey schedulerKey) {
|
||||||
|
|
||||||
// How much does this request need?
|
// How much does this request need?
|
||||||
Resource capability = request.getCapability();
|
Resource capability = pendingAsk.getPerAllocationResource();
|
||||||
|
|
||||||
// How much does the node have?
|
// How much does the node have?
|
||||||
Resource available = node.getUnallocatedResource();
|
Resource available = node.getUnallocatedResource();
|
||||||
|
@ -794,7 +795,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
if (Resources.fitsIn(capability, available)) {
|
if (Resources.fitsIn(capability, available)) {
|
||||||
// Inform the application of the new container for this request
|
// Inform the application of the new container for this request
|
||||||
RMContainer allocatedContainer =
|
RMContainer allocatedContainer =
|
||||||
allocate(type, node, schedulerKey, request,
|
allocate(type, node, schedulerKey, pendingAsk,
|
||||||
reservedContainer);
|
reservedContainer);
|
||||||
if (allocatedContainer == null) {
|
if (allocatedContainer == null) {
|
||||||
// Did the application need this resource?
|
// Did the application need this resource?
|
||||||
|
@ -825,8 +826,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
}
|
}
|
||||||
|
|
||||||
// The desired container won't fit here, so reserve
|
// The desired container won't fit here, so reserve
|
||||||
if (isReservable(capability) &&
|
if (isReservable(capability) && reserve(
|
||||||
reserve(request, node, reservedContainer, type, schedulerKey)) {
|
pendingAsk.getPerAllocationResource(), node, reservedContainer, type,
|
||||||
|
schedulerKey)) {
|
||||||
if (isWaitingForAMContainer()) {
|
if (isWaitingForAMContainer()) {
|
||||||
updateAMDiagnosticMsg(capability,
|
updateAMDiagnosticMsg(capability,
|
||||||
" exceed the available resources of the node and the request is"
|
" exceed the available resources of the node and the request is"
|
||||||
|
@ -841,7 +843,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
}
|
}
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Couldn't creating reservation for " +
|
LOG.debug("Couldn't creating reservation for " +
|
||||||
getName() + ",at priority " + request.getPriority());
|
getName() + ",at priority " + schedulerKey.getPriority());
|
||||||
}
|
}
|
||||||
return Resources.none();
|
return Resources.none();
|
||||||
}
|
}
|
||||||
|
@ -852,19 +854,16 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
getQueue().getPolicy().getResourceCalculator(), capacity);
|
getQueue().getPolicy().getResourceCalculator(), capacity);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean hasNodeOrRackLocalRequests(SchedulerRequestKey schedulerKey) {
|
|
||||||
return getResourceRequests(schedulerKey).size() > 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Whether the AM container for this app is over maxAMShare limit.
|
* Whether the AM container for this app is over maxAMShare limit.
|
||||||
*/
|
*/
|
||||||
private boolean isOverAMShareLimit() {
|
private boolean isOverAMShareLimit() {
|
||||||
// Check the AM resource usage for the leaf queue
|
// Check the AM resource usage for the leaf queue
|
||||||
if (!isAmRunning() && !getUnmanagedAM()) {
|
if (!isAmRunning() && !getUnmanagedAM()) {
|
||||||
List<ResourceRequest> ask = appSchedulingInfo.getAllResourceRequests();
|
// Return true if we have not ask, or queue is not be able to run app's AM
|
||||||
if (ask.isEmpty() || !getQueue().canRunAppAM(
|
PendingAsk ask = appSchedulingInfo.getNextPendingAsk();
|
||||||
ask.get(0).getCapability())) {
|
if (ask.getCount() == 0 || !getQueue().canRunAppAM(
|
||||||
|
ask.getPerAllocationResource())) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -886,6 +885,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
// (not scheduled) in order to promote better locality.
|
// (not scheduled) in order to promote better locality.
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
|
||||||
|
// TODO (wandga): All logics in this method should be added to
|
||||||
|
// SchedulerPlacement#canDelayTo which is independent from scheduler.
|
||||||
|
// Scheduler can choose to use various/pluggable delay-scheduling
|
||||||
|
// implementation.
|
||||||
for (SchedulerRequestKey schedulerKey : keysToTry) {
|
for (SchedulerRequestKey schedulerKey : keysToTry) {
|
||||||
// Skip it for reserved container, since
|
// Skip it for reserved container, since
|
||||||
// we already check it in isValidReservation.
|
// we already check it in isValidReservation.
|
||||||
|
@ -895,14 +899,16 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
|
|
||||||
addSchedulingOpportunity(schedulerKey);
|
addSchedulingOpportunity(schedulerKey);
|
||||||
|
|
||||||
ResourceRequest rackLocalRequest = getResourceRequest(schedulerKey,
|
PendingAsk rackLocalPendingAsk = getPendingAsk(schedulerKey,
|
||||||
node.getRackName());
|
node.getRackName());
|
||||||
ResourceRequest localRequest = getResourceRequest(schedulerKey,
|
PendingAsk nodeLocalPendingAsk = getPendingAsk(schedulerKey,
|
||||||
node.getNodeName());
|
node.getNodeName());
|
||||||
|
|
||||||
if (localRequest != null && !localRequest.getRelaxLocality()) {
|
if (nodeLocalPendingAsk.getCount() > 0
|
||||||
|
&& !appSchedulingInfo.canDelayTo(schedulerKey,
|
||||||
|
node.getNodeName())) {
|
||||||
LOG.warn("Relax locality off is not supported on local request: "
|
LOG.warn("Relax locality off is not supported on local request: "
|
||||||
+ localRequest);
|
+ nodeLocalPendingAsk);
|
||||||
}
|
}
|
||||||
|
|
||||||
NodeType allowedLocality;
|
NodeType allowedLocality;
|
||||||
|
@ -918,23 +924,23 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
scheduler.getRackLocalityThreshold());
|
scheduler.getRackLocalityThreshold());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
|
if (rackLocalPendingAsk.getCount() > 0
|
||||||
&& localRequest != null && localRequest.getNumContainers() != 0) {
|
&& nodeLocalPendingAsk.getCount() > 0) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Assign container on " + node.getNodeName()
|
LOG.trace("Assign container on " + node.getNodeName()
|
||||||
+ " node, assignType: NODE_LOCAL" + ", allowedLocality: "
|
+ " node, assignType: NODE_LOCAL" + ", allowedLocality: "
|
||||||
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
||||||
+ ", app attempt id: " + this.attemptId);
|
+ ", app attempt id: " + this.attemptId);
|
||||||
}
|
}
|
||||||
return assignContainer(node, localRequest, NodeType.NODE_LOCAL,
|
return assignContainer(node, nodeLocalPendingAsk, NodeType.NODE_LOCAL,
|
||||||
reserved, schedulerKey);
|
reserved, schedulerKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) {
|
if (!appSchedulingInfo.canDelayTo(schedulerKey, node.getRackName())) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
|
if (rackLocalPendingAsk.getCount() > 0
|
||||||
&& (allowedLocality.equals(NodeType.RACK_LOCAL) || allowedLocality
|
&& (allowedLocality.equals(NodeType.RACK_LOCAL) || allowedLocality
|
||||||
.equals(NodeType.OFF_SWITCH))) {
|
.equals(NodeType.OFF_SWITCH))) {
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
|
@ -943,27 +949,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
||||||
+ ", app attempt id: " + this.attemptId);
|
+ ", app attempt id: " + this.attemptId);
|
||||||
}
|
}
|
||||||
return assignContainer(node, rackLocalRequest, NodeType.RACK_LOCAL,
|
return assignContainer(node, rackLocalPendingAsk, NodeType.RACK_LOCAL,
|
||||||
reserved, schedulerKey);
|
reserved, schedulerKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
ResourceRequest offSwitchRequest = getResourceRequest(schedulerKey,
|
PendingAsk offswitchAsk = getPendingAsk(schedulerKey,
|
||||||
ResourceRequest.ANY);
|
ResourceRequest.ANY);
|
||||||
if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) {
|
if (!appSchedulingInfo.canDelayTo(schedulerKey, ResourceRequest.ANY)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (offSwitchRequest != null
|
if (offswitchAsk.getCount() > 0) {
|
||||||
&& offSwitchRequest.getNumContainers() != 0) {
|
if (getSchedulingPlacementSet(schedulerKey).getUniqueLocationAsks()
|
||||||
if (!hasNodeOrRackLocalRequests(schedulerKey) || allowedLocality
|
<= 1 || allowedLocality.equals(NodeType.OFF_SWITCH)) {
|
||||||
.equals(NodeType.OFF_SWITCH)) {
|
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Assign container on " + node.getNodeName()
|
LOG.trace("Assign container on " + node.getNodeName()
|
||||||
+ " node, assignType: OFF_SWITCH" + ", allowedLocality: "
|
+ " node, assignType: OFF_SWITCH" + ", allowedLocality: "
|
||||||
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
+ allowedLocality + ", priority: " + schedulerKey.getPriority()
|
||||||
+ ", app attempt id: " + this.attemptId);
|
+ ", app attempt id: " + this.attemptId);
|
||||||
}
|
}
|
||||||
return assignContainer(node, offSwitchRequest, NodeType.OFF_SWITCH,
|
return assignContainer(node, offswitchAsk, NodeType.OFF_SWITCH,
|
||||||
reserved, schedulerKey);
|
reserved, schedulerKey);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -988,29 +993,35 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
*/
|
*/
|
||||||
private boolean hasContainerForNode(SchedulerRequestKey key,
|
private boolean hasContainerForNode(SchedulerRequestKey key,
|
||||||
FSSchedulerNode node) {
|
FSSchedulerNode node) {
|
||||||
ResourceRequest anyRequest = getResourceRequest(key, ResourceRequest.ANY);
|
PendingAsk offswitchAsk = getPendingAsk(key, ResourceRequest.ANY);
|
||||||
ResourceRequest rackRequest = getResourceRequest(key, node.getRackName());
|
Resource resource = offswitchAsk.getPerAllocationResource();
|
||||||
ResourceRequest nodeRequest = getResourceRequest(key, node.getNodeName());
|
boolean hasRequestForOffswitch =
|
||||||
|
offswitchAsk.getCount() > 0;
|
||||||
|
boolean hasRequestForRack = getOutstandingAsksCount(key,
|
||||||
|
node.getRackName()) > 0;
|
||||||
|
boolean hasRequestForNode = getOutstandingAsksCount(key,
|
||||||
|
node.getNodeName()) > 0;
|
||||||
|
|
||||||
boolean ret = true;
|
boolean ret = true;
|
||||||
if (!(// There must be outstanding requests at the given priority:
|
if (!(// There must be outstanding requests at the given priority:
|
||||||
anyRequest != null && anyRequest.getNumContainers() > 0 &&
|
hasRequestForOffswitch &&
|
||||||
// If locality relaxation is turned off at *-level, there must be a
|
// If locality relaxation is turned off at *-level, there must be a
|
||||||
// non-zero request for the node's rack:
|
// non-zero request for the node's rack:
|
||||||
(anyRequest.getRelaxLocality() ||
|
(appSchedulingInfo.canDelayTo(key, ResourceRequest.ANY) ||
|
||||||
(rackRequest != null && rackRequest.getNumContainers() > 0)) &&
|
(hasRequestForRack)) &&
|
||||||
// If locality relaxation is turned off at rack-level, there must be a
|
// If locality relaxation is turned off at rack-level,
|
||||||
// non-zero request at the node:
|
// there must be a non-zero request at the node:
|
||||||
(rackRequest == null || rackRequest.getRelaxLocality() ||
|
(!hasRequestForRack || appSchedulingInfo.canDelayTo(key,
|
||||||
(nodeRequest != null && nodeRequest.getNumContainers() > 0)) &&
|
node.getRackName()) || (hasRequestForNode)) &&
|
||||||
// The requested container must be able to fit on the node:
|
// The requested container must be able to fit on the node:
|
||||||
Resources.lessThanOrEqual(RESOURCE_CALCULATOR, null,
|
Resources.lessThanOrEqual(RESOURCE_CALCULATOR, null,
|
||||||
anyRequest.getCapability(), node.getRMNode().getTotalCapability()))) {
|
resource,
|
||||||
|
node.getRMNode().getTotalCapability()))) {
|
||||||
ret = false;
|
ret = false;
|
||||||
} else if (!getQueue().fitsInMaxShare(anyRequest.getCapability())) {
|
} else if (!getQueue().fitsInMaxShare(resource)) {
|
||||||
// The requested container must fit in queue maximum share
|
// The requested container must fit in queue maximum share
|
||||||
if (isWaitingForAMContainer()) {
|
if (isWaitingForAMContainer()) {
|
||||||
updateAMDiagnosticMsg(anyRequest.getCapability(),
|
updateAMDiagnosticMsg(resource,
|
||||||
" exceeds current queue or its parents maximum resource allowed).");
|
" exceeds current queue or its parents maximum resource allowed).");
|
||||||
}
|
}
|
||||||
ret = false;
|
ret = false;
|
||||||
|
@ -1091,10 +1102,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
return this.fairshareStarvation;
|
return this.fairshareStarvation;
|
||||||
}
|
}
|
||||||
|
|
||||||
ResourceRequest getNextResourceRequest() {
|
|
||||||
return appSchedulingInfo.getNextResourceRequest();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper method that captures if this app is identified to be starved.
|
* Helper method that captures if this app is identified to be starved.
|
||||||
* @return true if the app is starved for fairshare, false otherwise
|
* @return true if the app is starved for fairshare, false otherwise
|
||||||
|
@ -1174,10 +1181,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
for (SchedulerRequestKey k : getSchedulerKeys()) {
|
for (SchedulerRequestKey k : getSchedulerKeys()) {
|
||||||
ResourceRequest r = getResourceRequest(k, ResourceRequest.ANY);
|
PendingAsk pendingAsk = getPendingAsk(k, ResourceRequest.ANY);
|
||||||
if (r != null) {
|
if (pendingAsk.getCount() > 0) {
|
||||||
Resources.multiplyAndAddTo(demand, r.getCapability(),
|
Resources.multiplyAndAddTo(demand,
|
||||||
r.getNumContainers());
|
pendingAsk.getPerAllocationResource(),
|
||||||
|
pendingAsk.getCount());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -1189,9 +1197,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
|
||||||
public Resource assignContainer(FSSchedulerNode node) {
|
public Resource assignContainer(FSSchedulerNode node) {
|
||||||
if (isOverAMShareLimit()) {
|
if (isOverAMShareLimit()) {
|
||||||
if (isWaitingForAMContainer()) {
|
if (isWaitingForAMContainer()) {
|
||||||
List<ResourceRequest> ask = appSchedulingInfo.getAllResourceRequests();
|
PendingAsk amAsk = appSchedulingInfo.getNextPendingAsk();
|
||||||
updateAMDiagnosticMsg(ask.get(0).getCapability(), " exceeds maximum "
|
updateAMDiagnosticMsg(amAsk.getPerAllocationResource(),
|
||||||
+ "AM resource allowed).");
|
" exceeds maximum AM resource allowed).");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -90,14 +92,17 @@ class FSPreemptionThread extends Thread {
|
||||||
List<RMContainer> containers = new ArrayList<>(); // return value
|
List<RMContainer> containers = new ArrayList<>(); // return value
|
||||||
|
|
||||||
// Find the nodes that match the next resource request
|
// Find the nodes that match the next resource request
|
||||||
ResourceRequest request = starvedApp.getNextResourceRequest();
|
SchedulingPlacementSet nextPs =
|
||||||
|
starvedApp.getAppSchedulingInfo().getFirstSchedulingPlacementSet();
|
||||||
|
PendingAsk firstPendingAsk = nextPs.getPendingAsk(ResourceRequest.ANY);
|
||||||
// TODO (KK): Should we check other resource requests if we can't match
|
// TODO (KK): Should we check other resource requests if we can't match
|
||||||
// the first one?
|
// the first one?
|
||||||
|
|
||||||
Resource requestCapability = request.getCapability();
|
Resource requestCapability = firstPendingAsk.getPerAllocationResource();
|
||||||
|
|
||||||
List<FSSchedulerNode> potentialNodes =
|
List<FSSchedulerNode> potentialNodes =
|
||||||
scheduler.getNodeTracker().getNodesByResourceName(
|
scheduler.getNodeTracker().getNodesByResourceName(
|
||||||
request.getResourceName());
|
nextPs.getAcceptedResouceNames().next().toString());
|
||||||
|
|
||||||
// From the potential nodes, pick a node that has enough containers
|
// From the potential nodes, pick a node that has enough containers
|
||||||
// from apps over their fairshare
|
// from apps over their fairshare
|
||||||
|
|
|
@ -51,8 +51,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
||||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
SchedulerRequestKey schedulerKey, Container container) {
|
||||||
Container container) {
|
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
|
||||||
|
@ -62,15 +61,14 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
|
||||||
|
|
||||||
// Required sanity check - AM can call 'allocate' to update resource
|
// Required sanity check - AM can call 'allocate' to update resource
|
||||||
// request without locking the scheduler, hence we need to check
|
// request without locking the scheduler, hence we need to check
|
||||||
if (getTotalRequiredResources(schedulerKey) <= 0) {
|
if (getOutstandingAsksCount(schedulerKey) <= 0) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create RMContainer
|
// Create RMContainer
|
||||||
RMContainer rmContainer = new RMContainerImpl(container,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
schedulerKey, this.getApplicationAttemptId(), node.getNodeID(),
|
schedulerKey, this.getApplicationAttemptId(), node.getNodeID(),
|
||||||
appSchedulingInfo.getUser(), this.rmContext,
|
appSchedulingInfo.getUser(), this.rmContext, node.getPartition());
|
||||||
request.getNodeLabelExpression());
|
|
||||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||||
|
|
||||||
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
||||||
|
@ -83,7 +81,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
|
||||||
|
|
||||||
// Update consumption and track allocations
|
// Update consumption and track allocations
|
||||||
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
||||||
type, node, schedulerKey, request, container);
|
type, node, schedulerKey, container);
|
||||||
|
|
||||||
attemptResourceUsage.incUsed(node.getPartition(),
|
attemptResourceUsage.incUsed(node.getPartition(),
|
||||||
container.getResource());
|
container.getResource());
|
||||||
|
|
|
@ -18,16 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
|
||||||
|
|
||||||
import java.io.IOException;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
||||||
|
@ -90,6 +81,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.server.utils.Lock;
|
import org.apache.hadoop.yarn.server.utils.Lock;
|
||||||
|
@ -97,7 +89,15 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
|
|
||||||
@LimitedPrivate("yarn")
|
@LimitedPrivate("yarn")
|
||||||
@Evolving
|
@Evolving
|
||||||
|
@ -545,35 +545,32 @@ public class FifoScheduler extends
|
||||||
|
|
||||||
private int getMaxAllocatableContainers(FifoAppAttempt application,
|
private int getMaxAllocatableContainers(FifoAppAttempt application,
|
||||||
SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) {
|
SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) {
|
||||||
int maxContainers = 0;
|
PendingAsk offswitchAsk = application.getPendingAsk(schedulerKey,
|
||||||
|
ResourceRequest.ANY);
|
||||||
ResourceRequest offSwitchRequest =
|
int maxContainers = offswitchAsk.getCount();
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
|
||||||
if (offSwitchRequest != null) {
|
|
||||||
maxContainers = offSwitchRequest.getNumContainers();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (type == NodeType.OFF_SWITCH) {
|
if (type == NodeType.OFF_SWITCH) {
|
||||||
return maxContainers;
|
return maxContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (type == NodeType.RACK_LOCAL) {
|
if (type == NodeType.RACK_LOCAL) {
|
||||||
ResourceRequest rackLocalRequest =
|
PendingAsk rackLocalAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, node.getRMNode()
|
node.getRackName());
|
||||||
.getRackName());
|
if (rackLocalAsk.getCount() <= 0) {
|
||||||
if (rackLocalRequest == null) {
|
|
||||||
return maxContainers;
|
return maxContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
maxContainers = Math.min(maxContainers, rackLocalRequest.getNumContainers());
|
maxContainers = Math.min(maxContainers,
|
||||||
|
rackLocalAsk.getCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
ResourceRequest nodeLocalRequest =
|
PendingAsk nodeLocalAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, node.getRMNode()
|
node.getRMNode().getHostName());
|
||||||
.getNodeAddress());
|
|
||||||
if (nodeLocalRequest != null) {
|
if (nodeLocalAsk.getCount() > 0) {
|
||||||
maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers());
|
maxContainers = Math.min(maxContainers,
|
||||||
|
nodeLocalAsk.getCount());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -611,25 +608,21 @@ public class FifoScheduler extends
|
||||||
private int assignNodeLocalContainers(FiCaSchedulerNode node,
|
private int assignNodeLocalContainers(FiCaSchedulerNode node,
|
||||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||||
int assignedContainers = 0;
|
int assignedContainers = 0;
|
||||||
ResourceRequest request =
|
PendingAsk nodeLocalAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, node.getNodeName());
|
node.getNodeName());
|
||||||
if (request != null) {
|
if (nodeLocalAsk.getCount() > 0) {
|
||||||
// Don't allocate on this node if we don't need containers on this rack
|
// Don't allocate on this node if we don't need containers on this rack
|
||||||
ResourceRequest rackRequest =
|
if (application.getOutstandingAsksCount(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey,
|
node.getRackName()) <= 0) {
|
||||||
node.getRMNode().getRackName());
|
|
||||||
if (rackRequest == null || rackRequest.getNumContainers() <= 0) {
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
int assignableContainers =
|
int assignableContainers = Math.min(
|
||||||
Math.min(
|
getMaxAllocatableContainers(application, schedulerKey, node,
|
||||||
getMaxAllocatableContainers(application, schedulerKey, node,
|
NodeType.NODE_LOCAL), nodeLocalAsk.getCount());
|
||||||
NodeType.NODE_LOCAL),
|
|
||||||
request.getNumContainers());
|
|
||||||
assignedContainers =
|
assignedContainers =
|
||||||
assignContainer(node, application, schedulerKey,
|
assignContainer(node, application, schedulerKey, assignableContainers,
|
||||||
assignableContainers, request, NodeType.NODE_LOCAL);
|
nodeLocalAsk.getPerAllocationResource(), NodeType.NODE_LOCAL);
|
||||||
}
|
}
|
||||||
return assignedContainers;
|
return assignedContainers;
|
||||||
}
|
}
|
||||||
|
@ -637,25 +630,21 @@ public class FifoScheduler extends
|
||||||
private int assignRackLocalContainers(FiCaSchedulerNode node,
|
private int assignRackLocalContainers(FiCaSchedulerNode node,
|
||||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||||
int assignedContainers = 0;
|
int assignedContainers = 0;
|
||||||
ResourceRequest request =
|
PendingAsk rackAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, node.getRMNode()
|
node.getRMNode().getRackName());
|
||||||
.getRackName());
|
if (rackAsk.getCount() > 0) {
|
||||||
if (request != null) {
|
|
||||||
// Don't allocate on this rack if the application doens't need containers
|
// Don't allocate on this rack if the application doens't need containers
|
||||||
ResourceRequest offSwitchRequest =
|
if (application.getOutstandingAsksCount(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
ResourceRequest.ANY) <= 0) {
|
||||||
if (offSwitchRequest.getNumContainers() <= 0) {
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
int assignableContainers =
|
int assignableContainers =
|
||||||
Math.min(
|
Math.min(getMaxAllocatableContainers(application, schedulerKey, node,
|
||||||
getMaxAllocatableContainers(application, schedulerKey, node,
|
NodeType.RACK_LOCAL), rackAsk.getCount());
|
||||||
NodeType.RACK_LOCAL),
|
|
||||||
request.getNumContainers());
|
|
||||||
assignedContainers =
|
assignedContainers =
|
||||||
assignContainer(node, application, schedulerKey,
|
assignContainer(node, application, schedulerKey, assignableContainers,
|
||||||
assignableContainers, request, NodeType.RACK_LOCAL);
|
rackAsk.getPerAllocationResource(), NodeType.RACK_LOCAL);
|
||||||
}
|
}
|
||||||
return assignedContainers;
|
return assignedContainers;
|
||||||
}
|
}
|
||||||
|
@ -663,26 +652,26 @@ public class FifoScheduler extends
|
||||||
private int assignOffSwitchContainers(FiCaSchedulerNode node,
|
private int assignOffSwitchContainers(FiCaSchedulerNode node,
|
||||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||||
int assignedContainers = 0;
|
int assignedContainers = 0;
|
||||||
ResourceRequest request =
|
PendingAsk offswitchAsk = application.getPendingAsk(schedulerKey,
|
||||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
ResourceRequest.ANY);
|
||||||
if (request != null) {
|
if (offswitchAsk.getCount() > 0) {
|
||||||
assignedContainers =
|
assignedContainers =
|
||||||
assignContainer(node, application, schedulerKey,
|
assignContainer(node, application, schedulerKey,
|
||||||
request.getNumContainers(), request, NodeType.OFF_SWITCH);
|
offswitchAsk.getCount(),
|
||||||
|
offswitchAsk.getPerAllocationResource(), NodeType.OFF_SWITCH);
|
||||||
}
|
}
|
||||||
return assignedContainers;
|
return assignedContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
private int assignContainer(FiCaSchedulerNode node, FifoAppAttempt application,
|
private int assignContainer(FiCaSchedulerNode node, FifoAppAttempt application,
|
||||||
SchedulerRequestKey schedulerKey, int assignableContainers,
|
SchedulerRequestKey schedulerKey, int assignableContainers,
|
||||||
ResourceRequest request, NodeType type) {
|
Resource capability, NodeType type) {
|
||||||
LOG.debug("assignContainers:" +
|
LOG.debug("assignContainers:" +
|
||||||
" node=" + node.getRMNode().getNodeAddress() +
|
" node=" + node.getRMNode().getNodeAddress() +
|
||||||
" application=" + application.getApplicationId().getId() +
|
" application=" + application.getApplicationId().getId() +
|
||||||
" priority=" + schedulerKey.getPriority().getPriority() +
|
" priority=" + schedulerKey.getPriority().getPriority() +
|
||||||
" assignableContainers=" + assignableContainers +
|
" assignableContainers=" + assignableContainers +
|
||||||
" request=" + request + " type=" + type);
|
" capability=" + capability + " type=" + type);
|
||||||
Resource capability = request.getCapability();
|
|
||||||
|
|
||||||
// TODO: A buggy application with this zero would crash the scheduler.
|
// TODO: A buggy application with this zero would crash the scheduler.
|
||||||
int availableContainers =
|
int availableContainers =
|
||||||
|
@ -708,7 +697,7 @@ public class FifoScheduler extends
|
||||||
|
|
||||||
// Inform the application
|
// Inform the application
|
||||||
RMContainer rmContainer = application.allocate(type, node, schedulerKey,
|
RMContainer rmContainer = application.allocate(type, node, schedulerKey,
|
||||||
request, container);
|
container);
|
||||||
|
|
||||||
// Inform the node
|
// Inform the node
|
||||||
node.allocateContainer(rmContainer);
|
node.allocateContainer(rmContainer);
|
||||||
|
|
|
@ -19,12 +19,16 @@
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||||
|
|
||||||
import org.apache.commons.collections.IteratorUtils;
|
import org.apache.commons.collections.IteratorUtils;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -37,9 +41,14 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
||||||
implements SchedulingPlacementSet<N> {
|
implements SchedulingPlacementSet<N> {
|
||||||
|
private static final Log LOG =
|
||||||
|
LogFactory.getLog(LocalitySchedulingPlacementSet.class);
|
||||||
|
|
||||||
private final Map<String, ResourceRequest> resourceRequestMap =
|
private final Map<String, ResourceRequest> resourceRequestMap =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
private AppSchedulingInfo appSchedulingInfo;
|
private AppSchedulingInfo appSchedulingInfo;
|
||||||
|
private volatile String primaryRequestedPartition =
|
||||||
|
RMNodeLabelsManager.NO_LABEL;
|
||||||
|
|
||||||
private final ReentrantReadWriteLock.ReadLock readLock;
|
private final ReentrantReadWriteLock.ReadLock readLock;
|
||||||
private final ReentrantReadWriteLock.WriteLock writeLock;
|
private final ReentrantReadWriteLock.WriteLock writeLock;
|
||||||
|
@ -132,11 +141,14 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
||||||
resourceRequestMap.put(resourceName, request);
|
resourceRequestMap.put(resourceName, request);
|
||||||
|
|
||||||
if (resourceName.equals(ResourceRequest.ANY)) {
|
if (resourceName.equals(ResourceRequest.ANY)) {
|
||||||
|
String partition = request.getNodeLabelExpression() == null ?
|
||||||
|
RMNodeLabelsManager.NO_LABEL :
|
||||||
|
request.getNodeLabelExpression();
|
||||||
|
|
||||||
|
this.primaryRequestedPartition = partition;
|
||||||
|
|
||||||
//update the applications requested labels set
|
//update the applications requested labels set
|
||||||
appSchedulingInfo.addRequestedPartition(
|
appSchedulingInfo.addRequestedPartition(partition);
|
||||||
request.getNodeLabelExpression() == null ?
|
|
||||||
RMNodeLabelsManager.NO_LABEL :
|
|
||||||
request.getNodeLabelExpression());
|
|
||||||
|
|
||||||
updateResult = new ResourceRequestUpdateResult(lastRequest, request);
|
updateResult = new ResourceRequestUpdateResult(lastRequest, request);
|
||||||
}
|
}
|
||||||
|
@ -152,11 +164,43 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
||||||
return resourceRequestMap;
|
return resourceRequestMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
private ResourceRequest getResourceRequest(String resourceName) {
|
||||||
public ResourceRequest getResourceRequest(String resourceName) {
|
|
||||||
return resourceRequestMap.get(resourceName);
|
return resourceRequestMap.get(resourceName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PendingAsk getPendingAsk(String resourceName) {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
ResourceRequest request = getResourceRequest(resourceName);
|
||||||
|
if (null == request) {
|
||||||
|
return PendingAsk.ZERO;
|
||||||
|
} else{
|
||||||
|
return new PendingAsk(request.getCapability(),
|
||||||
|
request.getNumContainers());
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getOutstandingAsksCount(String resourceName) {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
ResourceRequest request = getResourceRequest(resourceName);
|
||||||
|
if (null == request) {
|
||||||
|
return 0;
|
||||||
|
} else{
|
||||||
|
return request.getNumContainers();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
private void decrementOutstanding(SchedulerRequestKey schedulerRequestKey,
|
private void decrementOutstanding(SchedulerRequestKey schedulerRequestKey,
|
||||||
ResourceRequest offSwitchRequest) {
|
ResourceRequest offSwitchRequest) {
|
||||||
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
||||||
|
@ -281,22 +325,67 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canDelayTo(String resourceName) {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
ResourceRequest request = getResourceRequest(resourceName);
|
||||||
|
return request == null || request.getRelaxLocality();
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean acceptNodePartition(String nodePartition,
|
||||||
|
SchedulingMode schedulingMode) {
|
||||||
|
// We will only look at node label = nodeLabelToLookAt according to
|
||||||
|
// schedulingMode and partition of node.
|
||||||
|
String nodePartitionToLookAt;
|
||||||
|
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
|
||||||
|
nodePartitionToLookAt = nodePartition;
|
||||||
|
} else {
|
||||||
|
nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
|
||||||
|
}
|
||||||
|
|
||||||
|
return primaryRequestedPartition.equals(nodePartitionToLookAt);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getPrimaryRequestedNodePartition() {
|
||||||
|
return primaryRequestedPartition;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getUniqueLocationAsks() {
|
||||||
|
return resourceRequestMap.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void showRequests() {
|
||||||
|
for (ResourceRequest request : resourceRequestMap.values()) {
|
||||||
|
if (request.getNumContainers() > 0) {
|
||||||
|
LOG.debug("\tRequest=" + request);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
||||||
NodeType type, SchedulerNode node, ResourceRequest request) {
|
NodeType type, SchedulerNode node) {
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
|
||||||
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
||||||
|
|
||||||
if (null == request) {
|
ResourceRequest request;
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
request = resourceRequestMap.get(node.getNodeName());
|
request = resourceRequestMap.get(node.getNodeName());
|
||||||
} else if (type == NodeType.RACK_LOCAL) {
|
} else if (type == NodeType.RACK_LOCAL) {
|
||||||
request = resourceRequestMap.get(node.getRackName());
|
request = resourceRequestMap.get(node.getRackName());
|
||||||
} else{
|
} else{
|
||||||
request = resourceRequestMap.get(ResourceRequest.ANY);
|
request = resourceRequestMap.get(ResourceRequest.ANY);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
|
@ -312,4 +401,14 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<String> getAcceptedResouceNames() {
|
||||||
|
try {
|
||||||
|
readLock.lock();
|
||||||
|
return resourceRequestMap.keySet().iterator();
|
||||||
|
} finally {
|
||||||
|
readLock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -70,22 +72,38 @@ public interface SchedulingPlacementSet<N extends SchedulerNode> {
|
||||||
Map<String, ResourceRequest> getResourceRequests();
|
Map<String, ResourceRequest> getResourceRequests();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get ResourceRequest by given schedulerKey and resourceName
|
* Get pending ask for given resourceName. If there's no such pendingAsk,
|
||||||
|
* returns {@link PendingAsk#ZERO}
|
||||||
|
*
|
||||||
* @param resourceName resourceName
|
* @param resourceName resourceName
|
||||||
* @return ResourceRequest
|
* @return PendingAsk
|
||||||
*/
|
*/
|
||||||
ResourceRequest getResourceRequest(String resourceName);
|
PendingAsk getPendingAsk(String resourceName);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get #pending-allocations for given resourceName. If there's no such
|
||||||
|
* pendingAsk, returns 0
|
||||||
|
*
|
||||||
|
* @param resourceName resourceName
|
||||||
|
* @return #pending-allocations
|
||||||
|
*/
|
||||||
|
int getOutstandingAsksCount(String resourceName);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Notify container allocated.
|
* Notify container allocated.
|
||||||
* @param schedulerKey SchedulerRequestKey for this ResourceRequest
|
* @param schedulerKey SchedulerRequestKey for this ResourceRequest
|
||||||
* @param type Type of the allocation
|
* @param type Type of the allocation
|
||||||
* @param node Which node this container allocated on
|
* @param node Which node this container allocated on
|
||||||
* @param request Which resource request to allocate
|
|
||||||
* @return list of ResourceRequests deducted
|
* @return list of ResourceRequests deducted
|
||||||
*/
|
*/
|
||||||
List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
||||||
NodeType type, SchedulerNode node, ResourceRequest request);
|
NodeType type, SchedulerNode node);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns list of accepted resourceNames.
|
||||||
|
* @return Iterator of accepted resourceNames
|
||||||
|
*/
|
||||||
|
Iterator<String> getAcceptedResouceNames();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* We can still have pending requirement for a given NodeType and node
|
* We can still have pending requirement for a given NodeType and node
|
||||||
|
@ -94,4 +112,47 @@ public interface SchedulingPlacementSet<N extends SchedulerNode> {
|
||||||
* @return true if we has pending requirement
|
* @return true if we has pending requirement
|
||||||
*/
|
*/
|
||||||
boolean canAllocate(NodeType type, SchedulerNode node);
|
boolean canAllocate(NodeType type, SchedulerNode node);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Can delay to give locality?
|
||||||
|
* TODO (wangda): This should be moved out of SchedulingPlacementSet
|
||||||
|
* and should belong to specific delay scheduling policy impl.
|
||||||
|
*
|
||||||
|
* @param resourceName resourceName
|
||||||
|
* @return can/cannot
|
||||||
|
*/
|
||||||
|
boolean canDelayTo(String resourceName);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does this {@link SchedulingPlacementSet} accept resources on nodePartition?
|
||||||
|
*
|
||||||
|
* @param nodePartition nodePartition
|
||||||
|
* @param schedulingMode schedulingMode
|
||||||
|
* @return accepted/not
|
||||||
|
*/
|
||||||
|
boolean acceptNodePartition(String nodePartition,
|
||||||
|
SchedulingMode schedulingMode);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* It is possible that one request can accept multiple node partition,
|
||||||
|
* So this method returns primary node partition for pending resource /
|
||||||
|
* headroom calculation.
|
||||||
|
*
|
||||||
|
* @return primary requested node partition
|
||||||
|
*/
|
||||||
|
String getPrimaryRequestedNodePartition();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return number of unique location asks with #pending greater than 0,
|
||||||
|
* (like /rack1, host1, etc.).
|
||||||
|
*
|
||||||
|
* TODO (wangda): This should be moved out of SchedulingPlacementSet
|
||||||
|
* and should belong to specific delay scheduling policy impl.
|
||||||
|
*/
|
||||||
|
int getUniqueLocationAsks();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Print human-readable requests to LOG debug.
|
||||||
|
*/
|
||||||
|
void showRequests();
|
||||||
}
|
}
|
||||||
|
|
|
@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -588,12 +589,14 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
|
||||||
// The core part of this test
|
// The core part of this test
|
||||||
// The killed containers' ResourceRequests are recovered back to the
|
// The killed containers' ResourceRequests are recovered back to the
|
||||||
// original app-attempt, not the new one
|
// original app-attempt, not the new one
|
||||||
for (ResourceRequest request : firstSchedulerAppAttempt
|
for (SchedulerRequestKey key : firstSchedulerAppAttempt.getSchedulerKeys()) {
|
||||||
.getAppSchedulingInfo().getAllResourceRequests()) {
|
if (key.getPriority().getPriority() == 0) {
|
||||||
if (request.getPriority().getPriority() == 0) {
|
Assert.assertEquals(0,
|
||||||
Assert.assertEquals(0, request.getNumContainers());
|
firstSchedulerAppAttempt.getOutstandingAsksCount(key));
|
||||||
} else if (request.getPriority().getPriority() == ALLOCATED_CONTAINER_PRIORITY) {
|
} else if (key.getPriority().getPriority() ==
|
||||||
Assert.assertEquals(1, request.getNumContainers());
|
ALLOCATED_CONTAINER_PRIORITY) {
|
||||||
|
Assert.assertEquals(1,
|
||||||
|
firstSchedulerAppAttempt.getOutstandingAsksCount(key));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -141,7 +141,7 @@ public class TestAppSchedulingInfo {
|
||||||
|
|
||||||
// iterate to verify no ConcurrentModificationException
|
// iterate to verify no ConcurrentModificationException
|
||||||
for (SchedulerRequestKey schedulerKey : info.getSchedulerKeys()) {
|
for (SchedulerRequestKey schedulerKey : info.getSchedulerKeys()) {
|
||||||
info.allocate(NodeType.OFF_SWITCH, null, schedulerKey, req1, null);
|
info.allocate(NodeType.OFF_SWITCH, null, schedulerKey, null);
|
||||||
}
|
}
|
||||||
Assert.assertEquals(1, info.getSchedulerKeys().size());
|
Assert.assertEquals(1, info.getSchedulerKeys().size());
|
||||||
Assert.assertEquals(SchedulerRequestKey.create(req2),
|
Assert.assertEquals(SchedulerRequestKey.create(req2),
|
||||||
|
@ -153,7 +153,7 @@ public class TestAppSchedulingInfo {
|
||||||
reqs.add(req2);
|
reqs.add(req2);
|
||||||
info.updateResourceRequests(reqs, false);
|
info.updateResourceRequests(reqs, false);
|
||||||
info.allocate(NodeType.OFF_SWITCH, null, SchedulerRequestKey.create(req2),
|
info.allocate(NodeType.OFF_SWITCH, null, SchedulerRequestKey.create(req2),
|
||||||
req2, null);
|
null);
|
||||||
Assert.assertEquals(0, info.getSchedulerKeys().size());
|
Assert.assertEquals(0, info.getSchedulerKeys().size());
|
||||||
|
|
||||||
req1 = ResourceRequest.newInstance(pri1,
|
req1 = ResourceRequest.newInstance(pri1,
|
||||||
|
|
|
@ -93,8 +93,7 @@ public class TestSchedulerApplicationAttempt {
|
||||||
app.liveContainers.put(container1.getContainerId(), container1);
|
app.liveContainers.put(container1.getContainerId(), container1);
|
||||||
SchedulerNode node = createNode();
|
SchedulerNode node = createNode();
|
||||||
app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node,
|
app.appSchedulingInfo.allocate(NodeType.OFF_SWITCH, node,
|
||||||
toSchedulerKey(requestedPriority),
|
toSchedulerKey(requestedPriority), container1.getContainer());
|
||||||
request, container1.getContainer());
|
|
||||||
|
|
||||||
// Reserved container
|
// Reserved container
|
||||||
Priority prio1 = Priority.newInstance(1);
|
Priority prio1 = Priority.newInstance(1);
|
||||||
|
|
|
@ -277,14 +277,14 @@ public class TestCapacityScheduler {
|
||||||
// Register node1
|
// Register node1
|
||||||
String host_0 = "host_0";
|
String host_0 = "host_0";
|
||||||
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 =
|
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 =
|
||||||
registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
||||||
Resources.createResource(4 * GB, 1));
|
Resources.createResource(4 * GB, 1));
|
||||||
|
|
||||||
// Register node2
|
// Register node2
|
||||||
String host_1 = "host_1";
|
String host_1 = "host_1";
|
||||||
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 =
|
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 =
|
||||||
registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK,
|
||||||
Resources.createResource(2 * GB, 1));
|
Resources.createResource(2 * GB, 1));
|
||||||
|
|
||||||
// ResourceRequest priorities
|
// ResourceRequest priorities
|
||||||
Priority priority_0 = Priority.newInstance(0);
|
Priority priority_0 = Priority.newInstance(0);
|
||||||
|
@ -1335,8 +1335,9 @@ public class TestCapacityScheduler {
|
||||||
|
|
||||||
// Already the node local resource request is cleared from RM after
|
// Already the node local resource request is cleared from RM after
|
||||||
// allocation.
|
// allocation.
|
||||||
Assert.assertNull(app.getResourceRequest(
|
Assert.assertEquals(0,
|
||||||
SchedulerRequestKey.create(request), request.getResourceName()));
|
app.getOutstandingAsksCount(SchedulerRequestKey.create(request),
|
||||||
|
request.getResourceName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Call killContainer to preempt the container
|
// Call killContainer to preempt the container
|
||||||
|
@ -1346,10 +1347,9 @@ public class TestCapacityScheduler {
|
||||||
for (ResourceRequest request : requests) {
|
for (ResourceRequest request : requests) {
|
||||||
// Resource request must have added back in RM after preempt event
|
// Resource request must have added back in RM after preempt event
|
||||||
// handling.
|
// handling.
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(1,
|
||||||
1,
|
app.getOutstandingAsksCount(SchedulerRequestKey.create(request),
|
||||||
app.getResourceRequest(SchedulerRequestKey.create(request),
|
request.getResourceName()));
|
||||||
request.getResourceName()).getNumContainers());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// New container will be allocated and will move to ALLOCATED state
|
// New container will be allocated and will move to ALLOCATED state
|
||||||
|
|
|
@ -23,7 +23,6 @@ import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.Matchers.any;
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyBoolean;
|
import static org.mockito.Matchers.anyBoolean;
|
||||||
import static org.mockito.Matchers.anyString;
|
|
||||||
import static org.mockito.Mockito.doNothing;
|
import static org.mockito.Mockito.doNothing;
|
||||||
import static org.mockito.Mockito.doReturn;
|
import static org.mockito.Mockito.doReturn;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
@ -1056,9 +1055,13 @@ public class TestLeafQueue {
|
||||||
//test case 3
|
//test case 3
|
||||||
qb.finishApplication(app_0.getApplicationId(), user_0);
|
qb.finishApplication(app_0.getApplicationId(), user_0);
|
||||||
qb.finishApplication(app_2.getApplicationId(), user_1);
|
qb.finishApplication(app_2.getApplicationId(), user_1);
|
||||||
qb.releaseResource(clusterResource, app_0, app_0.getResource(u0SchedKey),
|
qb.releaseResource(clusterResource, app_0,
|
||||||
|
app_0.getAppSchedulingInfo().getPendingAsk(u0SchedKey)
|
||||||
|
.getPerAllocationResource(),
|
||||||
null, null, false);
|
null, null, false);
|
||||||
qb.releaseResource(clusterResource, app_2, app_2.getResource(u1SchedKey),
|
qb.releaseResource(clusterResource, app_2,
|
||||||
|
app_2.getAppSchedulingInfo().getPendingAsk(u1SchedKey)
|
||||||
|
.getPerAllocationResource(),
|
||||||
null, null, false);
|
null, null, false);
|
||||||
|
|
||||||
qb.setUserLimit(50);
|
qb.setUserLimit(50);
|
||||||
|
@ -1956,7 +1959,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
||||||
|
|
||||||
// Another off switch, shouldn't allocate due to delay scheduling
|
// Another off switch, shouldn't allocate due to delay scheduling
|
||||||
|
@ -1965,7 +1968,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
||||||
|
|
||||||
// Another off switch, shouldn't allocate due to delay scheduling
|
// Another off switch, shouldn't allocate due to delay scheduling
|
||||||
|
@ -1974,7 +1977,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
|
||||||
|
|
||||||
// Another off switch, now we should allocate
|
// Another off switch, now we should allocate
|
||||||
|
@ -1985,7 +1988,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
||||||
// should NOT reset
|
// should NOT reset
|
||||||
assertEquals(4, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(4, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(2, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// NODE_LOCAL - node_0
|
// NODE_LOCAL - node_0
|
||||||
assignment = a.assignContainers(clusterResource, node_0,
|
assignment = a.assignContainers(clusterResource, node_0,
|
||||||
|
@ -1994,7 +1997,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
// should reset
|
// should reset
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// NODE_LOCAL - node_1
|
// NODE_LOCAL - node_1
|
||||||
assignment = a.assignContainers(clusterResource, node_1,
|
assignment = a.assignContainers(clusterResource, node_1,
|
||||||
|
@ -2003,7 +2006,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
// should reset
|
// should reset
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
assertEquals(NodeType.NODE_LOCAL, assignment.getType());
|
assertEquals(NodeType.NODE_LOCAL, assignment.getType());
|
||||||
|
|
||||||
// Add 1 more request to check for RACK_LOCAL
|
// Add 1 more request to check for RACK_LOCAL
|
||||||
|
@ -2018,7 +2021,7 @@ public class TestLeafQueue {
|
||||||
TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
|
TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 4, // one extra
|
||||||
true, priority, recordFactory));
|
true, priority, recordFactory));
|
||||||
app_0.updateResourceRequests(app_0_requests_0);
|
app_0.updateResourceRequests(app_0_requests_0);
|
||||||
assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(4, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Rack-delay
|
// Rack-delay
|
||||||
doReturn(true).when(a).getRackLocalityFullReset();
|
doReturn(true).when(a).getRackLocalityFullReset();
|
||||||
|
@ -2029,7 +2032,7 @@ public class TestLeafQueue {
|
||||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(4, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(4, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Should assign RACK_LOCAL now
|
// Should assign RACK_LOCAL now
|
||||||
assignment = a.assignContainers(clusterResource, node_3,
|
assignment = a.assignContainers(clusterResource, node_3,
|
||||||
|
@ -2038,14 +2041,14 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
||||||
// should reset
|
// should reset
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
|
// Shouldn't assign RACK_LOCAL because schedulingOpportunities should have gotten reset.
|
||||||
assignment = a.assignContainers(clusterResource, node_3,
|
assignment = a.assignContainers(clusterResource, node_3,
|
||||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Next time we schedule RACK_LOCAL, don't reset
|
// Next time we schedule RACK_LOCAL, don't reset
|
||||||
doReturn(false).when(a).getRackLocalityFullReset();
|
doReturn(false).when(a).getRackLocalityFullReset();
|
||||||
|
@ -2057,7 +2060,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
||||||
// should NOT reset
|
// should NOT reset
|
||||||
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(2, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Another RACK_LOCAL since schedulingOpportunities not reset
|
// Another RACK_LOCAL since schedulingOpportunities not reset
|
||||||
assignment = a.assignContainers(clusterResource, node_3,
|
assignment = a.assignContainers(clusterResource, node_3,
|
||||||
|
@ -2066,7 +2069,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.RACK_LOCAL);
|
||||||
// should NOT reset
|
// should NOT reset
|
||||||
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Add a request larger than cluster size to verify
|
// Add a request larger than cluster size to verify
|
||||||
// OFF_SWITCH delay is capped by cluster size
|
// OFF_SWITCH delay is capped by cluster size
|
||||||
|
@ -2185,9 +2188,9 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey1));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey1));
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
|
assertEquals(2, app_0.getOutstandingAsksCount(schedulerKey1));
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey2));
|
||||||
|
|
||||||
// Another off-switch, shouldn't allocate P1 due to delay scheduling
|
// Another off-switch, shouldn't allocate P1 due to delay scheduling
|
||||||
// thus, no P2 either!
|
// thus, no P2 either!
|
||||||
|
@ -2196,9 +2199,9 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey1));
|
assertEquals(2, app_0.getSchedulingOpportunities(schedulerKey1));
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(schedulerKey1));
|
assertEquals(2, app_0.getOutstandingAsksCount(schedulerKey1));
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey2));
|
||||||
|
|
||||||
// Another off-switch, shouldn't allocate OFF_SWITCH P1
|
// Another off-switch, shouldn't allocate OFF_SWITCH P1
|
||||||
assignment = a.assignContainers(clusterResource, node_2,
|
assignment = a.assignContainers(clusterResource, node_2,
|
||||||
|
@ -2206,9 +2209,9 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
||||||
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey1));
|
assertEquals(3, app_0.getSchedulingOpportunities(schedulerKey1));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey1));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey1));
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey2));
|
||||||
|
|
||||||
// Now, DATA_LOCAL for P1
|
// Now, DATA_LOCAL for P1
|
||||||
assignment = a.assignContainers(clusterResource, node_0,
|
assignment = a.assignContainers(clusterResource, node_0,
|
||||||
|
@ -2216,9 +2219,9 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey1));
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey2));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey2));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey2));
|
||||||
|
|
||||||
// Now, OFF_SWITCH for P2
|
// Now, OFF_SWITCH for P2
|
||||||
assignment = a.assignContainers(clusterResource, node_1,
|
assignment = a.assignContainers(clusterResource, node_1,
|
||||||
|
@ -2226,9 +2229,9 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
verifyContainerAllocated(assignment, NodeType.OFF_SWITCH);
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey1));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey1));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey1));
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey2));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey2));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey2));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey2));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2309,7 +2312,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
// should reset
|
// should reset
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// No allocation on node_1_0 even though it's node/rack local since
|
// No allocation on node_1_0 even though it's node/rack local since
|
||||||
// required(ANY) == 0
|
// required(ANY) == 0
|
||||||
|
@ -2320,7 +2323,7 @@ public class TestLeafQueue {
|
||||||
// Still zero
|
// Still zero
|
||||||
// since #req=0
|
// since #req=0
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Add one request
|
// Add one request
|
||||||
app_0_requests_0.clear();
|
app_0_requests_0.clear();
|
||||||
|
@ -2336,7 +2339,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// NODE_LOCAL - node_1
|
// NODE_LOCAL - node_1
|
||||||
assignment = a.assignContainers(clusterResource, node_1_0,
|
assignment = a.assignContainers(clusterResource, node_1_0,
|
||||||
|
@ -2345,7 +2348,7 @@ public class TestLeafQueue {
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
// should reset
|
// should reset
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 30000)
|
@Test (timeout = 30000)
|
||||||
|
@ -2721,7 +2724,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyNoContainerAllocated(assignment);
|
verifyNoContainerAllocated(assignment);
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(1, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
// Now sanity-check node_local
|
// Now sanity-check node_local
|
||||||
app_0_requests_0.add(
|
app_0_requests_0.add(
|
||||||
|
@ -2752,7 +2755,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(0, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(0, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3205,7 +3208,7 @@ public class TestLeafQueue {
|
||||||
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
applyCSAssignment(clusterResource, assignment, a, nodes, apps);
|
||||||
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
verifyContainerAllocated(assignment, NodeType.NODE_LOCAL);
|
||||||
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
assertEquals(1, app_0.getSchedulingOpportunities(schedulerKey));
|
||||||
assertEquals(3, app_0.getTotalRequiredResources(schedulerKey));
|
assertEquals(3, app_0.getOutstandingAsksCount(schedulerKey));
|
||||||
assertEquals(0, app_0.getLiveContainers().size());
|
assertEquals(0, app_0.getLiveContainers().size());
|
||||||
assertEquals(1, app_1.getLiveContainers().size());
|
assertEquals(1, app_1.getLiveContainers().size());
|
||||||
}
|
}
|
||||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
|
||||||
|
|
||||||
|
@ -56,6 +57,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -548,11 +551,12 @@ public class TestNodeLabelContainerAllocation {
|
||||||
ApplicationAttemptId attemptId, int memory) {
|
ApplicationAttemptId attemptId, int memory) {
|
||||||
CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
|
CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
|
||||||
FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
|
FiCaSchedulerApp app = cs.getApplicationAttempt(attemptId);
|
||||||
ResourceRequest rr =
|
PendingAsk ask =
|
||||||
app.getAppSchedulingInfo().getResourceRequest(
|
app.getAppSchedulingInfo().getPendingAsk(
|
||||||
TestUtils.toSchedulerKey(priority), "*");
|
TestUtils.toSchedulerKey(priority), "*");
|
||||||
Assert.assertEquals(memory,
|
Assert.assertEquals(memory,
|
||||||
rr.getCapability().getMemorySize() * rr.getNumContainers());
|
ask.getPerAllocationResource().getMemorySize() * ask
|
||||||
|
.getCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkLaunchedContainerNumOnNode(MockRM rm, NodeId nodeId,
|
private void checkLaunchedContainerNumOnNode(MockRM rm, NodeId nodeId,
|
||||||
|
@ -607,18 +611,10 @@ public class TestNodeLabelContainerAllocation {
|
||||||
(CapacityScheduler) rm1.getRMContext().getScheduler();
|
(CapacityScheduler) rm1.getRMContext().getScheduler();
|
||||||
FiCaSchedulerApp app =
|
FiCaSchedulerApp app =
|
||||||
cs.getApplicationAttempt(am1.getApplicationAttemptId());
|
cs.getApplicationAttempt(am1.getApplicationAttemptId());
|
||||||
List<ResourceRequest> allResourceRequests =
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 2, "y");
|
||||||
app.getAppSchedulingInfo().getAllResourceRequests();
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 3, "y");
|
||||||
for (ResourceRequest changeReq : allResourceRequests) {
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 4,
|
||||||
if (changeReq.getPriority().getPriority() == 2
|
RMNodeLabelsManager.NO_LABEL);
|
||||||
|| changeReq.getPriority().getPriority() == 3) {
|
|
||||||
Assert.assertEquals("Expected label y", "y",
|
|
||||||
changeReq.getNodeLabelExpression());
|
|
||||||
} else if (changeReq.getPriority().getPriority() == 4) {
|
|
||||||
Assert.assertEquals("Expected label EMPTY",
|
|
||||||
RMNodeLabelsManager.NO_LABEL, changeReq.getNodeLabelExpression());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Previous any request was Y trying to update with z and the
|
// Previous any request was Y trying to update with z and the
|
||||||
// request before ANY label is null
|
// request before ANY label is null
|
||||||
|
@ -628,17 +624,11 @@ public class TestNodeLabelContainerAllocation {
|
||||||
newReq.add(am1.createResourceReq("h1:1234", 1024, 3, 4, null));
|
newReq.add(am1.createResourceReq("h1:1234", 1024, 3, 4, null));
|
||||||
newReq.add(am1.createResourceReq("*", 1024, 4, 5, "z"));
|
newReq.add(am1.createResourceReq("*", 1024, 4, 5, "z"));
|
||||||
am1.allocate(newReq, new ArrayList<ContainerId>());
|
am1.allocate(newReq, new ArrayList<ContainerId>());
|
||||||
allResourceRequests = app.getAppSchedulingInfo().getAllResourceRequests();
|
|
||||||
for (ResourceRequest changeReq : allResourceRequests) {
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 3, "z");
|
||||||
if (changeReq.getPriority().getPriority() == 3
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 4, "z");
|
||||||
|| changeReq.getPriority().getPriority() == 4) {
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 2, "y");
|
||||||
Assert.assertEquals("Expected label z", "z",
|
|
||||||
changeReq.getNodeLabelExpression());
|
|
||||||
} else if (changeReq.getPriority().getPriority() == 2) {
|
|
||||||
Assert.assertEquals("Expected label y", "y",
|
|
||||||
changeReq.getNodeLabelExpression());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Request before ANY and ANY request is set as NULL. Request should be set
|
// Request before ANY and ANY request is set as NULL. Request should be set
|
||||||
// with Empty Label
|
// with Empty Label
|
||||||
List<ResourceRequest> resourceRequest1 = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> resourceRequest1 = new ArrayList<ResourceRequest>();
|
||||||
|
@ -653,14 +643,21 @@ public class TestNodeLabelContainerAllocation {
|
||||||
RMNodeLabelsManager.NO_LABEL));
|
RMNodeLabelsManager.NO_LABEL));
|
||||||
resourceRequest1.add(am1.createResourceReq("h2:1234", 1024, 2, 4, null));
|
resourceRequest1.add(am1.createResourceReq("h2:1234", 1024, 2, 4, null));
|
||||||
am1.allocate(resourceRequest1, new ArrayList<ContainerId>());
|
am1.allocate(resourceRequest1, new ArrayList<ContainerId>());
|
||||||
allResourceRequests = app.getAppSchedulingInfo().getAllResourceRequests();
|
|
||||||
for (ResourceRequest changeReq : allResourceRequests) {
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 3,
|
||||||
if (changeReq.getPriority().getPriority() == 3) {
|
RMNodeLabelsManager.NO_LABEL);
|
||||||
Assert.assertEquals("Expected label Empty",
|
checkNodePartitionOfRequestedPriority(app.getAppSchedulingInfo(), 2,
|
||||||
RMNodeLabelsManager.NO_LABEL, changeReq.getNodeLabelExpression());
|
RMNodeLabelsManager.NO_LABEL);
|
||||||
} else if (changeReq.getPriority().getPriority() == 2) {
|
}
|
||||||
Assert.assertEquals("Expected label y", RMNodeLabelsManager.NO_LABEL,
|
|
||||||
changeReq.getNodeLabelExpression());
|
private void checkNodePartitionOfRequestedPriority(AppSchedulingInfo info,
|
||||||
|
int priority, String expectedPartition) {
|
||||||
|
for (SchedulerRequestKey key : info.getSchedulerKeys()) {
|
||||||
|
if (key.getPriority().getPriority() == priority) {
|
||||||
|
Assert.assertEquals("Expected partition is " + expectedPartition,
|
||||||
|
expectedPartition,
|
||||||
|
info.getSchedulingPlacementSet(key)
|
||||||
|
.getPrimaryRequestedNodePartition());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -329,7 +329,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// try to assign reducer (5G on node 0 and should reserve)
|
// try to assign reducer (5G on node 0 and should reserve)
|
||||||
|
@ -348,7 +348,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// assign reducer to node 2
|
// assign reducer to node 2
|
||||||
|
@ -367,7 +367,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(
|
assertEquals(1, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// node_1 heartbeat and unreserves from node_0 in order to allocate
|
// node_1 heartbeat and unreserves from node_0 in order to allocate
|
||||||
|
@ -386,7 +386,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(0, app_0.getTotalRequiredResources(
|
assertEquals(0, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -662,7 +662,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// try to assign reducer (5G on node 0 and should reserve)
|
// try to assign reducer (5G on node 0 and should reserve)
|
||||||
|
@ -681,7 +681,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// assign reducer to node 2
|
// assign reducer to node 2
|
||||||
|
@ -700,7 +700,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(
|
assertEquals(1, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// node_1 heartbeat and won't unreserve from node_0, potentially stuck
|
// node_1 heartbeat and won't unreserve from node_0, potentially stuck
|
||||||
|
@ -720,7 +720,7 @@ public class TestReservations {
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(
|
assertEquals(1, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -841,7 +841,7 @@ public class TestReservations {
|
||||||
assertEquals(null, node_0.getReservedContainer());
|
assertEquals(null, node_0.getReservedContainer());
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// try to assign reducer (5G on node 0 and should reserve)
|
// try to assign reducer (5G on node 0 and should reserve)
|
||||||
|
@ -859,7 +859,7 @@ public class TestReservations {
|
||||||
.getMemorySize());
|
.getMemorySize());
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(3 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(2, app_0.getTotalRequiredResources(
|
assertEquals(2, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
|
|
||||||
// could allocate but told need to unreserve first
|
// could allocate but told need to unreserve first
|
||||||
|
@ -876,7 +876,7 @@ public class TestReservations {
|
||||||
assertEquals(null, node_0.getReservedContainer());
|
assertEquals(null, node_0.getReservedContainer());
|
||||||
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
assertEquals(5 * GB, node_0.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
|
assertEquals(8 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||||
assertEquals(1, app_0.getTotalRequiredResources(
|
assertEquals(1, app_0.getOutstandingAsksCount(
|
||||||
toSchedulerKey(priorityReduce)));
|
toSchedulerKey(priorityReduce)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue