|
@@ -20,8 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager;
|
|
|
|
|
|
import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
|
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
|
import static org.mockito.ArgumentMatchers.any;
|
|
|
import static org.mockito.ArgumentMatchers.anyBoolean;
|
|
|
import static org.mockito.ArgumentMatchers.isA;
|
|
@@ -39,10 +39,12 @@ import java.io.FileNotFoundException;
|
|
|
import java.io.FileWriter;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Random;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
@@ -60,23 +62,32 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.ConfigurationException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.ResourceLocalizationSpec;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings.AssignedResources;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.numa.NumaResourceAllocation;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.numa.NumaResourceAllocator;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.MockLocalizerHeartbeatResponse;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReacquisitionContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
+import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -86,6 +97,14 @@ public class TestDefaultContainerExecutor {
|
|
|
private static Path BASE_TMP_PATH = new Path("target",
|
|
|
TestDefaultContainerExecutor.class.getSimpleName());
|
|
|
|
|
|
+ private YarnConfiguration yarnConfiguration;
|
|
|
+
|
|
|
+ private DefaultContainerExecutor containerExecutor;
|
|
|
+
|
|
|
+ private Container mockContainer;
|
|
|
+
|
|
|
+ private NumaResourceAllocator numaResourceAllocator;
|
|
|
+
|
|
|
@AfterClass
|
|
|
public static void deleteTmpFiles() throws IOException {
|
|
|
FileContext lfs = FileContext.getLocalFSFileContext();
|
|
@@ -736,4 +755,204 @@ public class TestDefaultContainerExecutor {
|
|
|
// new FsPermission(ApplicationLocalizer.LOGDIR_PERM), true);
|
|
|
// }
|
|
|
|
|
|
+ @Before
|
|
|
+ public void setUp() throws IOException, YarnException {
|
|
|
+ yarnConfiguration = new YarnConfiguration();
|
|
|
+ setNumaConfig();
|
|
|
+ Context mockContext = createAndGetMockContext();
|
|
|
+ NMStateStoreService nmStateStoreService =
|
|
|
+ mock(NMStateStoreService.class);
|
|
|
+ when(mockContext.getNMStateStore()).thenReturn(nmStateStoreService);
|
|
|
+ numaResourceAllocator = new NumaResourceAllocator(mockContext) {
|
|
|
+ @Override
|
|
|
+ public String executeNGetCmdOutput(Configuration config)
|
|
|
+ throws YarnRuntimeException {
|
|
|
+ return getNumaCmdOutput();
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ numaResourceAllocator.init(yarnConfiguration);
|
|
|
+ FileContext lfs = FileContext.getLocalFSFileContext();
|
|
|
+ containerExecutor = new DefaultContainerExecutor(lfs) {
|
|
|
+ @Override
|
|
|
+ public Configuration getConf() {
|
|
|
+ return yarnConfiguration;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ containerExecutor.setNumaResourceAllocator(numaResourceAllocator);
|
|
|
+ mockContainer = mock(Container.class);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setNumaConfig() {
|
|
|
+ yarnConfiguration.set(YarnConfiguration.NM_NUMA_AWARENESS_ENABLED, "true");
|
|
|
+ yarnConfiguration.set(YarnConfiguration.NM_NUMA_AWARENESS_READ_TOPOLOGY, "true");
|
|
|
+ yarnConfiguration.set(YarnConfiguration.NM_NUMA_AWARENESS_NUMACTL_CMD, "/usr/bin/numactl");
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ private String getNumaCmdOutput() {
|
|
|
+ // architecture of 8 cpu cores
|
|
|
+ // randomly picked size of memory
|
|
|
+ return "available: 2 nodes (0-1)\n\t"
|
|
|
+ + "node 0 cpus: 0 2 4 6\n\t"
|
|
|
+ + "node 0 size: 73717 MB\n\t"
|
|
|
+ + "node 0 free: 73717 MB\n\t"
|
|
|
+ + "node 1 cpus: 1 3 5 7\n\t"
|
|
|
+ + "node 1 size: 73717 MB\n\t"
|
|
|
+ + "node 1 free: 73717 MB\n\t"
|
|
|
+ + "node distances:\n\t"
|
|
|
+ + "node 0 1\n\t"
|
|
|
+ + "0: 10 20\n\t"
|
|
|
+ + "1: 20 10";
|
|
|
+ }
|
|
|
+
|
|
|
+ private Context createAndGetMockContext() {
|
|
|
+ Context mockContext = mock(Context.class);
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ ConcurrentHashMap<ContainerId, Container> mockContainers = mock(
|
|
|
+ ConcurrentHashMap.class);
|
|
|
+ mockContainer = mock(Container.class);
|
|
|
+ when(mockContainer.getResourceMappings())
|
|
|
+ .thenReturn(new ResourceMappings());
|
|
|
+ when(mockContainers.get(any())).thenReturn(mockContainer);
|
|
|
+ when(mockContext.getContainers()).thenReturn(mockContainers);
|
|
|
+ when(mockContainer.getResource()).thenReturn(Resource.newInstance(2048, 2));
|
|
|
+ return mockContext;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testAllocateNumaResource(String containerId, Resource resource,
|
|
|
+ String memNodes, String cpuNodes) throws Exception {
|
|
|
+ when(mockContainer.getContainerId())
|
|
|
+ .thenReturn(ContainerId.fromString(containerId));
|
|
|
+ when(mockContainer.getResource()).thenReturn(resource);
|
|
|
+ NumaResourceAllocation numaResourceAllocation =
|
|
|
+ numaResourceAllocator.allocateNumaNodes(mockContainer);
|
|
|
+ containerExecutor.setNumactl(containerExecutor.getConf().get(YarnConfiguration.NM_NUMA_AWARENESS_NUMACTL_CMD,
|
|
|
+ YarnConfiguration.DEFAULT_NM_NUMA_AWARENESS_NUMACTL_CMD));
|
|
|
+ String[] commands = containerExecutor.getNumaCommands(numaResourceAllocation);
|
|
|
+ assertEquals(Arrays.asList(commands), Arrays.asList("/usr/bin/numactl",
|
|
|
+ "--interleave=" + memNodes, "--cpunodebind=" + cpuNodes));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testAllocateNumaMemoryResource() throws Exception {
|
|
|
+ // keeping cores constant for testing memory resources
|
|
|
+
|
|
|
+ // allocates node 0 for memory and cpu
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000001",
|
|
|
+ Resource.newInstance(2048, 2), "0", "0");
|
|
|
+
|
|
|
+ // allocates node 1 for memory and cpu since allocator uses round robin assignment
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000002",
|
|
|
+ Resource.newInstance(60000, 2), "1", "1");
|
|
|
+
|
|
|
+ // allocates node 0,1 for memory since there is no sufficient memory in any one node
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000003",
|
|
|
+ Resource.newInstance(80000, 2), "0,1", "0");
|
|
|
+
|
|
|
+ // returns null since there are no sufficient resources available for the request
|
|
|
+ when(mockContainer.getContainerId()).thenReturn(
|
|
|
+ ContainerId.fromString("container_1481156246874_0001_01_000004"));
|
|
|
+ when(mockContainer.getResource())
|
|
|
+ .thenReturn(Resource.newInstance(80000, 2));
|
|
|
+ Assert.assertNull(numaResourceAllocator.allocateNumaNodes(mockContainer));
|
|
|
+
|
|
|
+ // allocates node 1 for memory and cpu
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000005",
|
|
|
+ Resource.newInstance(1024, 2), "1", "1");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testAllocateNumaCpusResource() throws Exception {
|
|
|
+ // keeping memory constant
|
|
|
+
|
|
|
+ // allocates node 0 for memory and cpu
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000001",
|
|
|
+ Resource.newInstance(2048, 2), "0", "0");
|
|
|
+
|
|
|
+ // allocates node 1 for memory and cpu since allocator uses round robin assignment
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000002",
|
|
|
+ Resource.newInstance(2048, 2), "1", "1");
|
|
|
+
|
|
|
+ // allocates node 0,1 for cpus since there is are no sufficient cpus available in any one node
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000003",
|
|
|
+ Resource.newInstance(2048, 3), "0", "0,1");
|
|
|
+
|
|
|
+ // returns null since there are no sufficient resources available for the request
|
|
|
+ when(mockContainer.getContainerId()).thenReturn(
|
|
|
+ ContainerId.fromString("container_1481156246874_0001_01_000004"));
|
|
|
+ when(mockContainer.getResource()).thenReturn(Resource.newInstance(2048, 2));
|
|
|
+ Assert.assertNull(numaResourceAllocator.allocateNumaNodes(mockContainer));
|
|
|
+
|
|
|
+ // allocates node 1 for memory and cpu
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000005",
|
|
|
+ Resource.newInstance(2048, 1), "1", "1");
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testReacquireContainer() throws Exception {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ ConcurrentHashMap<ContainerId, Container> mockContainers = mock(
|
|
|
+ ConcurrentHashMap.class);
|
|
|
+ Context mockContext = mock(Context.class);
|
|
|
+ NMStateStoreService mock = mock(NMStateStoreService.class);
|
|
|
+ when(mockContext.getNMStateStore()).thenReturn(mock);
|
|
|
+ ResourceMappings resourceMappings = new ResourceMappings();
|
|
|
+ AssignedResources assignedRscs = new AssignedResources();
|
|
|
+ when(mockContainer.getResource())
|
|
|
+ .thenReturn(Resource.newInstance(147434, 2));
|
|
|
+ ContainerId cid = ContainerId.fromString("container_1481156246874_0001_01_000001");
|
|
|
+ when(mockContainer.getContainerId()).thenReturn(cid);
|
|
|
+ NumaResourceAllocation numaResourceAllocation =
|
|
|
+ numaResourceAllocator.allocateNumaNodes(mockContainer);
|
|
|
+ assignedRscs.updateAssignedResources(Arrays.asList(numaResourceAllocation));
|
|
|
+ resourceMappings.addAssignedResources("numa", assignedRscs);
|
|
|
+ when(mockContainer.getResourceMappings()).thenReturn(resourceMappings);
|
|
|
+ when(mockContainers.get(any())).thenReturn(mockContainer);
|
|
|
+ when(mockContext.getContainers()).thenReturn(mockContainers);
|
|
|
+
|
|
|
+ // recovered numa resources should be added to the used resources and
|
|
|
+ // remaining will be available for further allocation.
|
|
|
+
|
|
|
+ ContainerReacquisitionContext containerReacquisitionContext =
|
|
|
+ new ContainerReacquisitionContext.Builder()
|
|
|
+ .setContainerId(cid)
|
|
|
+ .setUser("user")
|
|
|
+ .setContainer(mockContainer)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ containerExecutor.reacquireContainer(containerReacquisitionContext);
|
|
|
+
|
|
|
+ // reacquireContainer recovers all the numa resources ,
|
|
|
+ // that should be free to use next
|
|
|
+ testAllocateNumaResource("container_1481156246874_0001_01_000001",
|
|
|
+ Resource.newInstance(147434, 2), "0,1", "1");
|
|
|
+ when(mockContainer.getContainerId()).thenReturn(
|
|
|
+ ContainerId.fromString("container_1481156246874_0001_01_000004"));
|
|
|
+ when(mockContainer.getResource())
|
|
|
+ .thenReturn(Resource.newInstance(1024, 2));
|
|
|
+
|
|
|
+ // returns null since there are no sufficient resources available for the request
|
|
|
+ Assert.assertNull(numaResourceAllocator.allocateNumaNodes(mockContainer));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testConcatStringCommands() {
|
|
|
+ // test one array of string as null
|
|
|
+ assertEquals(containerExecutor.concatStringCommands(null, new String[]{"hello"})[0],
|
|
|
+ new String[]{"hello"}[0]);
|
|
|
+ // test both array of string as null
|
|
|
+ Assert.assertNull(containerExecutor.concatStringCommands(null, null));
|
|
|
+ // test case when both arrays are not null and of equal length
|
|
|
+ String[] res = containerExecutor.concatStringCommands(new String[]{"one"},
|
|
|
+ new String[]{"two"});
|
|
|
+ assertEquals(res[0]+res[1], "one" + "two");
|
|
|
+ // test both array of different length
|
|
|
+ res = containerExecutor.concatStringCommands(new String[]{"one"},
|
|
|
+ new String[]{"two", "three"});
|
|
|
+ assertEquals(res[0] + res[1] + res[2], "one" + "two" + "three");
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
}
|