Revert "YARN-4502. Fix two AM containers get allocated when AM restart. (Vinod Kumar Vavilapalli via wangda)"

This reverts commit 3fe5728563.

Conflicts:
	hadoop-yarn-project/CHANGES.txt

(cherry picked from commit 150f5ae034)
This commit is contained in:
Wangda Tan 2016-01-19 09:27:36 +08:00
parent cdef283cd0
commit bacb38aaf3
23 changed files with 190 additions and 257 deletions

View File

@ -70,10 +70,10 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
@ -958,7 +958,7 @@ public synchronized List<Container> getTransferredContainers(
}
@Override
protected void completedContainerInternal(RMContainer rmContainer,
protected void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event) {
// do nothing
}

View File

@ -1146,9 +1146,6 @@ Release 2.8.0 - UNRELEASED
YARN-4538. QueueMetrics pending cores and memory metrics wrong.
(Bibin A Chundatt via wangda)
YARN-4502. Fix two AM containers get allocated when AM restart.
(Vinod Kumar Vavilapalli via wangda)
YARN-4596. SystemMetricPublisher should not swallow error messages from
TimelineClient#putEntities. (Li Lu via jianhe)

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
@ -50,7 +51,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
@ -257,7 +257,7 @@ private void containerBasedPreemptOrKill(CSQueue root,
// kill it
rmContext.getDispatcher().getEventHandler().handle(
new ContainerPreemptEvent(appAttemptId, container,
SchedulerEventType.KILL_PREEMPTED_CONTAINER));
SchedulerEventType.KILL_CONTAINER));
preempted.remove(container);
} else {
if (preempted.get(container) != null) {
@ -764,7 +764,7 @@ private void preemptFrom(FiCaSchedulerApp app,
if (!observeOnly) {
rmContext.getDispatcher().getEventHandler().handle(
new ContainerPreemptEvent(
appId, c, SchedulerEventType.KILL_RESERVED_CONTAINER));
appId, c, SchedulerEventType.DROP_RESERVATION));
}
}

View File

