|
@@ -18,19 +18,10 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.service.component;
|
|
|
|
|
|
-import org.apache.hadoop.registry.client.api.RegistryOperations;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
-import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.client.api.NMClient;
|
|
|
-import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
|
|
|
-import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.service.ServiceContext;
|
|
|
-import org.apache.hadoop.yarn.service.ServiceScheduler;
|
|
|
import org.apache.hadoop.yarn.service.ServiceTestUtils;
|
|
|
import org.apache.hadoop.yarn.service.TestServiceManager;
|
|
|
import org.apache.hadoop.yarn.service.api.records.ComponentState;
|
|
@@ -38,23 +29,15 @@ import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
|
|
|
import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
|
|
|
-
|
|
|
-import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
|
|
|
-import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
|
|
|
+import org.apache.hadoop.yarn.service.MockRunningServiceContext;
|
|
|
import org.apache.log4j.Logger;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
-import org.mockito.stubbing.Answer;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
import java.util.Iterator;
|
|
|
-import java.util.Map;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.STOP;
|
|
|
-
|
|
|
-import static org.mockito.Matchers.anyObject;
|
|
|
-import static org.mockito.Mockito.doNothing;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
@@ -63,7 +46,6 @@ import static org.mockito.Mockito.when;
|
|
|
*/
|
|
|
public class TestComponent {
|
|
|
|
|
|
- private static final int WAIT_MS_PER_LOOP = 1000;
|
|
|
static final Logger LOG = Logger.getLogger(TestComponent.class);
|
|
|
|
|
|
@Rule
|
|
@@ -115,7 +97,7 @@ public class TestComponent {
|
|
|
@Test
|
|
|
public void testContainerCompletedWhenUpgrading() throws Exception {
|
|
|
String serviceName = "testContainerComplete";
|
|
|
- ServiceContext context = createTestContext(rule, serviceName);
|
|
|
+ MockRunningServiceContext context = createTestContext(rule, serviceName);
|
|
|
Component comp = context.scheduler.getAllComponents().entrySet().iterator()
|
|
|
.next().getValue();
|
|
|
|
|
@@ -148,7 +130,7 @@ public class TestComponent {
|
|
|
ComponentState.FLEXING, comp.getComponentSpec().getState());
|
|
|
|
|
|
// new container get allocated
|
|
|
- assignNewContainer(context.attemptId, 10, context, comp);
|
|
|
+ context.assignNewContainer(context.attemptId, 10, comp);
|
|
|
|
|
|
// second instance finished upgrading
|
|
|
ComponentInstance instance2 = instanceIter.next();
|
|
@@ -174,7 +156,7 @@ public class TestComponent {
|
|
|
serviceName);
|
|
|
TestServiceManager.createDef(serviceName, testService);
|
|
|
|
|
|
- ServiceContext context = createTestContext(rule, testService);
|
|
|
+ ServiceContext context = new MockRunningServiceContext(rule, testService);
|
|
|
|
|
|
for (Component comp : context.scheduler.getAllComponents().values()) {
|
|
|
|
|
@@ -225,114 +207,11 @@ public class TestComponent {
|
|
|
return spec;
|
|
|
}
|
|
|
|
|
|
- public static ServiceContext createTestContext(
|
|
|
+ public static MockRunningServiceContext createTestContext(
|
|
|
ServiceTestUtils.ServiceFSWatcher fsWatcher, String serviceName)
|
|
|
throws Exception {
|
|
|
- return createTestContext(fsWatcher,
|
|
|
+ return new MockRunningServiceContext(fsWatcher,
|
|
|
TestServiceManager.createBaseDef(serviceName));
|
|
|
}
|
|
|
-
|
|
|
- public static ServiceContext createTestContext(
|
|
|
- ServiceTestUtils.ServiceFSWatcher fsWatcher, Service serviceDef)
|
|
|
- throws Exception {
|
|
|
- ServiceContext context = new ServiceContext();
|
|
|
- context.service = serviceDef;
|
|
|
- context.fs = fsWatcher.getFs();
|
|
|
-
|
|
|
- ContainerLaunchService mockLaunchService = mock(
|
|
|
- ContainerLaunchService.class);
|
|
|
-
|
|
|
- context.scheduler = new ServiceScheduler(context) {
|
|
|
- @Override protected YarnRegistryViewForProviders
|
|
|
- createYarnRegistryOperations(
|
|
|
- ServiceContext context, RegistryOperations registryClient) {
|
|
|
- return mock(YarnRegistryViewForProviders.class);
|
|
|
- }
|
|
|
-
|
|
|
- @Override public NMClientAsync createNMClient() {
|
|
|
- NMClientAsync nmClientAsync = super.createNMClient();
|
|
|
- NMClient nmClient = mock(NMClient.class);
|
|
|
- try {
|
|
|
- when(nmClient.getContainerStatus(anyObject(), anyObject()))
|
|
|
- .thenAnswer(
|
|
|
- (Answer<ContainerStatus>) invocation -> ContainerStatus
|
|
|
- .newInstance((ContainerId) invocation.getArguments()[0],
|
|
|
- org.apache.hadoop.yarn.api.records.ContainerState
|
|
|
- .RUNNING,
|
|
|
- "", 0));
|
|
|
- } catch (YarnException | IOException e) {
|
|
|
- throw new RuntimeException(e);
|
|
|
- }
|
|
|
- nmClientAsync.setClient(nmClient);
|
|
|
- return nmClientAsync;
|
|
|
- }
|
|
|
-
|
|
|
- @Override public ContainerLaunchService getContainerLaunchService() {
|
|
|
- return mockLaunchService;
|
|
|
- }
|
|
|
- };
|
|
|
- context.scheduler.init(fsWatcher.getConf());
|
|
|
-
|
|
|
- ServiceTestUtils.createServiceManager(context);
|
|
|
-
|
|
|
- doNothing().when(mockLaunchService).
|
|
|
- reInitCompInstance(anyObject(), anyObject(), anyObject(), anyObject());
|
|
|
- stabilizeComponents(context);
|
|
|
-
|
|
|
- return context;
|
|
|
- }
|
|
|
-
|
|
|
- private static void stabilizeComponents(ServiceContext context) {
|
|
|
-
|
|
|
- ApplicationId appId = ApplicationId.fromString(context.service.getId());
|
|
|
- ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
|
|
|
- context.attemptId = attemptId;
|
|
|
- Map<String, Component>
|
|
|
- componentState = context.scheduler.getAllComponents();
|
|
|
-
|
|
|
- int counter = 0;
|
|
|
- for (org.apache.hadoop.yarn.service.api.records.Component componentSpec :
|
|
|
- context.service.getComponents()) {
|
|
|
- Component component = new org.apache.hadoop.yarn.service.component.
|
|
|
- Component(componentSpec, 1L, context);
|
|
|
- componentState.put(component.getName(), component);
|
|
|
- component.handle(new ComponentEvent(component.getName(),
|
|
|
- ComponentEventType.FLEX));
|
|
|
-
|
|
|
- for (int i = 0; i < componentSpec.getNumberOfContainers(); i++) {
|
|
|
- counter++;
|
|
|
- assignNewContainer(attemptId, counter, context, component);
|
|
|
- }
|
|
|
-
|
|
|
- component.handle(new ComponentEvent(component.getName(),
|
|
|
- ComponentEventType.CHECK_STABLE));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private static void assignNewContainer(
|
|
|
- ApplicationAttemptId attemptId, long containerNum,
|
|
|
- ServiceContext context, Component component) {
|
|
|
-
|
|
|
-
|
|
|
- Container container = org.apache.hadoop.yarn.api.records.Container
|
|
|
- .newInstance(ContainerId.newContainerId(attemptId, containerNum),
|
|
|
- NODE_ID, "localhost", null, null,
|
|
|
- null);
|
|
|
- component.handle(new ComponentEvent(component.getName(),
|
|
|
- ComponentEventType.CONTAINER_ALLOCATED)
|
|
|
- .setContainer(container).setContainerId(container.getId()));
|
|
|
- ComponentInstance instance = context.scheduler.getLiveInstances().get(
|
|
|
- container.getId());
|
|
|
- ComponentInstanceEvent startEvent = new ComponentInstanceEvent(
|
|
|
- container.getId(), ComponentInstanceEventType.START);
|
|
|
- instance.handle(startEvent);
|
|
|
-
|
|
|
- ComponentInstanceEvent readyEvent = new ComponentInstanceEvent(
|
|
|
- container.getId(), ComponentInstanceEventType.BECOME_READY);
|
|
|
- instance.handle(readyEvent);
|
|
|
- }
|
|
|
-
|
|
|
- private static final NodeId NODE_ID = NodeId.fromString("localhost:0");
|
|
|
-
|
|
|
}
|
|
|
|