|
@@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
-import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ResourceUtilization;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -43,19 +43,21 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
|
|
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.DeletionAsUserContext;
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
|
|
import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
|
|
|
|
+import org.apache.log4j.Logger;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
|
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
import static org.junit.Assert.assertNotNull;
|
|
-import static org.junit.Assert.assertNull;
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
|
|
|
|
public class TestContainersMonitorResourceChange {
|
|
public class TestContainersMonitorResourceChange {
|
|
|
|
|
|
|
|
+ static final Logger LOG = Logger
|
|
|
|
+ .getLogger(TestContainersMonitorResourceChange.class);
|
|
private ContainersMonitorImpl containersMonitor;
|
|
private ContainersMonitorImpl containersMonitor;
|
|
private MockExecutor executor;
|
|
private MockExecutor executor;
|
|
private Configuration conf;
|
|
private Configuration conf;
|
|
@@ -63,6 +65,8 @@ public class TestContainersMonitorResourceChange {
|
|
private Context context;
|
|
private Context context;
|
|
private MockContainerEventHandler containerEventHandler;
|
|
private MockContainerEventHandler containerEventHandler;
|
|
|
|
|
|
|
|
+ static final int WAIT_MS_PER_LOOP = 20; // 20 milli seconds
|
|
|
|
+
|
|
private static class MockExecutor extends ContainerExecutor {
|
|
private static class MockExecutor extends ContainerExecutor {
|
|
@Override
|
|
@Override
|
|
public void init() throws IOException {
|
|
public void init() throws IOException {
|
|
@@ -232,6 +236,60 @@ public class TestContainersMonitorResourceChange {
|
|
containersMonitor.stop();
|
|
containersMonitor.stop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testContainersCPUResourceForDefaultValue() throws Exception {
|
|
|
|
+ Configuration newConf = new Configuration(conf);
|
|
|
|
+ // set container monitor interval to be 20s
|
|
|
|
+ newConf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L);
|
|
|
|
+ containersMonitor = createContainersMonitor(executor, dispatcher, context);
|
|
|
|
+ newConf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
|
|
|
|
+ MockCPUResourceCalculatorProcessTree.class.getCanonicalName());
|
|
|
|
+ // set container monitor interval to be 20ms
|
|
|
|
+ containersMonitor.init(newConf);
|
|
|
|
+ containersMonitor.start();
|
|
|
|
+
|
|
|
|
+ // create container 1
|
|
|
|
+ containersMonitor.handle(new ContainerStartMonitoringEvent(
|
|
|
|
+ getContainerId(1), 2100L, 1000L, 1, 0, 0));
|
|
|
|
+
|
|
|
|
+ // Verify the container utilization value.
|
|
|
|
+ // Since MockCPUResourceCalculatorProcessTree will return a -1 as CPU
|
|
|
|
+ // utilization, containersUtilization will not be calculated and hence it
|
|
|
|
+ // will be 0.
|
|
|
|
+ assertEquals(
|
|
|
|
+ "Resource utilization must be default with MonitorThread's first run",
|
|
|
|
+ 0, containersMonitor.getContainersUtilization()
|
|
|
|
+ .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
|
|
|
|
+
|
|
|
|
+ // Verify the container utilization value. Since atleast one round is done,
|
|
|
|
+ // we can expect a non-zero value for container utilization as
|
|
|
|
+ // MockCPUResourceCalculatorProcessTree#getCpuUsagePercent will return 50.
|
|
|
|
+ waitForContainerResourceUtilizationChange(containersMonitor, 100);
|
|
|
|
+
|
|
|
|
+ containersMonitor.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static void waitForContainerResourceUtilizationChange(
|
|
|
|
+ ContainersMonitorImpl containersMonitor, int timeoutMsecs)
|
|
|
|
+ throws InterruptedException {
|
|
|
|
+ int timeWaiting = 0;
|
|
|
|
+ while (0 == containersMonitor.getContainersUtilization()
|
|
|
|
+ .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f))) {
|
|
|
|
+ if (timeWaiting >= timeoutMsecs) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ LOG.info(
|
|
|
|
+ "Monitor thread is waiting for resource utlization change.");
|
|
|
|
+ Thread.sleep(WAIT_MS_PER_LOOP);
|
|
|
|
+ timeWaiting += WAIT_MS_PER_LOOP;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ assertTrue("Resource utilization is not changed from second run onwards",
|
|
|
|
+ 0 != containersMonitor.getContainersUtilization()
|
|
|
|
+ .compareTo(ResourceUtilization.newInstance(0, 0, 0.0f)));
|
|
|
|
+ }
|
|
|
|
+
|
|
private ContainersMonitorImpl createContainersMonitor(
|
|
private ContainersMonitorImpl createContainersMonitor(
|
|
ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
|
|
ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
|
|
Context context) {
|
|
Context context) {
|