|
@@ -18,12 +18,13 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.service;
|
|
|
|
|
|
+import com.google.common.collect.HashMultimap;
|
|
|
+import com.google.common.collect.Multimap;
|
|
|
+import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
|
|
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
-import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
+import org.apache.hadoop.yarn.api.records.*;
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Component;
|
|
@@ -41,17 +42,12 @@ import org.junit.rules.TemporaryFolder;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.TreeSet;
|
|
|
+import java.util.*;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import static org.apache.hadoop.yarn.api.records.YarnApplicationState.FINISHED;
|
|
|
-import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
|
|
|
|
|
|
/**
|
|
|
* End to end tests to test deploying services with MiniYarnCluster and a in-JVM
|
|
@@ -67,7 +63,8 @@ public class TestYarnNativeServices extends ServiceTestUtils {
|
|
|
|
|
|
@Before
|
|
|
public void setup() throws Exception {
|
|
|
- setupInternal(NUM_NMS);
|
|
|
+ File tmpYarnDir = new File("target", "tmp");
|
|
|
+ FileUtils.deleteQuietly(tmpYarnDir);
|
|
|
}
|
|
|
|
|
|
@After
|
|
@@ -84,6 +81,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
|
|
|
// 6. Destroy the service
|
|
|
@Test (timeout = 200000)
|
|
|
public void testCreateFlexStopDestroyService() throws Exception {
|
|
|
+ setupInternal(NUM_NMS);
|
|
|
ServiceClient client = createClient();
|
|
|
Service exampleApp = createExampleApplication();
|
|
|
client.actionCreate(exampleApp);
|
|
@@ -135,6 +133,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
|
|
|
// Check containers for compa started before containers for compb
|
|
|
@Test (timeout = 200000)
|
|
|
public void testComponentStartOrder() throws Exception {
|
|
|
+ setupInternal(NUM_NMS);
|
|
|
ServiceClient client = createClient();
|
|
|
Service exampleApp = new Service();
|
|
|
exampleApp.setName("teststartorder");
|
|
@@ -155,6 +154,70 @@ public class TestYarnNativeServices extends ServiceTestUtils {
|
|
|
client.actionDestroy(exampleApp.getName());
|
|
|
}
|
|
|
|
|
|
+ // Test to verify recovery of SeviceMaster after RM is restarted.
|
|
|
+ // 1. Create an example service.
|
|
|
+ // 2. Restart RM.
|
|
|
+ // 3. Fail the application attempt.
|
|
|
+ // 4. Verify ServiceMaster recovers.
|
|
|
+ @Test(timeout = 200000)
|
|
|
+ public void testRecoverComponentsAfterRMRestart() throws Exception {
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
|
|
+ conf.setBoolean(
|
|
|
+ YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
|
|
|
+ conf.setLong(YarnConfiguration.NM_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
|
|
|
+ 500L);
|
|
|
+
|
|
|
+ conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
|
|
|
+ conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_USE_RPC, true);
|
|
|
+ setConf(conf);
|
|
|
+ setupInternal(NUM_NMS);
|
|
|
+
|
|
|
+ ServiceClient client = createClient();
|
|
|
+ Service exampleApp = createExampleApplication();
|
|
|
+ client.actionCreate(exampleApp);
|
|
|
+ waitForAllCompToBeReady(client, exampleApp);
|
|
|
+
|
|
|
+ LOG.info("Restart the resource manager");
|
|
|
+ getYarnCluster().restartResourceManager(
|
|
|
+ getYarnCluster().getActiveRMIndex());
|
|
|
+ GenericTestUtils.waitFor(() ->
|
|
|
+ getYarnCluster().getResourceManager().getServiceState() ==
|
|
|
+ org.apache.hadoop.service.Service.STATE.STARTED, 2000, 200000);
|
|
|
+ Assert.assertTrue("node managers connected",
|
|
|
+ getYarnCluster().waitForNodeManagersToConnect(5000));
|
|
|
+
|
|
|
+ ApplicationId exampleAppId = ApplicationId.fromString(exampleApp.getId());
|
|
|
+ ApplicationAttemptId applicationAttemptId = client.getYarnClient()
|
|
|
+ .getApplicationReport(exampleAppId).getCurrentApplicationAttemptId();
|
|
|
+
|
|
|
+ Multimap<String, String> containersBeforeFailure = getContainersForAllComp(
|
|
|
+ client, exampleApp);
|
|
|
+
|
|
|
+ LOG.info("Fail the application attempt {}", applicationAttemptId);
|
|
|
+ client.getYarnClient().failApplicationAttempt(applicationAttemptId);
|
|
|
+ //wait until attempt 2 is running
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ ApplicationReport ar = client.getYarnClient()
|
|
|
+ .getApplicationReport(exampleAppId);
|
|
|
+ return ar.getCurrentApplicationAttemptId().getAttemptId() == 2 &&
|
|
|
+ ar.getYarnApplicationState() == YarnApplicationState.RUNNING;
|
|
|
+ } catch (YarnException | IOException e) {
|
|
|
+ throw new RuntimeException("while waiting", e);
|
|
|
+ }
|
|
|
+ }, 2000, 200000);
|
|
|
+
|
|
|
+ Multimap<String, String> containersAfterFailure = getContainersForAllComp(
|
|
|
+ client, exampleApp);
|
|
|
+ Assert.assertEquals("component container affected by restart",
|
|
|
+ containersBeforeFailure, containersAfterFailure);
|
|
|
+
|
|
|
+ LOG.info("Stop/destroy service {}", exampleApp);
|
|
|
+ client.actionStop(exampleApp.getName(), true);
|
|
|
+ client.actionDestroy(exampleApp.getName());
|
|
|
+ }
|
|
|
+
|
|
|
// Check containers launched are in dependency order
|
|
|
// Get all containers into a list and sort based on container launch time e.g.
|
|
|
// compa-c1, compa-c2, compb-c1, compb-c2;
|
|
@@ -297,6 +360,24 @@ public class TestYarnNativeServices extends ServiceTestUtils {
|
|
|
}, 2000, 200000);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get all containers of a service.
|
|
|
+ */
|
|
|
+ private Multimap<String, String> getContainersForAllComp(ServiceClient client,
|
|
|
+ Service example) throws IOException, YarnException {
|
|
|
+
|
|
|
+ Multimap<String, String> allContainers = HashMultimap.create();
|
|
|
+ Service retrievedApp = client.getStatus(example.getName());
|
|
|
+ retrievedApp.getComponents().forEach(component -> {
|
|
|
+ if (component.getContainers() != null) {
|
|
|
+ component.getContainers().forEach(container -> {
|
|
|
+ allContainers.put(component.getName(), container.getId());
|
|
|
+ });
|
|
|
+ }
|
|
|
+ });
|
|
|
+ return allContainers;
|
|
|
+ }
|
|
|
+
|
|
|
private ServiceClient createClient() throws Exception {
|
|
|
ServiceClient client = new ServiceClient() {
|
|
|
@Override protected Path addJarResource(String appName,
|