|
@@ -19,6 +19,8 @@
|
|
|
package org.apache.hadoop.yarn.service;
|
|
|
|
|
|
import com.google.common.base.Throwables;
|
|
|
+import com.google.common.collect.HashMultimap;
|
|
|
+import com.google.common.collect.Multimap;
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.curator.test.TestingCluster;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -29,13 +31,17 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.http.HttpServer2;
|
|
|
import org.apache.hadoop.registry.client.impl.zk.CuratorService;
|
|
|
import org.apache.hadoop.service.ServiceOperations;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.records.LocalResource;
|
|
|
import org.apache.hadoop.yarn.client.api.YarnClient;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Component;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.Container;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Service;
|
|
|
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
|
|
|
import org.apache.hadoop.yarn.service.client.ServiceClient;
|
|
|
import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
|
|
|
import org.apache.hadoop.yarn.service.exceptions.SliderException;
|
|
@@ -60,6 +66,7 @@ import java.nio.file.Paths;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import static org.apache.hadoop.registry.client.api.RegistryConstants.KEY_REGISTRY_ZK_QUORUM;
|
|
|
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC;
|
|
@@ -418,4 +425,132 @@ public class ServiceTestUtils {
|
|
|
return serviceBasePath;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Wait until all the containers for all components become ready state.
|
|
|
+ *
|
|
|
+ * @param client
|
|
|
+ * @param exampleApp
|
|
|
+ * @return all ready containers of a service.
|
|
|
+ * @throws TimeoutException
|
|
|
+ * @throws InterruptedException
|
|
|
+ */
|
|
|
+ protected Multimap<String, String> waitForAllCompToBeReady(ServiceClient
|
|
|
+ client, Service exampleApp) throws TimeoutException,
|
|
|
+ InterruptedException {
|
|
|
+ int expectedTotalContainers = countTotalContainers(exampleApp);
|
|
|
+
|
|
|
+ Multimap<String, String> allContainers = HashMultimap.create();
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ Service retrievedApp = client.getStatus(exampleApp.getName());
|
|
|
+ int totalReadyContainers = 0;
|
|
|
+ allContainers.clear();
|
|
|
+ LOG.info("Num Components " + retrievedApp.getComponents().size());
|
|
|
+ for (Component component : retrievedApp.getComponents()) {
|
|
|
+ LOG.info("looking for " + component.getName());
|
|
|
+ LOG.info(component.toString());
|
|
|
+ if (component.getContainers() != null) {
|
|
|
+ if (component.getContainers().size() == exampleApp
|
|
|
+ .getComponent(component.getName()).getNumberOfContainers()) {
|
|
|
+ for (Container container : component.getContainers()) {
|
|
|
+ LOG.info(
|
|
|
+ "Container state " + container.getState() + ", component "
|
|
|
+ + component.getName());
|
|
|
+ if (container.getState() == ContainerState.READY) {
|
|
|
+ totalReadyContainers++;
|
|
|
+ allContainers.put(component.getName(), container.getId());
|
|
|
+ LOG.info("Found 1 ready container " + container.getId());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ LOG.info(component.getName() + " Expected number of containers "
|
|
|
+ + exampleApp.getComponent(component.getName())
|
|
|
+ .getNumberOfContainers() + ", current = " + component
|
|
|
+ .getContainers());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ LOG.info("Exit loop, totalReadyContainers= " + totalReadyContainers
|
|
|
+ + " expected = " + expectedTotalContainers);
|
|
|
+ return totalReadyContainers == expectedTotalContainers;
|
|
|
+ } catch (Exception e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, 2000, 200000);
|
|
|
+ return allContainers;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Wait until service state becomes stable. A service is stable when all
|
|
|
+ * requested containers of all components are running and in ready state.
|
|
|
+ *
|
|
|
+ * @param client
|
|
|
+ * @param exampleApp
|
|
|
+ * @throws TimeoutException
|
|
|
+ * @throws InterruptedException
|
|
|
+ */
|
|
|
+ protected void waitForServiceToBeStable(ServiceClient client,
|
|
|
+ Service exampleApp) throws TimeoutException, InterruptedException {
|
|
|
+ waitForServiceToBeStable(client, exampleApp, 200000);
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void waitForServiceToBeStable(ServiceClient client,
|
|
|
+ Service exampleApp, int waitForMillis)
|
|
|
+ throws TimeoutException, InterruptedException {
|
|
|
+ waitForServiceToBeInState(client, exampleApp, ServiceState.STABLE,
|
|
|
+ waitForMillis);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Wait until service is started. It does not have to reach a stable state.
|
|
|
+ *
|
|
|
+ * @param client
|
|
|
+ * @param exampleApp
|
|
|
+ * @throws TimeoutException
|
|
|
+ * @throws InterruptedException
|
|
|
+ */
|
|
|
+ protected void waitForServiceToBeStarted(ServiceClient client,
|
|
|
+ Service exampleApp) throws TimeoutException, InterruptedException {
|
|
|
+ waitForServiceToBeInState(client, exampleApp, ServiceState.STARTED);
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void waitForServiceToBeInState(ServiceClient client,
|
|
|
+ Service exampleApp, ServiceState desiredState) throws TimeoutException,
|
|
|
+ InterruptedException {
|
|
|
+ waitForServiceToBeInState(client, exampleApp, desiredState, 200000);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Wait until service is started. It does not have to reach a stable state.
|
|
|
+ *
|
|
|
+ * @param client
|
|
|
+ * @param exampleApp
|
|
|
+ * @throws TimeoutException
|
|
|
+ * @throws InterruptedException
|
|
|
+ */
|
|
|
+ protected void waitForServiceToBeInState(ServiceClient client,
|
|
|
+ Service exampleApp, ServiceState desiredState, int waitForMillis) throws
|
|
|
+ TimeoutException, InterruptedException {
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ Service retrievedApp = client.getStatus(exampleApp.getName());
|
|
|
+ System.out.println(retrievedApp);
|
|
|
+ return retrievedApp.getState() == desiredState;
|
|
|
+ } catch (Exception e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, 2000, waitForMillis);
|
|
|
+ }
|
|
|
+
|
|
|
+ private int countTotalContainers(Service service) {
|
|
|
+ int totalContainers = 0;
|
|
|
+ for (Component component : service.getComponents()) {
|
|
|
+ totalContainers += component.getNumberOfContainers();
|
|
|
+ }
|
|
|
+ return totalContainers;
|
|
|
+ }
|
|
|
}
|