@ -49,6 +49,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerRescheduledEvent;
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
import org.apache.hadoop.yarn.state.SingleArcTransition;
@ -96,7 +97,7 @@ RMContainerEventType.ACQUIRED, new AcquiredTransition())
.addTransition(RMContainerState.ALLOCATED, RMContainerState.EXPIRED,
RMContainerEventType.EXPIRE, new FinishedTransition())
.addTransition(RMContainerState.ALLOCATED, RMContainerState.KILLED,
RMContainerEventType.KILL, new FinishedTransition())
RMContainerEventType.KILL, new ContainerRescheduledTransition())
// Transitions from ACQUIRED state
.addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
@ -520,8 +521,7 @@ private static final class AcquiredTransition extends BaseTransition {
@Override
public void transition(RMContainerImpl container, RMContainerEvent event) {
// Clear ResourceRequest stored in RMContainer, we don't need to remember
// this anymore.
// Clear ResourceRequest stored in RMContainer
container.setResourceRequests(null);
// Register with containerAllocationExpirer.
@ -597,6 +597,17 @@ public void transition(RMContainerImpl container, RMContainerEvent event) {
}
}
private static final class ContainerRescheduledTransition extends
FinishedTransition {
@Override
public void transition(RMContainerImpl container, RMContainerEvent event) {
// Tell scheduler to recover request of this container to app
container.eventHandler.handle(new ContainerRescheduledEvent(container));
super.transition(container, event);
}
}
private static class FinishedTransition extends BaseTransition {
@Override

View File

@ -496,28 +496,20 @@ private RMContainer recoverAndCreateContainer(NMContainerStatus status,
* Recover resource request back from RMContainer when a container is
* preempted before AM pulled the same. If container is pulled by
* AM, then RMContainer will not have resource request to recover.
* @param rmContainer rmContainer
* @param rmContainer
*/
private void recoverResourceRequestForContainer(RMContainer rmContainer) {
protected void recoverResourceRequestForContainer(RMContainer rmContainer) {
List<ResourceRequest> requests = rmContainer.getResourceRequests();
// If container state is moved to ACQUIRED, request will be empty.
if (requests == null) {
return;
}
// Add resource request back to Scheduler ApplicationAttempt.
// We lookup the application-attempt here again using
// getCurrentApplicationAttempt() because there is only one app-attempt at
// any point in the scheduler. But in corner cases, AMs can crash,
// corresponding containers get killed and recovered to the same-attempt,
// but because the app-attempt is extinguished right after, the recovered
// requests don't serve any purpose, but that's okay.
SchedulerApplicationAttempt schedulerAttempt =
getCurrentAttemptForContainer(rmContainer.getContainerId());
// Add resource request back to Scheduler.
SchedulerApplicationAttempt schedulerAttempt
= getCurrentAttemptForContainer(rmContainer.getContainerId());
if (schedulerAttempt != null) {
schedulerAttempt.recoverResourceRequestsForContainer(requests);
schedulerAttempt.recoverResourceRequests(requests);
}
}
@ -552,30 +544,8 @@ public void clearPendingContainerCache() {
}
}
@VisibleForTesting
@Private
// clean up a completed container
public void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event) {
if (rmContainer == null) {
LOG.info("Container " + containerStatus.getContainerId()
+ " completed with event " + event
+ ", but corresponding RMContainer doesn't exist.");
return;
}
completedContainerInternal(rmContainer, containerStatus, event);
// If the container is getting killed in ACQUIRED state, the requester (AM
// for regular containers and RM itself for AM container) will not know what
// happened. Simply add the ResourceRequest back again so that requester
// doesn't need to do anything conditionally.
recoverResourceRequestForContainer(rmContainer);
}
// clean up a completed container
protected abstract void completedContainerInternal(RMContainer rmContainer,
protected abstract void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event);
protected void releaseContainers(List<ContainerId> containers,

View File

@ -288,15 +288,12 @@ public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
* application, by asking for more resources and releasing resources acquired
* by the application.
*
* @param requests
* resources to be acquired
* @param recoverPreemptedRequestForAContainer
* recover ResourceRequest on preemption
* @param requests resources to be acquired
* @param recoverPreemptedRequest recover ResourceRequest on preemption
* @return true if any resource was updated, false otherwise
*/
public synchronized boolean updateResourceRequests(
List<ResourceRequest> requests,
boolean recoverPreemptedRequestForAContainer) {
List<ResourceRequest> requests, boolean recoverPreemptedRequest) {
// Flag to track if any incoming requests update "ANY" requests
boolean anyResourcesUpdated = false;
@ -317,7 +314,7 @@ public synchronized boolean updateResourceRequests(
// Increment number of containers if recovering preempted resources
ResourceRequest lastRequest = asks.get(resourceName);
if (recoverPreemptedRequestForAContainer && lastRequest != null) {
if (recoverPreemptedRequest && lastRequest != null) {
request.setNumContainers(lastRequest.getNumContainers() + 1);
}

View File

@ -16,10 +16,12 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
/**
* Simple event class used to communicate containers unreservations, preemption, killing

View File

@ -31,7 +31,7 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
* ask the scheduler to drop the reservation for the given container.
* @param container Reference to reserved container allocation.
*/
void killReservedContainer(RMContainer container);
void dropContainerReservation(RMContainer container);
/**
* Ask the scheduler to obtain back the container from a specific application
@ -45,6 +45,6 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
* Ask the scheduler to forcibly interrupt the container given as input
* @param container
*/
void killPreemptedContainer(RMContainer container);
void killContainer(RMContainer container);
}

View File

@ -322,7 +322,7 @@ public synchronized boolean updateResourceRequests(
return false;
}
public synchronized void recoverResourceRequestsForContainer(
public synchronized void recoverResourceRequests(
List<ResourceRequest> requests) {
if (!isStopped) {
appSchedulingInfo.updateResourceRequests(requests, true);

View File

@ -92,6 +92,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
@ -113,14 +114,13 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerRescheduledEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
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.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.Lock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -862,7 +862,7 @@ private synchronized void doneApplicationAttempt(
LOG.info("Skip killing " + rmContainer.getContainerId());
continue;
}
super.completedContainer(
completedContainer(
rmContainer,
SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
@ -871,7 +871,7 @@ private synchronized void doneApplicationAttempt(
// Release all reserved containers
for (RMContainer rmContainer : attempt.getReservedContainers()) {
super.completedContainer(
completedContainer(
rmContainer,
SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(), "Application Complete"),
@ -1044,7 +1044,7 @@ private synchronized void nodeUpdate(RMNode nm) {
for (ContainerStatus completedContainer : completedContainers) {
ContainerId containerId = completedContainer.getContainerId();
RMContainer container = getRMContainer(containerId);
super.completedContainer(container, completedContainer,
completedContainer(container, completedContainer,
RMContainerEventType.FINISHED);
if (container != null) {
releasedContainers++;
@ -1125,7 +1125,7 @@ private synchronized void updateLabelsOnNode(NodeId nodeId,
// Unreserve container on this node
RMContainer reservedContainer = node.getReservedContainer();
if (null != reservedContainer) {
killReservedContainer(reservedContainer);
dropContainerReservation(reservedContainer);
}
// Update node labels after we've done this
@ -1369,19 +1369,18 @@ public void handle(SchedulerEvent event) {
ContainerExpiredSchedulerEvent containerExpiredEvent =
(ContainerExpiredSchedulerEvent) event;
ContainerId containerId = containerExpiredEvent.getContainerId();
super.completedContainer(getRMContainer(containerId),
completedContainer(getRMContainer(containerId),
SchedulerUtils.createAbnormalContainerStatus(
containerId,
SchedulerUtils.EXPIRED_CONTAINER),
RMContainerEventType.EXPIRE);
}
break;
case KILL_RESERVED_CONTAINER:
case DROP_RESERVATION:
{
ContainerPreemptEvent killReservedContainerEvent =
(ContainerPreemptEvent) event;
RMContainer container = killReservedContainerEvent.getContainer();
killReservedContainer(container);
ContainerPreemptEvent dropReservationEvent = (ContainerPreemptEvent)event;
RMContainer container = dropReservationEvent.getContainer();
dropContainerReservation(container);
}
break;
case PREEMPT_CONTAINER:
@ -1393,11 +1392,19 @@ public void handle(SchedulerEvent event) {
preemptContainer(aid, containerToBePreempted);
}
break;
case KILL_PREEMPTED_CONTAINER:
case KILL_CONTAINER:
{
ContainerPreemptEvent killContainerEvent = (ContainerPreemptEvent)event;
RMContainer containerToBeKilled = killContainerEvent.getContainer();
killPreemptedContainer(containerToBeKilled);
killContainer(containerToBeKilled);
}
break;
case CONTAINER_RESCHEDULED:
{
ContainerRescheduledEvent containerRescheduledEvent =
(ContainerRescheduledEvent) event;
RMContainer container = containerRescheduledEvent.getContainer();
recoverResourceRequestForContainer(container);
}
break;
default:
@ -1452,7 +1459,7 @@ private synchronized void removeNode(RMNode nodeInfo) {
// Remove running containers
List<RMContainer> runningContainers = node.getRunningContainers();
for (RMContainer container : runningContainers) {
super.completedContainer(container,
completedContainer(container,
SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(),
SchedulerUtils.LOST_CONTAINER),
@ -1462,7 +1469,7 @@ private synchronized void removeNode(RMNode nodeInfo) {
// Remove reservations, if any
RMContainer reservedContainer = node.getReservedContainer();
if (reservedContainer != null) {
super.completedContainer(reservedContainer,
completedContainer(reservedContainer,
SchedulerUtils.createAbnormalContainerStatus(
reservedContainer.getContainerId(),
SchedulerUtils.LOST_CONTAINER),
@ -1478,8 +1485,13 @@ private synchronized void removeNode(RMNode nodeInfo) {
@Lock(CapacityScheduler.class)
@Override
protected synchronized void completedContainerInternal(RMContainer rmContainer,
protected synchronized void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event) {
if (rmContainer == null) {
LOG.info("Container " + containerStatus.getContainerId() +
" completed with event " + event);
return;
}
Container container = rmContainer.getContainer();
@ -1581,14 +1593,11 @@ public void recover(RMState state) throws Exception {
}
@Override
public void killReservedContainer(RMContainer container) {
public void dropContainerReservation(RMContainer container) {
if(LOG.isDebugEnabled()){
LOG.debug(SchedulerEventType.KILL_RESERVED_CONTAINER + ":"
+ container.toString());
LOG.debug("DROP_RESERVATION:" + container.toString());
}
// TODO: What happens if this is no longer a reserved container, for e.g if
// the reservation became an allocation.
super.completedContainer(container,
completedContainer(container,
SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(),
SchedulerUtils.UNRESERVED_CONTAINER),
@ -1598,24 +1607,23 @@ public void killReservedContainer(RMContainer container) {
@Override
public void preemptContainer(ApplicationAttemptId aid, RMContainer cont) {
if(LOG.isDebugEnabled()){
LOG.debug(SchedulerEventType.PREEMPT_CONTAINER + ": appAttempt:"
+ aid.toString() + " container: " + cont.toString());
LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() +
" container: " + cont.toString());
}
FiCaSchedulerApp app = getApplicationAttempt(aid);
if (app != null) {
app.preemptContainer(cont.getContainerId());
app.addPreemptContainer(cont.getContainerId());
}
}
@Override
public void killPreemptedContainer(RMContainer cont) {
public void killContainer(RMContainer cont) {
if (LOG.isDebugEnabled()) {
LOG.debug(SchedulerEventType.KILL_PREEMPTED_CONTAINER + ": container"
+ cont.toString());
LOG.debug("KILL_CONTAINER: container" + cont.toString());
}
super.completedContainer(cont, SchedulerUtils
.createPreemptedContainerStatus(cont.getContainerId(),
SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL);
completedContainer(cont, SchedulerUtils.createPreemptedContainerStatus(
cont.getContainerId(), SchedulerUtils.PREEMPTED_CONTAINER),
RMContainerEventType.KILL);
}
@Override

View File

@ -301,7 +301,7 @@ public synchronized Resource getTotalPendingRequests() {
return ret;
}
public synchronized void preemptContainer(ContainerId cont) {
public synchronized void addPreemptContainer(ContainerId cont) {
// ignore already completed containers
if (liveContainers.containsKey(cont)) {
containersToPreempt.add(cont);

View File

@ -0,0 +1,35 @@
/**
* 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.event;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
public class ContainerRescheduledEvent extends SchedulerEvent {
private RMContainer container;
public ContainerRescheduledEvent(RMContainer container) {
super(SchedulerEventType.CONTAINER_RESCHEDULED);
this.container = container;
}
public RMContainer getContainer() {
return container;
}
}

View File

@ -38,9 +38,11 @@ public enum SchedulerEventType {
// Source: ContainerAllocationExpirer
CONTAINER_EXPIRED,
// Source: RMContainer
CONTAINER_RESCHEDULED,
// Source: SchedulingEditPolicy
KILL_RESERVED_CONTAINER,
PREEMPT_CONTAINER, // Mark a container for preemption in the near future
KILL_PREEMPTED_CONTAINER // Kill a container previously marked for
// preemption
DROP_RESERVATION,
PREEMPT_CONTAINER,
KILL_CONTAINER
}

View File

@ -85,6 +85,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerRescheduledEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
@ -497,7 +498,7 @@ protected void warnOrKillContainer(RMContainer container) {
// TODO: Not sure if this ever actually adds this to the list of cleanup
// containers on the RMNode (see SchedulerNode.releaseContainer()).
super.completedContainer(container, status, RMContainerEventType.KILL);
completedContainer(container, status, RMContainerEventType.KILL);
LOG.info("Killing container" + container +
" (after waiting for preemption for " +
(getClock().getTime() - time) + "ms)");
@ -806,7 +807,7 @@ private synchronized void removeApplicationAttempt(
LOG.info("Skip killing " + rmContainer.getContainerId());
continue;
}
super.completedContainer(rmContainer,
completedContainer(rmContainer,
SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(),
SchedulerUtils.COMPLETED_APPLICATION),
@ -815,7 +816,7 @@ private synchronized void removeApplicationAttempt(
// Release all reserved containers
for (RMContainer rmContainer : attempt.getReservedContainers()) {
super.completedContainer(rmContainer,
completedContainer(rmContainer,
SchedulerUtils.createAbnormalContainerStatus(
rmContainer.getContainerId(),
"Application Complete"),
@ -842,9 +843,13 @@ private synchronized void removeApplicationAttempt(
* Clean up a completed container.
*/
@Override
protected synchronized void completedContainerInternal(
RMContainer rmContainer, ContainerStatus containerStatus,
RMContainerEventType event) {
protected synchronized void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event) {
if (rmContainer == null) {
LOG.info("Container " + containerStatus.getContainerId()
+ " completed with event " + event);
return;
}
Container container = rmContainer.getContainer();
@ -914,7 +919,7 @@ private synchronized void removeNode(RMNode rmNode) {
// Remove running containers
List<RMContainer> runningContainers = node.getRunningContainers();
for (RMContainer container : runningContainers) {
super.completedContainer(container,
completedContainer(container,
SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(),
SchedulerUtils.LOST_CONTAINER),
@ -924,7 +929,7 @@ private synchronized void removeNode(RMNode rmNode) {
// Remove reservations, if any
RMContainer reservedContainer = node.getReservedContainer();
if (reservedContainer != null) {
super.completedContainer(reservedContainer,
completedContainer(reservedContainer,
SchedulerUtils.createAbnormalContainerStatus(
reservedContainer.getContainerId(),
SchedulerUtils.LOST_CONTAINER),
@ -1052,7 +1057,7 @@ private synchronized void nodeUpdate(RMNode nm) {
for (ContainerStatus completedContainer : completedContainers) {
ContainerId containerId = completedContainer.getContainerId();
LOG.debug("Container FINISHED: " + containerId);
super.completedContainer(getRMContainer(containerId),
completedContainer(getRMContainer(containerId),
completedContainer, RMContainerEventType.FINISHED);
}
@ -1297,12 +1302,21 @@ public void handle(SchedulerEvent event) {
ContainerExpiredSchedulerEvent containerExpiredEvent =
(ContainerExpiredSchedulerEvent)event;
ContainerId containerId = containerExpiredEvent.getContainerId();
super.completedContainer(getRMContainer(containerId),
completedContainer(getRMContainer(containerId),
SchedulerUtils.createAbnormalContainerStatus(
containerId,
SchedulerUtils.EXPIRED_CONTAINER),
RMContainerEventType.EXPIRE);
break;
case CONTAINER_RESCHEDULED:
if (!(event instanceof ContainerRescheduledEvent)) {
throw new RuntimeException("Unexpected event type: " + event);
}
ContainerRescheduledEvent containerRescheduledEvent =
(ContainerRescheduledEvent) event;
RMContainer container = containerRescheduledEvent.getContainer();
recoverResourceRequestForContainer(container);
break;
default:
LOG.error("Unknown event arrived at FairScheduler: " + event.toString());
}

View File

@ -74,10 +74,10 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
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.SchedContainerChangeRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@ -86,6 +86,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerRescheduledEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
@ -467,7 +468,7 @@ private synchronized void doneApplicationAttempt(
LOG.info("Skip killing " + container.getContainerId());
continue;
}
super.completedContainer(container,
completedContainer(container,
SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
RMContainerEventType.KILL);
@ -738,7 +739,7 @@ private synchronized void nodeUpdate(RMNode rmNode) {
for (ContainerStatus completedContainer : completedContainers) {
ContainerId containerId = completedContainer.getContainerId();
LOG.debug("Container FINISHED: " + containerId);
super.completedContainer(getRMContainer(containerId),
completedContainer(getRMContainer(containerId),
completedContainer, RMContainerEventType.FINISHED);
}
@ -857,13 +858,21 @@ public void handle(SchedulerEvent event) {
ContainerExpiredSchedulerEvent containerExpiredEvent =
(ContainerExpiredSchedulerEvent) event;
ContainerId containerid = containerExpiredEvent.getContainerId();
super.completedContainer(getRMContainer(containerid),
completedContainer(getRMContainer(containerid),
SchedulerUtils.createAbnormalContainerStatus(
containerid,
SchedulerUtils.EXPIRED_CONTAINER),
RMContainerEventType.EXPIRE);
}
break;
case CONTAINER_RESCHEDULED:
{
ContainerRescheduledEvent containerRescheduledEvent =
(ContainerRescheduledEvent) event;
RMContainer container = containerRescheduledEvent.getContainer();
recoverResourceRequestForContainer(container);
}
break;
default:
LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
}
@ -871,9 +880,12 @@ public void handle(SchedulerEvent event) {
@Lock(FifoScheduler.class)
@Override
protected synchronized void completedContainerInternal(
RMContainer rmContainer, ContainerStatus containerStatus,
RMContainerEventType event) {
protected synchronized void completedContainer(RMContainer rmContainer,
ContainerStatus containerStatus, RMContainerEventType event) {
if (rmContainer == null) {
LOG.info("Null container completed...");
return;
}
// Get the application for the finished container
Container container = rmContainer.getContainer();
@ -919,7 +931,7 @@ private synchronized void removeNode(RMNode nodeInfo) {
}
// Kill running containers
for(RMContainer container : node.getRunningContainers()) {
super.completedContainer(container,
completedContainer(container,
SchedulerUtils.createAbnormalContainerStatus(
container.getContainerId(),
SchedulerUtils.LOST_CONTAINER),

View File

@ -275,8 +275,7 @@ public boolean waitForState(Collection<MockNM> nms, ContainerId containerId,
nm.nodeHeartbeat(true);
}
container = getResourceScheduler().getRMContainer(containerId);
System.out.println("Waiting for container " + containerId + " to be "
+ containerState + ", container is null right now.");
System.out.println("Waiting for container " + containerId + " to be allocated.");
Thread.sleep(100);
if (timeoutMillisecs <= timeoutSecs * 100) {

View File

@ -29,8 +29,8 @@
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.SchedulerEventDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.junit.Assert;
@ -55,11 +55,10 @@ public void testSchedulerEventDispatcherForPreemptionEvents() {
ApplicationAttemptId appAttemptId = mock(ApplicationAttemptId.class);
RMContainer container = mock(RMContainer.class);
ContainerPreemptEvent event1 = new ContainerPreemptEvent(
appAttemptId, container, SchedulerEventType.KILL_RESERVED_CONTAINER);
appAttemptId, container, SchedulerEventType.DROP_RESERVATION);
rmDispatcher.getEventHandler().handle(event1);
ContainerPreemptEvent event2 =
new ContainerPreemptEvent(appAttemptId, container,
SchedulerEventType.KILL_PREEMPTED_CONTAINER);
ContainerPreemptEvent event2 = new ContainerPreemptEvent(
appAttemptId, container, SchedulerEventType.KILL_CONTAINER);
rmDispatcher.getEventHandler().handle(event2);
ContainerPreemptEvent event3 = new ContainerPreemptEvent(
appAttemptId, container, SchedulerEventType.PREEMPT_CONTAINER);
@ -67,9 +66,9 @@ public void testSchedulerEventDispatcherForPreemptionEvents() {
// Wait for events to be processed by scheduler dispatcher.
Thread.sleep(1000);
verify(sched, times(3)).handle(any(SchedulerEvent.class));
verify(sched).killReservedContainer(container);
verify(sched).dropContainerReservation(container);
verify(sched).preemptContainer(appAttemptId, container);
verify(sched).killPreemptedContainer(container);
verify(sched).killContainer(container);
} catch (InterruptedException e) {
Assert.fail();
} finally {

View File

@ -564,7 +564,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception {
ContainerId amContainer =
ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
// Preempt the first attempt;
scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
scheduler.killContainer(scheduler.getRMContainer(amContainer));
am1.waitForState(RMAppAttemptState.FAILED);
Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
@ -580,7 +580,7 @@ public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception {
// Preempt the second attempt.
ContainerId amContainer2 =
ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer2));
scheduler.killContainer(scheduler.getRMContainer(amContainer2));
am2.waitForState(RMAppAttemptState.FAILED);
Assert.assertTrue(! attempt2.shouldCountTowardsMaxAttemptRetry());
@ -675,7 +675,7 @@ public void testPreemptedAMRestartOnRMRestart() throws Exception {
ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
// Forcibly preempt the am container;
scheduler.killPreemptedContainer(scheduler.getRMContainer(amContainer));
scheduler.killContainer(scheduler.getRMContainer(amContainer));
am1.waitForState(RMAppAttemptState.FAILED);
Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());

View File

@ -23,7 +23,7 @@
import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.OBSERVE_ONLY;
import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.TOTAL_PREEMPTION_PER_ROUND;
import static org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy.WAIT_TIME_BEFORE_KILL;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_PREEMPTED_CONTAINER;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.KILL_CONTAINER;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType.PREEMPT_CONTAINER;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
@ -67,6 +67,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Priority;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
@ -76,7 +77,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@ -289,7 +289,7 @@ public void testExpireKill() {
List<ContainerPreemptEvent> events = evtCaptor.getAllValues();
for (ContainerPreemptEvent e : events.subList(20, 20)) {
assertEquals(appC, e.getAppId());
assertEquals(KILL_PREEMPTED_CONTAINER, e.getType());
assertEquals(KILL_CONTAINER, e.getType());
}
}

View File

@ -29,7 +29,6 @@
import java.util.Map;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
@ -47,15 +46,11 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
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.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.Assert;
import org.junit.Test;
@ -498,114 +493,6 @@ public void testResourceRequestRestoreWhenRMContainerIsAtAllocated()
}
}
/**
* Test to verify that ResourceRequests recovery back to the right app-attempt
* after a container gets killed at ACQUIRED state: YARN-4502.
*
* @throws Exception
*/
@Test
public void testResourceRequestRecoveryToTheRightAppAttempt()
throws Exception {
configureScheduler();
YarnConfiguration conf = getConf();
MockRM rm = new MockRM(conf);
try {
rm.start();
RMApp rmApp =
rm.submitApp(200, "name", "user",
new HashMap<ApplicationAccessType, String>(), false, "default", -1,
null, "Test", false, true);
MockNM node =
new MockNM("127.0.0.1:1234", 10240, rm.getResourceTrackerService());
node.registerNode();
MockAM am1 = MockRM.launchAndRegisterAM(rmApp, rm, node);
ApplicationAttemptId applicationAttemptOneID =
am1.getApplicationAttemptId();
ContainerId am1ContainerID =
ContainerId.newContainerId(applicationAttemptOneID, 1);
// allocate NUM_CONTAINERS containers
am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
node.nodeHeartbeat(true);
// wait for containers to be allocated.
List<Container> containers =
am1.allocate(new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>()).getAllocatedContainers();
while (containers.size() != 1) {
node.nodeHeartbeat(true);
containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>()).getAllocatedContainers());
Thread.sleep(200);
}
// launch a 2nd container, for testing running-containers transfer.
node.nodeHeartbeat(applicationAttemptOneID, 2, ContainerState.RUNNING);
ContainerId runningContainerID =
ContainerId.newContainerId(applicationAttemptOneID, 2);
rm.waitForState(node, runningContainerID, RMContainerState.RUNNING);
// 3rd container is in Allocated state.
int ALLOCATED_CONTAINER_PRIORITY = 1047;
am1.allocate("127.0.0.1", 1024, 1, ALLOCATED_CONTAINER_PRIORITY,
new ArrayList<ContainerId>(), null);
node.nodeHeartbeat(true);
ContainerId allocatedContainerID =
ContainerId.newContainerId(applicationAttemptOneID, 3);
rm.waitForContainerAllocated(node, allocatedContainerID);
rm.waitForState(node, allocatedContainerID, RMContainerState.ALLOCATED);
RMContainer allocatedContainer =
rm.getResourceScheduler().getRMContainer(allocatedContainerID);
// Capture scheduler app-attempt before AM crash.
SchedulerApplicationAttempt firstSchedulerAppAttempt =
((AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>) rm
.getResourceScheduler())
.getApplicationAttempt(applicationAttemptOneID);
// AM crashes, and a new app-attempt gets created
node.nodeHeartbeat(applicationAttemptOneID, 1, ContainerState.COMPLETE);
rm.waitForState(node, am1ContainerID, RMContainerState.COMPLETED);
RMAppAttempt rmAppAttempt2 = MockRM.waitForAttemptScheduled(rmApp, rm);
ApplicationAttemptId applicationAttemptTwoID =
rmAppAttempt2.getAppAttemptId();
Assert.assertEquals(2, applicationAttemptTwoID.getAttemptId());
// All outstanding allocated containers will be killed (irrespective of
// keep-alive of container across app-attempts)
Assert.assertEquals(RMContainerState.KILLED,
allocatedContainer.getState());
// The core part of this test
// The killed containers' ResourceRequests are recovered back to the
// original app-attempt, not the new one
for (ResourceRequest request : firstSchedulerAppAttempt
.getAppSchedulingInfo().getAllResourceRequests()) {
if (request.getPriority().getPriority() == 0) {
Assert.assertEquals(0, request.getNumContainers());
} else if (request.getPriority().getPriority() == ALLOCATED_CONTAINER_PRIORITY) {
Assert.assertEquals(1, request.getNumContainers());
}
}
// Also, only one running container should be transferred after AM
// launches
MockRM.launchAM(rmApp, rm, node);
List<Container> transferredContainers =
rm.getResourceScheduler().getTransferredContainers(
applicationAttemptTwoID);
Assert.assertEquals(1, transferredContainers.size());
Assert.assertEquals(runningContainerID, transferredContainers.get(0)
.getId());
} finally {
rm.stop();
}
}
private void verifyMaximumResourceCapability(
Resource expectedMaximumResource, YarnScheduler scheduler) {

View File

@ -205,7 +205,7 @@ public void testApplicationPriorityAllocation() throws Exception {
if (++counter > 2) {
break;
}
cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
cs.killContainer(schedulerAppAttempt.getRMContainer(c.getId()));
}
// check node report, 12 GB used and 4 GB available
@ -513,7 +513,7 @@ public void testApplicationPriorityAllocationWithChangeInPriority()
if (++counter > 2) {
break;
}
cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
cs.killContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
iterator.remove();
}
@ -543,7 +543,7 @@ public void testApplicationPriorityAllocationWithChangeInPriority()
if (++counter > 1) {
break;
}
cs.killPreemptedContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
cs.killContainer(schedulerAppAttemptApp1.getRMContainer(c.getId()));
iterator.remove();
}

View File

@ -1170,7 +1170,7 @@ public void testPreemptionInfo() throws Exception {
// kill the 3 containers
for (Container c : allocatedContainers) {
cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
cs.killContainer(schedulerAppAttempt.getRMContainer(c.getId()));
}
// check values
@ -1179,7 +1179,7 @@ public void testPreemptionInfo() throws Exception {
Resource.newInstance(CONTAINER_MEMORY * 3, 3), false, 3);
// kill app0-attempt0 AM container
cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(app0
cs.killContainer(schedulerAppAttempt.getRMContainer(app0
.getCurrentAppAttempt().getMasterContainer().getId()));
// wait for app0 failed
@ -1202,7 +1202,7 @@ public void testPreemptionInfo() throws Exception {
allocatedContainers =
am1.allocateAndWaitForContainers(3, CONTAINER_MEMORY, nm1);
for (Container c : allocatedContainers) {
cs.killPreemptedContainer(schedulerAppAttempt.getRMContainer(c.getId()));
cs.killContainer(schedulerAppAttempt.getRMContainer(c.getId()));
}
// check values
@ -1251,7 +1251,7 @@ public void testRecoverRequestAfterPreemption() throws Exception {
}
// Call killContainer to preempt the container
cs.killPreemptedContainer(rmContainer);
cs.killContainer(rmContainer);
Assert.assertEquals(3, requests.size());
for (ResourceRequest request : requests) {

View File

@ -18,11 +18,13 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.isA;
@ -50,7 +52,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.GroupMappingServiceProvider;
import org.apache.hadoop.yarn.MockApps;
@ -94,11 +95,10 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerPreemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerRescheduledEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.Default;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
@ -4779,7 +4779,7 @@ public void testRecoverRequestAfterPreemption() throws Exception {
assertEquals(1, scheduler.getSchedulerApp(appAttemptId).getLiveContainers()
.size());
SchedulerApplicationAttempt app = scheduler.getSchedulerApp(appAttemptId);
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
// ResourceRequest will be empty once NodeUpdate is completed
Assert.assertNull(app.getResourceRequest(priority, host));
@ -4797,8 +4797,7 @@ public void testRecoverRequestAfterPreemption() throws Exception {
scheduler.warnOrKillContainer(rmContainer);
// Trigger container rescheduled event
scheduler.handle(new ContainerPreemptEvent(appAttemptId, rmContainer,
SchedulerEventType.KILL_PREEMPTED_CONTAINER));
scheduler.handle(new ContainerRescheduledEvent(rmContainer));
List<ResourceRequest> requests = rmContainer.getResourceRequests();
// Once recovered, resource request will be present again in app
@ -4821,6 +4820,7 @@ Collections.<ResourceRequest> emptyList(),
Assert.assertTrue(containers.size() == 1);
}
@SuppressWarnings("resource")
@Test
public void testBlacklistNodes() throws Exception {
scheduler.init(conf);