|
@@ -19,6 +19,8 @@
|
|
|
package org.apache.hadoop.yarn.server.nodemanager;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils.newNodeHeartbeatResponse;
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.mockito.ArgumentMatchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
@@ -57,6 +59,7 @@ import org.apache.hadoop.net.ServerSocketUtil;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.service.Service.STATE;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.service.ServiceOperations;
|
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
|
|
@@ -80,6 +83,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl;
|
|
|
import org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl;
|
|
|
+import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
|
|
|
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
@@ -96,11 +100,13 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
|
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
|
|
@@ -118,6 +124,10 @@ import org.apache.hadoop.ipc.Server;
|
|
|
|
|
|
@SuppressWarnings("rawtypes")
|
|
|
public class TestNodeStatusUpdater extends NodeManagerTestBase {
|
|
|
+
|
|
|
+ /** Bytes in a GigaByte. */
|
|
|
+ private static final long GB = 1024L * 1024L * 1024L;
|
|
|
+
|
|
|
volatile int heartBeatID = 0;
|
|
|
volatile Throwable nmStartError = null;
|
|
|
private final List<NodeId> registeredNodes = new ArrayList<NodeId>();
|
|
@@ -1774,6 +1784,99 @@ public class TestNodeStatusUpdater extends NodeManagerTestBase {
|
|
|
Assert.assertTrue("Test failed with exception(s)" + exceptions,
|
|
|
exceptions.isEmpty());
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test if the {@link NodeManager} updates the resources in the
|
|
|
+ * {@link ContainersMonitor} when the {@link ResourceManager} triggers the
|
|
|
+ * change.
|
|
|
+ * @throws Exception If the test cannot run.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testUpdateNMResources() throws Exception {
|
|
|
+
|
|
|
+ // The resource set for the Node Manager from the Resource Tracker
|
|
|
+ final Resource resource = Resource.newInstance(8 * 1024, 1);
|
|
|
+
|
|
|
+ LOG.info("Start the Resource Tracker to mock heartbeats");
|
|
|
+ Server resourceTracker = getMockResourceTracker(resource);
|
|
|
+ resourceTracker.start();
|
|
|
+
|
|
|
+ LOG.info("Start the Node Manager");
|
|
|
+ NodeManager nodeManager = new NodeManager();
|
|
|
+ YarnConfiguration nmConf = new YarnConfiguration();
|
|
|
+ nmConf.setSocketAddr(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
|
|
|
+ resourceTracker.getListenerAddress());
|
|
|
+ nmConf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:0");
|
|
|
+ nodeManager.init(nmConf);
|
|
|
+ nodeManager.start();
|
|
|
+
|
|
|
+ LOG.info("Initially the Node Manager should have the default resources");
|
|
|
+ ContainerManager containerManager = nodeManager.getContainerManager();
|
|
|
+ ContainersMonitor containerMonitor =
|
|
|
+ containerManager.getContainersMonitor();
|
|
|
+ assertEquals(8, containerMonitor.getVCoresAllocatedForContainers());
|
|
|
+ assertEquals(8 * GB, containerMonitor.getPmemAllocatedForContainers());
|
|
|
+
|
|
|
+ LOG.info("The first heartbeat should trigger a resource change to {}",
|
|
|
+ resource);
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> containerMonitor.getVCoresAllocatedForContainers() == 1,
|
|
|
+ 100, 2 * 1000);
|
|
|
+ assertEquals(8 * GB, containerMonitor.getPmemAllocatedForContainers());
|
|
|
+
|
|
|
+ resource.setVirtualCores(5);
|
|
|
+ resource.setMemorySize(4 * 1024);
|
|
|
+ LOG.info("Change the resources to {}", resource);
|
|
|
+ GenericTestUtils.waitFor(
|
|
|
+ () -> containerMonitor.getVCoresAllocatedForContainers() == 5,
|
|
|
+ 100, 2 * 1000);
|
|
|
+ assertEquals(4 * GB, containerMonitor.getPmemAllocatedForContainers());
|
|
|
+
|
|
|
+ LOG.info("Cleanup");
|
|
|
+ nodeManager.stop();
|
|
|
+ nodeManager.close();
|
|
|
+ resourceTracker.stop();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create a mock Resource Tracker server that returns the resources we want
|
|
|
+ * in the heartbeat.
|
|
|
+ * @param resource Resource to reply in the heartbeat.
|
|
|
+ * @return RPC server for the Resource Tracker.
|
|
|
+ * @throws Exception If it cannot create the Resource Tracker.
|
|
|
+ */
|
|
|
+ private static Server getMockResourceTracker(final Resource resource)
|
|
|
+ throws Exception {
|
|
|
+
|
|
|
+ // Setup the mock Resource Tracker
|
|
|
+ final ResourceTracker rt = mock(ResourceTracker.class);
|
|
|
+ when(rt.registerNodeManager(any())).thenAnswer(invocation -> {
|
|
|
+ RegisterNodeManagerResponse response = recordFactory.newRecordInstance(
|
|
|
+ RegisterNodeManagerResponse.class);
|
|
|
+ response.setContainerTokenMasterKey(createMasterKey());
|
|
|
+ response.setNMTokenMasterKey(createMasterKey());
|
|
|
+ return response;
|
|
|
+ });
|
|
|
+ when(rt.nodeHeartbeat(any())).thenAnswer(invocation -> {
|
|
|
+ NodeHeartbeatResponse response = recordFactory.newRecordInstance(
|
|
|
+ NodeHeartbeatResponse.class);
|
|
|
+ response.setResource(resource);
|
|
|
+ return response;
|
|
|
+ });
|
|
|
+ when(rt.unRegisterNodeManager(any())).thenAnswer(invocaiton -> {
|
|
|
+ UnRegisterNodeManagerResponse response = recordFactory.newRecordInstance(
|
|
|
+ UnRegisterNodeManagerResponse.class);
|
|
|
+ return response;
|
|
|
+ });
|
|
|
+
|
|
|
+ // Get the RPC server
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ YarnRPC rpc = YarnRPC.create(conf);
|
|
|
+ Server server = rpc.getServer(ResourceTracker.class, rt,
|
|
|
+ new InetSocketAddress("0.0.0.0", 0), conf, null, 1);
|
|
|
+ return server;
|
|
|
+ }
|
|
|
+
|
|
|
// Add new containers info into NM context each time node heart beats.
|
|
|
private class MyNMContext extends NMContext {
|
|
|
|