YARN-8423. GPU does not get released even though the application gets killed. (Sunil G via wangda)
Change-Id: I570db7d60f8c6c21762dd618a9207d1107c486a0
(cherry picked from commit ada8f63d0b
)
This commit is contained in:
parent
aa8db00fc3
commit
8f226f4f1d
|
@ -105,4 +105,10 @@ public interface Container extends EventHandler<ContainerEvent> {
|
|||
ResourceMappings getResourceMappings();
|
||||
|
||||
void sendPauseEvent(String description);
|
||||
|
||||
/**
|
||||
* Verify container is in final states.
|
||||
* @return true/false based on container's state
|
||||
*/
|
||||
boolean isContainerInFinalStates();
|
||||
}
|
||||
|
|
|
@ -2218,4 +2218,15 @@ public class ContainerImpl implements Container {
|
|||
SlidingWindowRetryPolicy getRetryPolicy() {
|
||||
return retryPolicy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isContainerInFinalStates() {
|
||||
ContainerState state = getContainerState();
|
||||
return state == ContainerState.KILLING || state == ContainerState.DONE
|
||||
|| state == ContainerState.LOCALIZATION_FAILED
|
||||
|| state == ContainerState.CONTAINER_RESOURCES_CLEANINGUP
|
||||
|| state == ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL
|
||||
|| state == ContainerState.EXITED_WITH_FAILURE
|
||||
|| state == ContainerState.EXITED_WITH_SUCCESS;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
||||
import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
|
@ -36,10 +37,8 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin
|
|||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -54,6 +53,7 @@ import static org.apache.hadoop.yarn.api.records.ResourceInformation.GPU_URI;
|
|||
*/
|
||||
public class GpuResourceAllocator {
|
||||
final static Log LOG = LogFactory.getLog(GpuResourceAllocator.class);
|
||||
private static final int WAIT_MS_PER_LOOP = 1000;
|
||||
|
||||
private Set<GpuDevice> allowedGpuDevices = new TreeSet<>();
|
||||
private Map<GpuDevice, ContainerId> usedDevices = new TreeMap<>();
|
||||
|
@ -168,13 +168,58 @@ public class GpuResourceAllocator {
|
|||
* @return allocation results.
|
||||
* @throws ResourceHandlerException When failed to assign GPUs.
|
||||
*/
|
||||
public synchronized GpuAllocation assignGpus(Container container)
|
||||
public GpuAllocation assignGpus(Container container)
|
||||
throws ResourceHandlerException {
|
||||
GpuAllocation allocation = internalAssignGpus(container);
|
||||
|
||||
// Wait for a maximum of 120 seconds if no available GPU are there which
|
||||
// are yet to be released.
|
||||
final int timeoutMsecs = 120 * WAIT_MS_PER_LOOP;
|
||||
int timeWaiting = 0;
|
||||
while (allocation == null) {
|
||||
if (timeWaiting >= timeoutMsecs) {
|
||||
break;
|
||||
}
|
||||
|
||||
// Sleep for 1 sec to ensure there are some free GPU devices which are
|
||||
// getting released.
|
||||
try {
|
||||
LOG.info("Container : " + container.getContainerId()
|
||||
+ " is waiting for free GPU devices.");
|
||||
Thread.sleep(WAIT_MS_PER_LOOP);
|
||||
timeWaiting += WAIT_MS_PER_LOOP;
|
||||
allocation = internalAssignGpus(container);
|
||||
} catch (InterruptedException e) {
|
||||
// On any interrupt, break the loop and continue execution.
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if(allocation == null) {
|
||||
String message = "Could not get valid GPU device for container '" +
|
||||
container.getContainerId()
|
||||
+ "' as some other containers might not releasing GPUs.";
|
||||
LOG.warn(message);
|
||||
throw new ResourceHandlerException(message);
|
||||
}
|
||||
return allocation;
|
||||
}
|
||||
|
||||
private synchronized GpuAllocation internalAssignGpus(Container container)
|
||||
throws ResourceHandlerException {
|
||||
Resource requestedResource = container.getResource();
|
||||
ContainerId containerId = container.getContainerId();
|
||||
int numRequestedGpuDevices = getRequestedGpus(requestedResource);
|
||||
// Assign Gpus to container if requested some.
|
||||
if (numRequestedGpuDevices > 0) {
|
||||
if (numRequestedGpuDevices > getAvailableGpus()) {
|
||||
// If there are some devices which are getting released, wait for few
|
||||
// seconds to get it.
|
||||
if (numRequestedGpuDevices <= getReleasingGpus() + getAvailableGpus()) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
if (numRequestedGpuDevices > getAvailableGpus()) {
|
||||
throw new ResourceHandlerException(
|
||||
getResourceHandlerExceptionMessage(numRequestedGpuDevices,
|
||||
|
@ -211,6 +256,23 @@ public class GpuResourceAllocator {
|
|||
return new GpuAllocation(null, allowedGpuDevices);
|
||||
}
|
||||
|
||||
private synchronized long getReleasingGpus() {
|
||||
long releasingGpus = 0;
|
||||
Iterator<Map.Entry<GpuDevice, ContainerId>> iter = usedDevices.entrySet()
|
||||
.iterator();
|
||||
while (iter.hasNext()) {
|
||||
ContainerId containerId = iter.next().getValue();
|
||||
Container container;
|
||||
if ((container = nmContext.getContainers().get(containerId)) != null) {
|
||||
if (container.isContainerInFinalStates()) {
|
||||
releasingGpus = releasingGpus + container.getResource()
|
||||
.getResourceInformation(ResourceInformation.GPU_URI).getValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
return releasingGpus;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up all Gpus assigned to containerId
|
||||
* @param containerId containerId
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.gpu;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
|
|
@ -250,4 +250,7 @@ public class MockContainer implements Container {
|
|||
public void sendPauseEvent(String description) {
|
||||
|
||||
}
|
||||
@Override public boolean isContainerInFinalStates() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue