|
@@ -19,23 +19,26 @@
|
|
|
package org.apache.hadoop.yarn.service;
|
|
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.registry.client.api.RegistryOperations;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Artifact;
|
|
|
import org.apache.hadoop.yarn.service.api.records.ComponentState;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
import org.apache.hadoop.yarn.service.api.records.ServiceState;
|
|
|
+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.exceptions.SliderException;
|
|
|
-import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
|
|
|
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.util.Map;
|
|
|
-
|
|
|
-import static org.mockito.Mockito.mock;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.List;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
/**
|
|
|
* Tests for {@link ServiceManager}.
|
|
@@ -46,117 +49,120 @@ public class TestServiceManager {
|
|
|
public ServiceTestUtils.ServiceFSWatcher rule =
|
|
|
new ServiceTestUtils.ServiceFSWatcher();
|
|
|
|
|
|
- @Test
|
|
|
- public void testUpgrade() throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager("testUpgrade");
|
|
|
- upgrade(serviceManager, "v2", false, false);
|
|
|
+ @Test (timeout = TIMEOUT)
|
|
|
+ public void testUpgrade() throws Exception {
|
|
|
+ ServiceContext context = createServiceContext("testUpgrade");
|
|
|
+ initUpgrade(context, "v2", false, false, false);
|
|
|
Assert.assertEquals("service not upgraded", ServiceState.UPGRADING,
|
|
|
- serviceManager.getServiceSpec().getState());
|
|
|
+ context.getServiceManager().getServiceSpec().getState());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test (timeout = TIMEOUT)
|
|
|
public void testRestartNothingToUpgrade()
|
|
|
- throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager(
|
|
|
+ throws Exception {
|
|
|
+ ServiceContext context = createServiceContext(
|
|
|
"testRestartNothingToUpgrade");
|
|
|
- upgrade(serviceManager, "v2", false, false);
|
|
|
-
|
|
|
- //make components stable
|
|
|
- serviceManager.getServiceSpec().getComponents().forEach(comp -> {
|
|
|
- comp.setState(ComponentState.STABLE);
|
|
|
- });
|
|
|
- serviceManager.handle(new ServiceEvent(ServiceEventType.START));
|
|
|
+ initUpgrade(context, "v2", false, false, false);
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
+ //make components stable by upgrading all instances
|
|
|
+ upgradeAllInstances(context);
|
|
|
+
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(
|
|
|
+ new ServiceEvent(ServiceEventType.START));
|
|
|
+ GenericTestUtils.waitFor(()->
|
|
|
+ context.service.getState().equals(ServiceState.STABLE),
|
|
|
+ CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
Assert.assertEquals("service not re-started", ServiceState.STABLE,
|
|
|
- serviceManager.getServiceSpec().getState());
|
|
|
+ manager.getServiceSpec().getState());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testAutoFinalizeNothingToUpgrade() throws IOException,
|
|
|
- SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager(
|
|
|
+ @Test(timeout = TIMEOUT)
|
|
|
+ public void testAutoFinalizeNothingToUpgrade() throws Exception {
|
|
|
+ ServiceContext context = createServiceContext(
|
|
|
"testAutoFinalizeNothingToUpgrade");
|
|
|
- upgrade(serviceManager, "v2", false, true);
|
|
|
-
|
|
|
- //make components stable
|
|
|
- serviceManager.getServiceSpec().getComponents().forEach(comp ->
|
|
|
- comp.setState(ComponentState.STABLE));
|
|
|
- serviceManager.handle(new ServiceEvent(ServiceEventType.CHECK_STABLE));
|
|
|
+ initUpgrade(context, "v2", false, true, false);
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
+ //make components stable by upgrading all instances
|
|
|
+ upgradeAllInstances(context);
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(()->
|
|
|
+ context.service.getState().equals(ServiceState.STABLE),
|
|
|
+ CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
Assert.assertEquals("service stable", ServiceState.STABLE,
|
|
|
- serviceManager.getServiceSpec().getState());
|
|
|
+ manager.getServiceSpec().getState());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
+ @Test(timeout = TIMEOUT)
|
|
|
public void testRestartWithPendingUpgrade()
|
|
|
- throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager("testRestart");
|
|
|
- upgrade(serviceManager, "v2", true, false);
|
|
|
- serviceManager.handle(new ServiceEvent(ServiceEventType.START));
|
|
|
+ throws Exception {
|
|
|
+ ServiceContext context = createServiceContext("testRestart");
|
|
|
+ initUpgrade(context, "v2", true, false, false);
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
+
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(
|
|
|
+ new ServiceEvent(ServiceEventType.START));
|
|
|
+ context.scheduler.getDispatcher().stop();
|
|
|
Assert.assertEquals("service should still be upgrading",
|
|
|
- ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
|
|
|
+ ServiceState.UPGRADING, manager.getServiceSpec().getState());
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testCheckState() throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager(
|
|
|
- "testCheckState");
|
|
|
- upgrade(serviceManager, "v2", true, false);
|
|
|
+ @Test(timeout = TIMEOUT)
|
|
|
+ public void testFinalize() throws Exception {
|
|
|
+ ServiceContext context = createServiceContext("testCheckState");
|
|
|
+ initUpgrade(context, "v2", true, false, false);
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
Assert.assertEquals("service not upgrading", ServiceState.UPGRADING,
|
|
|
- serviceManager.getServiceSpec().getState());
|
|
|
+ manager.getServiceSpec().getState());
|
|
|
|
|
|
- // make components stable
|
|
|
- serviceManager.getServiceSpec().getComponents().forEach(comp -> {
|
|
|
- comp.setState(ComponentState.STABLE);
|
|
|
- });
|
|
|
- ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
|
|
|
- serviceManager.handle(checkStable);
|
|
|
- Assert.assertEquals("service should still be upgrading",
|
|
|
- ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
|
|
|
+ //make components stable by upgrading all instances
|
|
|
+ upgradeAllInstances(context);
|
|
|
|
|
|
// finalize service
|
|
|
- ServiceEvent restart = new ServiceEvent(ServiceEventType.START);
|
|
|
- serviceManager.handle(restart);
|
|
|
- Assert.assertEquals("service not stable",
|
|
|
- ServiceState.STABLE, serviceManager.getServiceSpec().getState());
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(
|
|
|
+ new ServiceEvent(ServiceEventType.START));
|
|
|
+ GenericTestUtils.waitFor(()->
|
|
|
+ context.service.getState().equals(ServiceState.STABLE),
|
|
|
+ CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
+ Assert.assertEquals("service not re-started", ServiceState.STABLE,
|
|
|
+ manager.getServiceSpec().getState());
|
|
|
|
|
|
- validateUpgradeFinalization(serviceManager.getName(), "v2");
|
|
|
+ validateUpgradeFinalization(manager.getName(), "v2");
|
|
|
}
|
|
|
|
|
|
- @Test
|
|
|
- public void testCheckStateAutoFinalize() throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager(
|
|
|
- "testCheckState");
|
|
|
- serviceManager.getServiceSpec().setState(
|
|
|
+ @Test(timeout = TIMEOUT)
|
|
|
+ public void testAutoFinalize() throws Exception {
|
|
|
+ ServiceContext context = createServiceContext("testCheckStateAutoFinalize");
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
+ manager.getServiceSpec().setState(
|
|
|
ServiceState.UPGRADING_AUTO_FINALIZE);
|
|
|
- upgrade(serviceManager, "v2", true, true);
|
|
|
- Assert.assertEquals("service not upgrading",
|
|
|
- ServiceState.UPGRADING_AUTO_FINALIZE,
|
|
|
- serviceManager.getServiceSpec().getState());
|
|
|
+ initUpgrade(context, "v2", true, true, false);
|
|
|
|
|
|
// make components stable
|
|
|
- serviceManager.getServiceSpec().getComponents().forEach(comp ->
|
|
|
- comp.setState(ComponentState.STABLE));
|
|
|
- ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
|
|
|
- serviceManager.handle(checkStable);
|
|
|
+ upgradeAllInstances(context);
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() ->
|
|
|
+ context.service.getState().equals(ServiceState.STABLE),
|
|
|
+ CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
Assert.assertEquals("service not stable",
|
|
|
- ServiceState.STABLE, serviceManager.getServiceSpec().getState());
|
|
|
+ ServiceState.STABLE, manager.getServiceSpec().getState());
|
|
|
|
|
|
- validateUpgradeFinalization(serviceManager.getName(), "v2");
|
|
|
+ validateUpgradeFinalization(manager.getName(), "v2");
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testInvalidUpgrade() throws IOException, SliderException {
|
|
|
- ServiceManager serviceManager = createTestServiceManager(
|
|
|
- "testInvalidUpgrade");
|
|
|
- serviceManager.getServiceSpec().setState(
|
|
|
+ public void testInvalidUpgrade() throws Exception {
|
|
|
+ ServiceContext serviceContext = createServiceContext("testInvalidUpgrade");
|
|
|
+ ServiceManager manager = serviceContext.getServiceManager();
|
|
|
+ manager.getServiceSpec().setState(
|
|
|
ServiceState.UPGRADING_AUTO_FINALIZE);
|
|
|
Service upgradedDef = ServiceTestUtils.createExampleApplication();
|
|
|
- upgradedDef.setName(serviceManager.getName());
|
|
|
+ upgradedDef.setName(manager.getName());
|
|
|
upgradedDef.setVersion("v2");
|
|
|
upgradedDef.setLifetime(2L);
|
|
|
writeUpgradedDef(upgradedDef);
|
|
|
|
|
|
try {
|
|
|
- serviceManager.processUpgradeRequest("v2", true);
|
|
|
+ manager.processUpgradeRequest("v2", true, false);
|
|
|
} catch (Exception ex) {
|
|
|
Assert.assertTrue(ex instanceof UnsupportedOperationException);
|
|
|
return;
|
|
@@ -164,6 +170,32 @@ public class TestServiceManager {
|
|
|
Assert.fail();
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = TIMEOUT)
|
|
|
+ public void testExpressUpgrade() throws Exception {
|
|
|
+ ServiceContext context = createServiceContext("testExpressUpgrade");
|
|
|
+ ServiceManager manager = context.getServiceManager();
|
|
|
+ manager.getServiceSpec().setState(
|
|
|
+ ServiceState.EXPRESS_UPGRADING);
|
|
|
+ initUpgrade(context, "v2", true, true, true);
|
|
|
+
|
|
|
+ List<String> comps = ServiceApiUtil.resolveCompsDependency(context.service);
|
|
|
+ // wait till instances of first component are in upgrade
|
|
|
+ String comp1 = comps.get(0);
|
|
|
+ upgradeInstancesOf(context, comp1);
|
|
|
+
|
|
|
+ // wait till instances of second component are in upgrade
|
|
|
+ String comp2 = comps.get(1);
|
|
|
+ upgradeInstancesOf(context, comp2);
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() ->
|
|
|
+ context.service.getState().equals(ServiceState.STABLE),
|
|
|
+ CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
+
|
|
|
+ Assert.assertEquals("service not stable",
|
|
|
+ ServiceState.STABLE, manager.getServiceSpec().getState());
|
|
|
+ validateUpgradeFinalization(manager.getName(), "v2");
|
|
|
+ }
|
|
|
+
|
|
|
private void validateUpgradeFinalization(String serviceName,
|
|
|
String expectedVersion) throws IOException {
|
|
|
Service savedSpec = ServiceApiUtil.loadService(rule.getFs(), serviceName);
|
|
@@ -172,15 +204,16 @@ public class TestServiceManager {
|
|
|
Assert.assertNotNull("app id not present", savedSpec.getId());
|
|
|
Assert.assertEquals("state not stable", ServiceState.STABLE,
|
|
|
savedSpec.getState());
|
|
|
- savedSpec.getComponents().forEach(compSpec -> {
|
|
|
- Assert.assertEquals("comp not stable", ComponentState.STABLE,
|
|
|
- compSpec.getState());
|
|
|
- });
|
|
|
+ savedSpec.getComponents().forEach(compSpec ->
|
|
|
+ Assert.assertEquals("comp not stable", ComponentState.STABLE,
|
|
|
+ compSpec.getState()));
|
|
|
}
|
|
|
|
|
|
- private void upgrade(ServiceManager serviceManager, String version,
|
|
|
- boolean upgradeArtifact, boolean autoFinalize)
|
|
|
- throws IOException, SliderException {
|
|
|
+ private void initUpgrade(ServiceContext context, String version,
|
|
|
+ boolean upgradeArtifact, boolean autoFinalize, boolean expressUpgrade)
|
|
|
+ throws IOException, SliderException, TimeoutException,
|
|
|
+ InterruptedException {
|
|
|
+ ServiceManager serviceManager = context.getServiceManager();
|
|
|
Service upgradedDef = ServiceTestUtils.createExampleApplication();
|
|
|
upgradedDef.setName(serviceManager.getName());
|
|
|
upgradedDef.setVersion(version);
|
|
@@ -191,39 +224,81 @@ public class TestServiceManager {
|
|
|
});
|
|
|
}
|
|
|
writeUpgradedDef(upgradedDef);
|
|
|
- serviceManager.processUpgradeRequest(version, autoFinalize);
|
|
|
+ serviceManager.processUpgradeRequest(version, autoFinalize, expressUpgrade);
|
|
|
ServiceEvent upgradeEvent = new ServiceEvent(ServiceEventType.UPGRADE);
|
|
|
- upgradeEvent.setVersion(version);
|
|
|
- if (autoFinalize) {
|
|
|
- upgradeEvent.setAutoFinalize(true);
|
|
|
- }
|
|
|
- serviceManager.handle(upgradeEvent);
|
|
|
+ upgradeEvent.setVersion(version).setExpressUpgrade(expressUpgrade)
|
|
|
+ .setAutoFinalize(autoFinalize);
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(()-> {
|
|
|
+ ServiceState serviceState = context.service.getState();
|
|
|
+ if (serviceState.equals(ServiceState.UPGRADING) ||
|
|
|
+ serviceState.equals(ServiceState.UPGRADING_AUTO_FINALIZE) ||
|
|
|
+ serviceState.equals(ServiceState.EXPRESS_UPGRADING)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }, CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void upgradeAllInstances(ServiceContext context) throws
|
|
|
+ TimeoutException, InterruptedException {
|
|
|
+ // upgrade the instances
|
|
|
+ context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
|
|
|
+ ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
|
|
|
+ ComponentInstanceEventType.UPGRADE);
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(event);
|
|
|
+ }));
|
|
|
+
|
|
|
+ // become ready
|
|
|
+ context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
|
|
|
+ ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
|
|
|
+ ComponentInstanceEventType.BECOME_READY);
|
|
|
+
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(event);
|
|
|
+ }));
|
|
|
+ GenericTestUtils.waitFor(()-> {
|
|
|
+ for (ComponentInstance instance:
|
|
|
+ context.scheduler.getLiveInstances().values()) {
|
|
|
+ if (!instance.getContainerState().equals(ContainerState.READY)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }, CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
}
|
|
|
|
|
|
- private ServiceManager createTestServiceManager(String name)
|
|
|
- throws IOException {
|
|
|
- ServiceContext context = new ServiceContext();
|
|
|
- context.service = createBaseDef(name);
|
|
|
- context.fs = rule.getFs();
|
|
|
-
|
|
|
- context.scheduler = new ServiceScheduler(context) {
|
|
|
- @Override
|
|
|
- protected YarnRegistryViewForProviders createYarnRegistryOperations(
|
|
|
- ServiceContext context, RegistryOperations registryClient) {
|
|
|
- return mock(YarnRegistryViewForProviders.class);
|
|
|
+ private void upgradeInstancesOf(ServiceContext context, String compName)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ Collection<ComponentInstance> compInstances = context.scheduler
|
|
|
+ .getAllComponents().get(compName).getAllComponentInstances();
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ for (ComponentInstance instance : compInstances) {
|
|
|
+ if (!instance.getContainerState().equals(ContainerState.UPGRADING)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
}
|
|
|
- };
|
|
|
+ return true;
|
|
|
+ }, CHECK_EVERY_MILLIS, TIMEOUT);
|
|
|
|
|
|
- context.scheduler.init(rule.getConf());
|
|
|
+ // instances of comp1 get upgraded and become ready event is triggered
|
|
|
+ // become ready
|
|
|
+ compInstances.forEach(instance -> {
|
|
|
+ ComponentInstanceEvent event = new ComponentInstanceEvent(
|
|
|
+ instance.getContainer().getId(),
|
|
|
+ ComponentInstanceEventType.BECOME_READY);
|
|
|
|
|
|
- Map<String, org.apache.hadoop.yarn.service.component.Component>
|
|
|
- componentState = context.scheduler.getAllComponents();
|
|
|
- context.service.getComponents().forEach(component -> {
|
|
|
- componentState.put(component.getName(),
|
|
|
- new org.apache.hadoop.yarn.service.component.Component(component,
|
|
|
- 1L, context));
|
|
|
+ context.scheduler.getDispatcher().getEventHandler().handle(event);
|
|
|
});
|
|
|
- return new ServiceManager(context);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ServiceContext createServiceContext(String name)
|
|
|
+ throws Exception {
|
|
|
+ Service service = createBaseDef(name);
|
|
|
+ ServiceContext context = new MockRunningServiceContext(rule,
|
|
|
+ service);
|
|
|
+ context.scheduler.getDispatcher().setDrainEventsOnStop();
|
|
|
+ context.scheduler.getDispatcher().start();
|
|
|
+ return context;
|
|
|
}
|
|
|
|
|
|
public static Service createBaseDef(String name) {
|
|
@@ -257,4 +332,6 @@ public class TestServiceManager {
|
|
|
upgradedDef);
|
|
|
}
|
|
|
|
|
|
+ private static final int TIMEOUT = 200000;
|
|
|
+ private static final int CHECK_EVERY_MILLIS = 100;
|
|
|
}
|