|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.gpu;
|
|
|
|
|
|
+import com.google.common.collect.ImmutableMap;
|
|
|
+import com.google.common.collect.Maps;
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -28,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
|
|
@@ -43,9 +46,10 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
|
|
|
import org.junit.After;
|
|
|
import org.apache.hadoop.yarn.util.resource.TestResourceUtils;
|
|
|
-import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.ExpectedException;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.FileOutputStream;
|
|
@@ -53,11 +57,13 @@ 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.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.anyList;
|
|
|
import static org.mockito.Matchers.anyString;
|
|
@@ -65,7 +71,6 @@ import static org.mockito.Matchers.eq;
|
|
|
import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.never;
|
|
|
-import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
@@ -103,11 +108,14 @@ public class TestGpuResourceHandler {
|
|
|
}
|
|
|
|
|
|
private File setupFakeGpuDiscoveryBinary() throws IOException {
|
|
|
- File fakeBinary = new File(getTestParentDirectory() + "/fake-nvidia-smi");
|
|
|
+ File fakeBinary = new File(getTestParentDirectory() + "/nvidia-smi");
|
|
|
touchFile(fakeBinary);
|
|
|
return fakeBinary;
|
|
|
}
|
|
|
|
|
|
+ @Rule
|
|
|
+ public ExpectedException expected = ExpectedException.none();
|
|
|
+
|
|
|
@Before
|
|
|
public void setup() throws IOException {
|
|
|
createTestDataDirectory();
|
|
@@ -119,16 +127,20 @@ public class TestGpuResourceHandler {
|
|
|
mockNMStateStore = mock(NMStateStoreService.class);
|
|
|
|
|
|
Configuration conf = new Configuration();
|
|
|
+ Context nmContext = createMockNmContext(conf);
|
|
|
+
|
|
|
+ gpuDiscoverer = new GpuDiscoverer();
|
|
|
+ gpuResourceHandler = new GpuResourceHandlerImpl(nmContext,
|
|
|
+ mockCGroupsHandler, mockPrivilegedExecutor, gpuDiscoverer);
|
|
|
+ }
|
|
|
|
|
|
+ private Context createMockNmContext(Configuration conf) {
|
|
|
Context nmctx = mock(Context.class);
|
|
|
when(nmctx.getNMStateStore()).thenReturn(mockNMStateStore);
|
|
|
when(nmctx.getConf()).thenReturn(conf);
|
|
|
runningContainersMap = new ConcurrentHashMap<>();
|
|
|
when(nmctx.getContainers()).thenReturn(runningContainersMap);
|
|
|
-
|
|
|
- gpuDiscoverer = new GpuDiscoverer();
|
|
|
- gpuResourceHandler = new GpuResourceHandlerImpl(nmctx, mockCGroupsHandler,
|
|
|
- mockPrivilegedExecutor, gpuDiscoverer);
|
|
|
+ return nmctx;
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -137,54 +149,63 @@ public class TestGpuResourceHandler {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testBootStrap() throws Exception {
|
|
|
+ public void testBootstrapWithRealGpuDiscoverer() throws Exception {
|
|
|
Configuration conf = createDefaultConfig();
|
|
|
conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0");
|
|
|
-
|
|
|
gpuDiscoverer.initialize(conf);
|
|
|
|
|
|
gpuResourceHandler.bootstrap(conf);
|
|
|
- verify(mockCGroupsHandler, times(1)).initializeCGroupController(
|
|
|
+
|
|
|
+ List<GpuDevice> allowedGpus =
|
|
|
+ gpuResourceHandler.getGpuAllocator().getAllowedGpusCopy();
|
|
|
+ assertEquals("Unexpected number of allowed GPU devices!", 1,
|
|
|
+ allowedGpus.size());
|
|
|
+ assertEquals("Expected GPU device does not equal to found device!",
|
|
|
+ new GpuDevice(0, 0), allowedGpus.get(0));
|
|
|
+ verify(mockCGroupsHandler).initializeCGroupController(
|
|
|
CGroupsHandler.CGroupController.DEVICES);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testBootstrapWithMockGpuDiscoverer() throws Exception {
|
|
|
+ GpuDiscoverer mockDiscoverer = mock(GpuDiscoverer.class);
|
|
|
+ Configuration conf = new YarnConfiguration();
|
|
|
+ mockDiscoverer.initialize(conf);
|
|
|
+
|
|
|
+ expected.expect(ResourceHandlerException.class);
|
|
|
+ gpuResourceHandler.bootstrap(conf);
|
|
|
+ }
|
|
|
+
|
|
|
private static ContainerId getContainerId(int id) {
|
|
|
return ContainerId.newContainerId(ApplicationAttemptId
|
|
|
.newInstance(ApplicationId.newInstance(1234L, 1), 1), id);
|
|
|
}
|
|
|
|
|
|
- private static Container mockContainerWithGpuRequest(int id, int numGpuRequest,
|
|
|
- boolean dockerContainerEnabled) {
|
|
|
+ private static Container mockContainerWithGpuRequest(int id, Resource res,
|
|
|
+ ContainerLaunchContext launchContext) {
|
|
|
Container c = mock(Container.class);
|
|
|
when(c.getContainerId()).thenReturn(getContainerId(id));
|
|
|
+ when(c.getResource()).thenReturn(res);
|
|
|
+ when(c.getResourceMappings()).thenReturn(new ResourceMappings());
|
|
|
+ when(c.getLaunchContext()).thenReturn(launchContext);
|
|
|
+ return c;
|
|
|
+ }
|
|
|
|
|
|
+ private static Resource createResourceRequest(int numGpuRequest) {
|
|
|
Resource res = Resource.newInstance(1024, 1);
|
|
|
- ResourceMappings resMapping = new ResourceMappings();
|
|
|
-
|
|
|
res.setResourceValue(ResourceInformation.GPU_URI, numGpuRequest);
|
|
|
- when(c.getResource()).thenReturn(res);
|
|
|
- when(c.getResourceMappings()).thenReturn(resMapping);
|
|
|
-
|
|
|
- ContainerLaunchContext clc = mock(ContainerLaunchContext.class);
|
|
|
- Map<String, String> env = new HashMap<>();
|
|
|
- if (dockerContainerEnabled) {
|
|
|
- env.put(ContainerRuntimeConstants.ENV_CONTAINER_TYPE,
|
|
|
- ContainerRuntimeConstants.CONTAINER_RUNTIME_DOCKER);
|
|
|
- }
|
|
|
- when(clc.getEnvironment()).thenReturn(env);
|
|
|
- when(c.getLaunchContext()).thenReturn(clc);
|
|
|
- return c;
|
|
|
+ return res;
|
|
|
}
|
|
|
|
|
|
private static Container mockContainerWithGpuRequest(int id,
|
|
|
- int numGpuRequest) {
|
|
|
- return mockContainerWithGpuRequest(id, numGpuRequest, false);
|
|
|
+ Resource res) {
|
|
|
+ return mockContainerWithGpuRequest(id, res, createLaunchContext());
|
|
|
}
|
|
|
|
|
|
private void verifyDeniedDevices(ContainerId containerId,
|
|
|
List<GpuDevice> deniedDevices)
|
|
|
throws ResourceHandlerException, PrivilegedOperationException {
|
|
|
- verify(mockCGroupsHandler, times(1)).createCGroup(
|
|
|
+ verify(mockCGroupsHandler).createCGroup(
|
|
|
CGroupsHandler.CGroupController.DEVICES, containerId.toString());
|
|
|
|
|
|
if (null != deniedDevices && !deniedDevices.isEmpty()) {
|
|
@@ -192,7 +213,7 @@ public class TestGpuResourceHandler {
|
|
|
for (GpuDevice deniedDevice : deniedDevices) {
|
|
|
deniedDevicesMinorNumber.add(deniedDevice.getMinorNumber());
|
|
|
}
|
|
|
- verify(mockPrivilegedExecutor, times(1)).executePrivilegedOperation(
|
|
|
+ verify(mockPrivilegedExecutor).executePrivilegedOperation(
|
|
|
new PrivilegedOperation(PrivilegedOperation.OperationType.GPU, Arrays
|
|
|
.asList(GpuResourceHandlerImpl.CONTAINER_ID_CLI_OPTION,
|
|
|
containerId.toString(),
|
|
@@ -201,104 +222,139 @@ public class TestGpuResourceHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void commonTestAllocation(boolean dockerContainerEnabled)
|
|
|
- throws Exception {
|
|
|
+ private static ContainerLaunchContext createLaunchContextDocker() {
|
|
|
+ ContainerLaunchContext launchContext = mock(ContainerLaunchContext.class);
|
|
|
+ ImmutableMap<String, String> env = ImmutableMap.<String, String>builder()
|
|
|
+ .put(ContainerRuntimeConstants.ENV_CONTAINER_TYPE,
|
|
|
+ ContainerRuntimeConstants.CONTAINER_RUNTIME_DOCKER)
|
|
|
+ .build();
|
|
|
+ when(launchContext.getEnvironment()).thenReturn(env);
|
|
|
+ return launchContext;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static ContainerLaunchContext createLaunchContext() {
|
|
|
+ ContainerLaunchContext launchContext = mock(ContainerLaunchContext.class);
|
|
|
+ when(launchContext.getEnvironment()).thenReturn(Maps.newHashMap());
|
|
|
+ return launchContext;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void startContainerWithGpuRequestsDocker(int id, int gpus)
|
|
|
+ throws ResourceHandlerException {
|
|
|
+ gpuResourceHandler.preStart(
|
|
|
+ mockContainerWithGpuRequest(id, createResourceRequest(gpus),
|
|
|
+ createLaunchContextDocker()));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void startContainerWithGpuRequests(int id, int gpus)
|
|
|
+ throws ResourceHandlerException {
|
|
|
+ gpuResourceHandler.preStart(
|
|
|
+ mockContainerWithGpuRequest(id, createResourceRequest(gpus),
|
|
|
+ createLaunchContext()));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyNumberOfAvailableGpus(int expectedAvailable,
|
|
|
+ GpuResourceHandlerImpl resourceHandler) {
|
|
|
+ assertEquals("Unexpected number of available GPU devices!",
|
|
|
+ expectedAvailable,
|
|
|
+ resourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyCgroupsDeletedForContainer(int i)
|
|
|
+ throws ResourceHandlerException {
|
|
|
+ verify(mockCGroupsHandler).createCGroup(
|
|
|
+ CGroupsHandler.CGroupController.DEVICES, getContainerId(i).toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void initializeGpus() throws YarnException, IOException {
|
|
|
Configuration conf = createDefaultConfig();
|
|
|
conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
|
|
|
+
|
|
|
+ gpuDiscoverer = new GpuDiscoverer();
|
|
|
gpuDiscoverer.initialize(conf);
|
|
|
+ Context nmContext = createMockNmContext(conf);
|
|
|
+ gpuResourceHandler = new GpuResourceHandlerImpl(nmContext,
|
|
|
+ mockCGroupsHandler, mockPrivilegedExecutor, gpuDiscoverer);
|
|
|
|
|
|
gpuResourceHandler.bootstrap(conf);
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ verifyNumberOfAvailableGpus(4, gpuResourceHandler);
|
|
|
+ }
|
|
|
|
|
|
- /* Start container 1, asks 3 containers */
|
|
|
- gpuResourceHandler.preStart(
|
|
|
- mockContainerWithGpuRequest(1, 3, dockerContainerEnabled));
|
|
|
+ @Test
|
|
|
+ public void testAllocationWhenDockerContainerEnabled() throws Exception {
|
|
|
+ // When docker container is enabled, no devices should be written to
|
|
|
+ // devices.deny.
|
|
|
+ initializeGpus();
|
|
|
|
|
|
- // Only device=4 will be blocked.
|
|
|
- if (dockerContainerEnabled) {
|
|
|
- verifyDeniedDevices(getContainerId(1), Collections.emptyList());
|
|
|
- } else{
|
|
|
- verifyDeniedDevices(getContainerId(1), Arrays.asList(new GpuDevice(3,4)));
|
|
|
- }
|
|
|
+ startContainerWithGpuRequestsDocker(1, 3);
|
|
|
+ verifyDeniedDevices(getContainerId(1), Collections.emptyList());
|
|
|
|
|
|
/* Start container 2, asks 2 containers. Excepted to fail */
|
|
|
boolean failedToAllocate = false;
|
|
|
try {
|
|
|
- gpuResourceHandler.preStart(
|
|
|
- mockContainerWithGpuRequest(2, 2, dockerContainerEnabled));
|
|
|
+ startContainerWithGpuRequestsDocker(2, 2);
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
failedToAllocate = true;
|
|
|
}
|
|
|
- Assert.assertTrue(failedToAllocate);
|
|
|
-
|
|
|
- /* Start container 3, ask 1 container, succeeded */
|
|
|
- gpuResourceHandler.preStart(
|
|
|
- mockContainerWithGpuRequest(3, 1, dockerContainerEnabled));
|
|
|
-
|
|
|
- // devices = 0/1/3 will be blocked
|
|
|
- if (dockerContainerEnabled) {
|
|
|
- verifyDeniedDevices(getContainerId(3), Collections.emptyList());
|
|
|
- } else {
|
|
|
- verifyDeniedDevices(getContainerId(3), Arrays
|
|
|
- .asList(new GpuDevice(0, 0), new GpuDevice(1, 1),
|
|
|
- new GpuDevice(2, 3)));
|
|
|
- }
|
|
|
+ assertTrue("Container allocation is expected to fail!", failedToAllocate);
|
|
|
|
|
|
+ startContainerWithGpuRequestsDocker(3, 1);
|
|
|
+ verifyDeniedDevices(getContainerId(3), Collections.emptyList());
|
|
|
|
|
|
- /* Start container 4, ask 0 container, succeeded */
|
|
|
- gpuResourceHandler.preStart(
|
|
|
- mockContainerWithGpuRequest(4, 0, dockerContainerEnabled));
|
|
|
-
|
|
|
- if (dockerContainerEnabled) {
|
|
|
- verifyDeniedDevices(getContainerId(4), Collections.emptyList());
|
|
|
- } else{
|
|
|
- // All devices will be blocked
|
|
|
- verifyDeniedDevices(getContainerId(4), Arrays
|
|
|
- .asList(new GpuDevice(0, 0), new GpuDevice(1, 1), new GpuDevice(2, 3),
|
|
|
- new GpuDevice(3, 4)));
|
|
|
- }
|
|
|
+ startContainerWithGpuRequestsDocker(4, 0);
|
|
|
+ verifyDeniedDevices(getContainerId(4), Collections.emptyList());
|
|
|
|
|
|
- /* Release container-1, expect cgroups deleted */
|
|
|
gpuResourceHandler.postComplete(getContainerId(1));
|
|
|
+ verifyCgroupsDeletedForContainer(1);
|
|
|
+ verifyNumberOfAvailableGpus(3, gpuResourceHandler);
|
|
|
|
|
|
- verify(mockCGroupsHandler, times(1)).createCGroup(
|
|
|
- CGroupsHandler.CGroupController.DEVICES, getContainerId(1).toString());
|
|
|
- Assert.assertEquals(3,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
-
|
|
|
- /* Release container-3, expect cgroups deleted */
|
|
|
gpuResourceHandler.postComplete(getContainerId(3));
|
|
|
-
|
|
|
- verify(mockCGroupsHandler, times(1)).createCGroup(
|
|
|
- CGroupsHandler.CGroupController.DEVICES, getContainerId(3).toString());
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
- }
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testAllocationWhenDockerContainerEnabled() throws Exception {
|
|
|
- // When docker container is enabled, no devices should be written to
|
|
|
- // devices.deny.
|
|
|
- commonTestAllocation(true);
|
|
|
+ verifyCgroupsDeletedForContainer(3);
|
|
|
+ verifyNumberOfAvailableGpus(4, gpuResourceHandler);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testAllocation() throws Exception {
|
|
|
- commonTestAllocation(false);
|
|
|
+ initializeGpus();
|
|
|
+
|
|
|
+ //Start container 1, asks 3 containers --> Only device=4 will be blocked.
|
|
|
+ startContainerWithGpuRequests(1, 3);
|
|
|
+ verifyDeniedDevices(getContainerId(1),
|
|
|
+ Collections.singletonList(new GpuDevice(3, 4)));
|
|
|
+
|
|
|
+ /* Start container 2, asks 2 containers. Excepted to fail */
|
|
|
+ boolean failedToAllocate = false;
|
|
|
+ try {
|
|
|
+ startContainerWithGpuRequests(2, 2);
|
|
|
+ } catch (ResourceHandlerException e) {
|
|
|
+ failedToAllocate = true;
|
|
|
+ }
|
|
|
+ assertTrue("Container allocation is expected to fail!", failedToAllocate);
|
|
|
+
|
|
|
+ // Start container 3, ask 1 container, succeeded
|
|
|
+ // devices = 0/1/3 will be blocked
|
|
|
+ startContainerWithGpuRequests(3, 1);
|
|
|
+ verifyDeniedDevices(getContainerId(3), Arrays.asList(new GpuDevice(0, 0),
|
|
|
+ new GpuDevice(1, 1), new GpuDevice(2, 3)));
|
|
|
+
|
|
|
+ // Start container 4, ask 0 container, succeeded
|
|
|
+ // --> All devices will be blocked
|
|
|
+ startContainerWithGpuRequests(4, 0);
|
|
|
+ verifyDeniedDevices(getContainerId(4), Arrays.asList(new GpuDevice(0, 0),
|
|
|
+ new GpuDevice(1, 1), new GpuDevice(2, 3), new GpuDevice(3, 4)));
|
|
|
+
|
|
|
+ gpuResourceHandler.postComplete(getContainerId(1));
|
|
|
+ verifyCgroupsDeletedForContainer(1);
|
|
|
+ verifyNumberOfAvailableGpus(3, gpuResourceHandler);
|
|
|
+
|
|
|
+ gpuResourceHandler.postComplete(getContainerId(3));
|
|
|
+ verifyCgroupsDeletedForContainer(3);
|
|
|
+ verifyNumberOfAvailableGpus(4, gpuResourceHandler);
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
@Test
|
|
|
- public void testAssignedGpuWillBeCleanedupWhenStoreOpFails()
|
|
|
+ public void testAssignedGpuWillBeCleanedUpWhenStoreOpFails()
|
|
|
throws Exception {
|
|
|
- Configuration conf = createDefaultConfig();
|
|
|
- conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
|
|
|
- gpuDiscoverer.initialize(conf);
|
|
|
-
|
|
|
- gpuResourceHandler.bootstrap(conf);
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ initializeGpus();
|
|
|
|
|
|
doThrow(new IOException("Exception ...")).when(mockNMStateStore)
|
|
|
.storeAssignedResources(
|
|
@@ -307,16 +363,16 @@ public class TestGpuResourceHandler {
|
|
|
boolean exception = false;
|
|
|
/* Start container 1, asks 3 containers */
|
|
|
try {
|
|
|
- gpuResourceHandler.preStart(mockContainerWithGpuRequest(1, 3));
|
|
|
+ gpuResourceHandler.preStart(mockContainerWithGpuRequest(1,
|
|
|
+ createResourceRequest(3)));
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
exception = true;
|
|
|
}
|
|
|
|
|
|
- Assert.assertTrue("preStart should throw exception", exception);
|
|
|
+ assertTrue("preStart should throw exception", exception);
|
|
|
|
|
|
- // After preStart, we still have 4 available GPU since the store op fails.
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ // After preStart, we still have 4 available GPU since the store op failed.
|
|
|
+ verifyNumberOfAvailableGpus(4, gpuResourceHandler);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -327,45 +383,40 @@ public class TestGpuResourceHandler {
|
|
|
|
|
|
try {
|
|
|
gpuResourceHandler.bootstrap(conf);
|
|
|
- Assert.fail("Should fail because no GPU available");
|
|
|
+ fail("Should fail because no GPU available");
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
// Expected because of no resource available
|
|
|
}
|
|
|
|
|
|
/* Start container 1, asks 0 containers */
|
|
|
- gpuResourceHandler.preStart(mockContainerWithGpuRequest(1, 0));
|
|
|
+ gpuResourceHandler.preStart(mockContainerWithGpuRequest(1,
|
|
|
+ createResourceRequest(0)));
|
|
|
verifyDeniedDevices(getContainerId(1), Collections.emptyList());
|
|
|
|
|
|
/* Start container 2, asks 1 containers. Excepted to fail */
|
|
|
boolean failedToAllocate = false;
|
|
|
try {
|
|
|
- gpuResourceHandler.preStart(mockContainerWithGpuRequest(2, 1));
|
|
|
+ gpuResourceHandler.preStart(mockContainerWithGpuRequest(2,
|
|
|
+ createResourceRequest(1)));
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
failedToAllocate = true;
|
|
|
}
|
|
|
- Assert.assertTrue(failedToAllocate);
|
|
|
+ assertTrue("Container allocation is expected to fail!", failedToAllocate);
|
|
|
|
|
|
/* Release container 1, expect cgroups deleted */
|
|
|
gpuResourceHandler.postComplete(getContainerId(1));
|
|
|
|
|
|
- verify(mockCGroupsHandler, times(1)).createCGroup(
|
|
|
- CGroupsHandler.CGroupController.DEVICES, getContainerId(1).toString());
|
|
|
- Assert.assertEquals(0,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ verifyCgroupsDeletedForContainer(1);
|
|
|
+ verifyNumberOfAvailableGpus(0, gpuResourceHandler);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testAllocationStored() throws Exception {
|
|
|
- Configuration conf = createDefaultConfig();
|
|
|
- conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
|
|
|
- gpuDiscoverer.initialize(conf);
|
|
|
-
|
|
|
- gpuResourceHandler.bootstrap(conf);
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ initializeGpus();
|
|
|
|
|
|
/* Start container 1, asks 3 containers */
|
|
|
- Container container = mockContainerWithGpuRequest(1, 3);
|
|
|
+ Container container = mockContainerWithGpuRequest(1,
|
|
|
+ createResourceRequest(3));
|
|
|
gpuResourceHandler.preStart(container);
|
|
|
|
|
|
verify(mockNMStateStore).storeAssignedResources(container,
|
|
@@ -374,16 +425,18 @@ public class TestGpuResourceHandler {
|
|
|
new GpuDevice(2, 3)));
|
|
|
|
|
|
// Only device=4 will be blocked.
|
|
|
- verifyDeniedDevices(getContainerId(1), Arrays.asList(new GpuDevice(3, 4)));
|
|
|
+ verifyDeniedDevices(getContainerId(1),
|
|
|
+ Collections.singletonList(new GpuDevice(3, 4)));
|
|
|
|
|
|
/* Start container 2, ask 0 container, succeeded */
|
|
|
- container = mockContainerWithGpuRequest(2, 0);
|
|
|
+ container = mockContainerWithGpuRequest(2, createResourceRequest(0));
|
|
|
gpuResourceHandler.preStart(container);
|
|
|
|
|
|
verifyDeniedDevices(getContainerId(2), Arrays
|
|
|
.asList(new GpuDevice(0, 0), new GpuDevice(1, 1), new GpuDevice(2, 3),
|
|
|
new GpuDevice(3, 4)));
|
|
|
- Assert.assertEquals(0, container.getResourceMappings()
|
|
|
+ assertEquals("Number of GPU device allocations is not the expected!", 0,
|
|
|
+ container.getResourceMappings()
|
|
|
.getAssignedResources(ResourceInformation.GPU_URI).size());
|
|
|
|
|
|
// Store assigned resource will not be invoked.
|
|
@@ -393,7 +446,8 @@ public class TestGpuResourceHandler {
|
|
|
|
|
|
@Test
|
|
|
public void testAllocationStoredWithNULLStateStore() throws Exception {
|
|
|
- NMNullStateStoreService mockNMNULLStateStore = mock(NMNullStateStoreService.class);
|
|
|
+ NMNullStateStoreService mockNMNULLStateStore =
|
|
|
+ mock(NMNullStateStoreService.class);
|
|
|
|
|
|
Configuration conf = createDefaultConfig();
|
|
|
conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
|
|
@@ -409,11 +463,11 @@ public class TestGpuResourceHandler {
|
|
|
gpuDiscoverer.initialize(conf);
|
|
|
|
|
|
gpuNULLStateResourceHandler.bootstrap(conf);
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuNULLStateResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ verifyNumberOfAvailableGpus(4, gpuNULLStateResourceHandler);
|
|
|
|
|
|
/* Start container 1, asks 3 containers */
|
|
|
- Container container = mockContainerWithGpuRequest(1, 3);
|
|
|
+ Container container = mockContainerWithGpuRequest(1,
|
|
|
+ createResourceRequest(3));
|
|
|
gpuNULLStateResourceHandler.preStart(container);
|
|
|
|
|
|
verify(nmnctx.getNMStateStore()).storeAssignedResources(container,
|
|
@@ -424,13 +478,7 @@ public class TestGpuResourceHandler {
|
|
|
|
|
|
@Test
|
|
|
public void testRecoverResourceAllocation() throws Exception {
|
|
|
- Configuration conf = createDefaultConfig();
|
|
|
- conf.set(YarnConfiguration.NM_GPU_ALLOWED_DEVICES, "0:0,1:1,2:3,3:4");
|
|
|
- gpuDiscoverer.initialize(conf);
|
|
|
-
|
|
|
- gpuResourceHandler.bootstrap(conf);
|
|
|
- Assert.assertEquals(4,
|
|
|
- gpuResourceHandler.getGpuAllocator().getAvailableGpus());
|
|
|
+ initializeGpus();
|
|
|
|
|
|
Container nmContainer = mock(Container.class);
|
|
|
ResourceMappings rmap = new ResourceMappings();
|
|
@@ -449,12 +497,14 @@ public class TestGpuResourceHandler {
|
|
|
|
|
|
Map<GpuDevice, ContainerId> deviceAllocationMapping =
|
|
|
gpuResourceHandler.getGpuAllocator().getDeviceAllocationMappingCopy();
|
|
|
- Assert.assertEquals(2, deviceAllocationMapping.size());
|
|
|
- Assert.assertTrue(
|
|
|
+ assertEquals("Unexpected number of allocated GPU devices!", 2,
|
|
|
+ deviceAllocationMapping.size());
|
|
|
+ assertTrue("Expected GPU device is not found in allocations!",
|
|
|
deviceAllocationMapping.keySet().contains(new GpuDevice(1, 1)));
|
|
|
- Assert.assertTrue(
|
|
|
+ assertTrue("Expected GPU device is not found in allocations!",
|
|
|
deviceAllocationMapping.keySet().contains(new GpuDevice(2, 3)));
|
|
|
- Assert.assertEquals(deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
+ assertEquals("GPU device is not assigned to the expected container!",
|
|
|
+ deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
getContainerId(1));
|
|
|
|
|
|
// TEST CASE
|
|
@@ -476,17 +526,20 @@ public class TestGpuResourceHandler {
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
caughtException = true;
|
|
|
}
|
|
|
- Assert.assertTrue(
|
|
|
+ assertTrue(
|
|
|
"Should fail since requested device Id is not in allowed list",
|
|
|
caughtException);
|
|
|
|
|
|
// Make sure internal state not changed.
|
|
|
deviceAllocationMapping =
|
|
|
gpuResourceHandler.getGpuAllocator().getDeviceAllocationMappingCopy();
|
|
|
- Assert.assertEquals(2, deviceAllocationMapping.size());
|
|
|
- Assert.assertTrue(deviceAllocationMapping.keySet()
|
|
|
+ assertEquals("Unexpected number of allocated GPU devices!",
|
|
|
+ 2, deviceAllocationMapping.size());
|
|
|
+ assertTrue("Expected GPU devices are not found in allocations!",
|
|
|
+ deviceAllocationMapping.keySet()
|
|
|
.containsAll(Arrays.asList(new GpuDevice(1, 1), new GpuDevice(2, 3))));
|
|
|
- Assert.assertEquals(deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
+ assertEquals("GPU device is not assigned to the expected container!",
|
|
|
+ deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
getContainerId(1));
|
|
|
|
|
|
// TEST CASE
|
|
@@ -508,17 +561,20 @@ public class TestGpuResourceHandler {
|
|
|
} catch (ResourceHandlerException e) {
|
|
|
caughtException = true;
|
|
|
}
|
|
|
- Assert.assertTrue(
|
|
|
+ assertTrue(
|
|
|
"Should fail since requested device Id is not in allowed list",
|
|
|
caughtException);
|
|
|
|
|
|
// Make sure internal state not changed.
|
|
|
deviceAllocationMapping =
|
|
|
gpuResourceHandler.getGpuAllocator().getDeviceAllocationMappingCopy();
|
|
|
- Assert.assertEquals(2, deviceAllocationMapping.size());
|
|
|
- Assert.assertTrue(deviceAllocationMapping.keySet()
|
|
|
+ assertEquals("Unexpected number of allocated GPU devices!",
|
|
|
+ 2, deviceAllocationMapping.size());
|
|
|
+ assertTrue("Expected GPU devices are not found in allocations!",
|
|
|
+ deviceAllocationMapping.keySet()
|
|
|
.containsAll(Arrays.asList(new GpuDevice(1, 1), new GpuDevice(2, 3))));
|
|
|
- Assert.assertEquals(deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
+ assertEquals("GPU device is not assigned to the expected container!",
|
|
|
+ deviceAllocationMapping.get(new GpuDevice(1, 1)),
|
|
|
getContainerId(1));
|
|
|
}
|
|
|
}
|