|
@@ -45,10 +45,10 @@ 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.capacity.SchedulingMode;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
|
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
@@ -90,10 +90,12 @@ public class AppSchedulingInfo {
|
|
private final ReentrantReadWriteLock.WriteLock writeLock;
|
|
private final ReentrantReadWriteLock.WriteLock writeLock;
|
|
|
|
|
|
public final ContainerUpdateContext updateContext;
|
|
public final ContainerUpdateContext updateContext;
|
|
|
|
+ public final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
|
|
|
|
|
|
- public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
|
|
|
- String user, Queue queue, AbstractUsersManager abstractUsersManager,
|
|
|
|
- long epoch, ResourceUsage appResourceUsage) {
|
|
|
|
|
|
+ public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
|
|
|
|
+ Queue queue, AbstractUsersManager abstractUsersManager, long epoch,
|
|
|
|
+ ResourceUsage appResourceUsage,
|
|
|
|
+ Map<String, String> applicationSchedulingEnvs) {
|
|
this.applicationAttemptId = appAttemptId;
|
|
this.applicationAttemptId = appAttemptId;
|
|
this.applicationId = appAttemptId.getApplicationId();
|
|
this.applicationId = appAttemptId.getApplicationId();
|
|
this.queue = queue;
|
|
this.queue = queue;
|
|
@@ -102,6 +104,7 @@ public class AppSchedulingInfo {
|
|
this.containerIdCounter = new AtomicLong(
|
|
this.containerIdCounter = new AtomicLong(
|
|
epoch << ResourceManager.EPOCH_BIT_SHIFT);
|
|
epoch << ResourceManager.EPOCH_BIT_SHIFT);
|
|
this.appResourceUsage = appResourceUsage;
|
|
this.appResourceUsage = appResourceUsage;
|
|
|
|
+ this.applicationSchedulingEnvs.putAll(applicationSchedulingEnvs);
|
|
|
|
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
updateContext = new ContainerUpdateContext(this);
|
|
updateContext = new ContainerUpdateContext(this);
|
|
@@ -211,24 +214,27 @@ public class AppSchedulingInfo {
|
|
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
|
|
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
|
|
boolean offswitchResourcesUpdated = false;
|
|
boolean offswitchResourcesUpdated = false;
|
|
for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
|
|
for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
|
|
- dedupRequests.entrySet()) {
|
|
|
|
|
|
+ dedupRequests.entrySet()) {
|
|
SchedulerRequestKey schedulerRequestKey = entry.getKey();
|
|
SchedulerRequestKey schedulerRequestKey = entry.getKey();
|
|
|
|
|
|
- if (!schedulerKeyToAppPlacementAllocator.containsKey(
|
|
|
|
- schedulerRequestKey)) {
|
|
|
|
|
|
+ if (!schedulerKeyToAppPlacementAllocator
|
|
|
|
+ .containsKey(schedulerRequestKey)) {
|
|
|
|
+ AppPlacementAllocator<SchedulerNode> placementAllocatorInstance = ApplicationPlacementFactory
|
|
|
|
+ .getAppPlacementAllocator(applicationSchedulingEnvs
|
|
|
|
+ .get(ApplicationSchedulingConfig.ENV_APPLICATION_PLACEMENT_TYPE_CLASS));
|
|
|
|
+ placementAllocatorInstance.setAppSchedulingInfo(this);
|
|
|
|
+
|
|
schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
|
|
schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
|
|
- new LocalityAppPlacementAllocator<>(this));
|
|
|
|
|
|
+ placementAllocatorInstance);
|
|
}
|
|
}
|
|
|
|
|
|
// Update AppPlacementAllocator
|
|
// Update AppPlacementAllocator
|
|
- PendingAskUpdateResult pendingAmountChanges =
|
|
|
|
- schedulerKeyToAppPlacementAllocator.get(schedulerRequestKey)
|
|
|
|
- .updatePendingAsk(entry.getValue().values(),
|
|
|
|
- recoverPreemptedRequestForAContainer);
|
|
|
|
|
|
+ PendingAskUpdateResult pendingAmountChanges = schedulerKeyToAppPlacementAllocator
|
|
|
|
+ .get(schedulerRequestKey).updatePendingAsk(entry.getValue().values(),
|
|
|
|
+ recoverPreemptedRequestForAContainer);
|
|
|
|
|
|
if (null != pendingAmountChanges) {
|
|
if (null != pendingAmountChanges) {
|
|
- updatePendingResources(
|
|
|
|
- pendingAmountChanges, schedulerRequestKey,
|
|
|
|
|
|
+ updatePendingResources(pendingAmountChanges, schedulerRequestKey,
|
|
queue.getMetrics());
|
|
queue.getMetrics());
|
|
offswitchResourcesUpdated = true;
|
|
offswitchResourcesUpdated = true;
|
|
}
|
|
}
|