001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase;
020
021import static org.apache.hadoop.hbase.HBaseClusterManager.DEFAULT_RETRY_ATTEMPTS;
022import static org.apache.hadoop.hbase.HBaseClusterManager.DEFAULT_RETRY_SLEEP_INTERVAL;
023import static org.apache.hadoop.hbase.HBaseClusterManager.RETRY_ATTEMPTS_KEY;
024import static org.apache.hadoop.hbase.HBaseClusterManager.RETRY_SLEEP_INTERVAL_KEY;
025
026import java.io.IOException;
027import java.net.URI;
028import java.util.Collections;
029import java.util.HashMap;
030import java.util.Locale;
031import java.util.Map;
032import java.util.Objects;
033import java.util.Optional;
034import java.util.concurrent.Callable;
035import javax.ws.rs.client.Client;
036import javax.ws.rs.client.ClientBuilder;
037import javax.ws.rs.client.Entity;
038import javax.ws.rs.client.Invocation;
039import javax.ws.rs.client.WebTarget;
040import javax.ws.rs.core.MediaType;
041import javax.ws.rs.core.Response;
042import javax.ws.rs.core.UriBuilder;
043import javax.xml.ws.http.HTTPException;
044import org.apache.commons.lang3.StringUtils;
045import org.apache.hadoop.conf.Configuration;
046import org.apache.hadoop.conf.Configured;
047import org.apache.hadoop.hbase.util.RetryCounter;
048import org.apache.hadoop.hbase.util.RetryCounter.RetryConfig;
049import org.apache.hadoop.hbase.util.RetryCounterFactory;
050import org.apache.hadoop.util.ReflectionUtils;
051import org.slf4j.Logger;
052import org.slf4j.LoggerFactory;
053
054import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
055import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
056import org.apache.hbase.thirdparty.com.google.gson.JsonParser;
057import org.apache.hbase.thirdparty.org.glassfish.jersey.client.authentication.HttpAuthenticationFeature;
058
059/**
060 * A ClusterManager implementation designed to control Cloudera Manager (http://www.cloudera.com)
061 * clusters via REST API. This API uses HTTP GET requests against the cluster manager server to
062 * retrieve information and POST/PUT requests to perform actions. As a simple example, to retrieve a
063 * list of hosts from a CM server with login credentials admin:admin, a simple curl command would be
064 *     curl -X POST -H "Content-Type:application/json" -u admin:admin \
065 *         "http://this.is.my.server.com:7180/api/v8/hosts"
066 *
067 * This command would return a JSON result, which would need to be parsed to retrieve relevant
068 * information. This action and many others are covered by this class.
069 *
070 * A note on nomenclature: while the ClusterManager interface uses a ServiceType enum when
071 * referring to things like RegionServers and DataNodes, cluster managers often use different
072 * terminology. As an example, Cloudera Manager (http://www.cloudera.com) would refer to a
073 * RegionServer as a "role" of the HBase "service." It would further refer to "hbase" as the
074 * "serviceType." Apache Ambari (http://ambari.apache.org) would call the RegionServer a
075 * "component" of the HBase "service."
076 *
077 * This class will defer to the ClusterManager terminology in methods that it implements from
078 * that interface, but uses Cloudera Manager's terminology when dealing with its API directly.
079 *
080 * DEBUG-level logging gives more details of the actions this class takes as they happen. Log at
081 * TRACE-level to see the API requests and responses. TRACE-level logging on RetryCounter displays
082 * wait times, so that can be helpful too.
083 */
084public class RESTApiClusterManager extends Configured implements ClusterManager {
085  // Properties that need to be in the Configuration object to interact with the REST API cluster
086  // manager. Most easily defined in hbase-site.xml, but can also be passed on the command line.
087  private static final String REST_API_CLUSTER_MANAGER_HOSTNAME =
088      "hbase.it.clustermanager.restapi.hostname";
089  private static final String REST_API_CLUSTER_MANAGER_USERNAME =
090      "hbase.it.clustermanager.restapi.username";
091  private static final String REST_API_CLUSTER_MANAGER_PASSWORD =
092      "hbase.it.clustermanager.restapi.password";
093  private static final String REST_API_CLUSTER_MANAGER_CLUSTER_NAME =
094      "hbase.it.clustermanager.restapi.clustername";
095  private static final String REST_API_DELEGATE_CLUSTER_MANAGER =
096    "hbase.it.clustermanager.restapi.delegate";
097
098  private static final JsonParser parser = new JsonParser();
099
100  // Some default values for the above properties.
101  private static final String DEFAULT_SERVER_HOSTNAME = "http://localhost:7180";
102  private static final String DEFAULT_SERVER_USERNAME = "admin";
103  private static final String DEFAULT_SERVER_PASSWORD = "admin";
104  private static final String DEFAULT_CLUSTER_NAME = "Cluster 1";
105
106  // Fields for the hostname, username, password, and cluster name of the cluster management server
107  // to be used.
108  private String serverHostname;
109  private String clusterName;
110
111  // Each version of Cloudera Manager supports a particular API versions. Version 6 of this API
112  // provides all the features needed by this class.
113  private static final String API_VERSION = "v6";
114
115  // Client instances are expensive, so use the same one for all our REST queries.
116  private final Client client = ClientBuilder.newClient();
117
118  // An instance of HBaseClusterManager is used for methods like the kill, resume, and suspend
119  // because cluster managers don't tend to implement these operations.
120  private ClusterManager hBaseClusterManager;
121
122  private RetryCounterFactory retryCounterFactory;
123
124  private static final Logger LOG = LoggerFactory.getLogger(RESTApiClusterManager.class);
125
126  RESTApiClusterManager() { }
127
128  @Override
129  public void setConf(Configuration conf) {
130    super.setConf(conf);
131    if (conf == null) {
132      // `Configured()` constructor calls `setConf(null)` before calling again with a real value.
133      return;
134    }
135
136    final Class<? extends ClusterManager> clazz = conf.getClass(REST_API_DELEGATE_CLUSTER_MANAGER,
137      HBaseClusterManager.class, ClusterManager.class);
138    hBaseClusterManager = ReflectionUtils.newInstance(clazz, conf);
139
140    serverHostname = conf.get(REST_API_CLUSTER_MANAGER_HOSTNAME, DEFAULT_SERVER_HOSTNAME);
141    clusterName = conf.get(REST_API_CLUSTER_MANAGER_CLUSTER_NAME, DEFAULT_CLUSTER_NAME);
142
143    // Add filter to Client instance to enable server authentication.
144    String serverUsername = conf.get(REST_API_CLUSTER_MANAGER_USERNAME, DEFAULT_SERVER_USERNAME);
145    String serverPassword = conf.get(REST_API_CLUSTER_MANAGER_PASSWORD, DEFAULT_SERVER_PASSWORD);
146    client.register(HttpAuthenticationFeature.basic(serverUsername, serverPassword));
147
148    this.retryCounterFactory = new RetryCounterFactory(new RetryConfig()
149      .setMaxAttempts(conf.getInt(RETRY_ATTEMPTS_KEY, DEFAULT_RETRY_ATTEMPTS))
150      .setSleepInterval(conf.getLong(RETRY_SLEEP_INTERVAL_KEY, DEFAULT_RETRY_SLEEP_INTERVAL)));
151  }
152
153  @Override
154  public void start(ServiceType service, String hostname, int port) {
155    // With Cloudera Manager (6.3.x), sending a START command to a service role
156    // that is already in the "Started" state is an error. CM will log a message
157    // saying "Role must be stopped". It will complain similarly for other
158    // expected state transitions.
159    // A role process that has been `kill -9`'ed ends up with the service role
160    // retaining the "Started" state but with the process marked as "unhealthy".
161    // Instead of blindly issuing the START command, first send a STOP command
162    // to ensure the START will be accepted.
163    LOG.debug("Performing start of {} on {}:{}", service, hostname, port);
164    final RoleState currentState = getRoleState(service, hostname);
165    switch (currentState) {
166      case NA:
167      case BUSY:
168      case UNKNOWN:
169      case HISTORY_NOT_AVAILABLE:
170        LOG.warn("Unexpected service state detected. Service START requested, but currently in"
171          + " {} state. Attempting to start. {}, {}:{}", currentState, service, hostname, port);
172        performClusterManagerCommand(service, hostname, RoleCommand.START);
173        return;
174      case STOPPING:
175        LOG.warn("Unexpected service state detected. Service START requested, but currently in"
176          + " {} state. Waiting for stop before attempting start. {}, {}:{}", currentState,
177          service, hostname, port);
178        waitFor(() -> Objects.equals(RoleState.STOPPED, getRoleState(service, hostname)));
179        performClusterManagerCommand(service, hostname, RoleCommand.START);
180        return;
181      case STOPPED:
182        performClusterManagerCommand(service, hostname, RoleCommand.START);
183        return;
184      case STARTING:
185        LOG.warn("Unexpected service state detected. Service START requested, but already in"
186          + " {} state. Ignoring current request and waiting for start to complete. {}, {}:{}",
187          currentState, service, hostname, port);
188        waitFor(()-> Objects.equals(RoleState.STARTED, getRoleState(service, hostname)));
189        return;
190      case STARTED:
191        LOG.warn("Unexpected service state detected. Service START requested, but already in"
192          + " {} state. Restarting. {}, {}:{}", currentState, service, hostname, port);
193        performClusterManagerCommand(service, hostname, RoleCommand.RESTART);
194        return;
195    }
196    throw new RuntimeException("should not happen.");
197  }
198
199  @Override
200  public void stop(ServiceType service, String hostname, int port) {
201    LOG.debug("Performing stop of {} on {}:{}", service, hostname, port);
202    final RoleState currentState = getRoleState(service, hostname);
203    switch (currentState) {
204      case NA:
205      case BUSY:
206      case UNKNOWN:
207      case HISTORY_NOT_AVAILABLE:
208        LOG.warn("Unexpected service state detected. Service STOP requested, but already in"
209          + " {} state. Attempting to stop. {}, {}:{}", currentState, service, hostname, port);
210        performClusterManagerCommand(service, hostname, RoleCommand.STOP);
211        return;
212      case STOPPING:
213        waitFor(() -> Objects.equals(RoleState.STOPPED, getRoleState(service, hostname)));
214        return;
215      case STOPPED:
216        LOG.warn("Unexpected service state detected. Service STOP requested, but already in"
217          + " {} state. Ignoring current request. {}, {}:{}", currentState, service, hostname,
218          port);
219        return;
220      case STARTING:
221        LOG.warn("Unexpected service state detected. Service STOP requested, but already in"
222          + " {} state. Waiting for start to complete. {}, {}:{}", currentState, service, hostname,
223          port);
224        waitFor(()-> Objects.equals(RoleState.STARTED, getRoleState(service, hostname)));
225        performClusterManagerCommand(service, hostname, RoleCommand.STOP);
226        return;
227      case STARTED:
228        performClusterManagerCommand(service, hostname, RoleCommand.STOP);
229        return;
230    }
231    throw new RuntimeException("should not happen.");
232  }
233
234  @Override
235  public void restart(ServiceType service, String hostname, int port) {
236    LOG.debug("Performing stop followed by start of {} on {}:{}", service, hostname, port);
237    stop(service, hostname, port);
238    start(service, hostname, port);
239  }
240
241  @Override
242  public boolean isRunning(ServiceType service, String hostname, int port) {
243    LOG.debug("Issuing isRunning request against {} on {}:{}", service, hostname, port);
244    return executeWithRetries(() -> {
245      String serviceName = getServiceName(roleServiceType.get(service));
246      String hostId = getHostId(hostname);
247      RoleState roleState = getRoleState(serviceName, service.toString(), hostId);
248      HealthSummary healthSummary = getHealthSummary(serviceName, service.toString(), hostId);
249      return Objects.equals(RoleState.STARTED, roleState)
250        && Objects.equals(HealthSummary.GOOD, healthSummary);
251    });
252  }
253
254  @Override
255  public void kill(ServiceType service, String hostname, int port) throws IOException {
256    hBaseClusterManager.kill(service, hostname, port);
257  }
258
259  @Override
260  public void suspend(ServiceType service, String hostname, int port) throws IOException {
261    hBaseClusterManager.suspend(service, hostname, port);
262  }
263
264  @Override
265  public void resume(ServiceType service, String hostname, int port) throws IOException {
266    hBaseClusterManager.resume(service, hostname, port);
267  }
268
269  // Convenience method to execute command against role on hostname. Only graceful commands are
270  // supported since cluster management APIs don't tend to let you SIGKILL things.
271  private void performClusterManagerCommand(ServiceType role, String hostname,
272    RoleCommand command) {
273    // retry submitting the command until the submission succeeds.
274    final long commandId = executeWithRetries(() -> {
275      final String serviceName = getServiceName(roleServiceType.get(role));
276      final String hostId = getHostId(hostname);
277      final String roleName = getRoleName(serviceName, role.toString(), hostId);
278      return doRoleCommand(serviceName, roleName, command);
279    });
280    LOG.debug("Command {} of {} on {} submitted as commandId {}",
281      command, role, hostname, commandId);
282
283    // assume the submitted command was asynchronous. wait on the commandId to be marked as
284    // successful.
285    waitFor(() -> hasCommandCompleted(commandId));
286    if (!executeWithRetries(() -> hasCommandCompletedSuccessfully(commandId))) {
287      final String msg = String.format("Command %s of %s on %s submitted as commandId %s failed.",
288        command, role, hostname, commandId);
289      // TODO: this does not interrupt the monkey. should it?
290      throw new RuntimeException(msg);
291    }
292    LOG.debug("Command {} of {} on {} submitted as commandId {} completed successfully.",
293      command, role, hostname, commandId);
294  }
295
296  /**
297   * Issues a command (e.g. starting or stopping a role).
298   * @return the commandId of a successfully submitted asynchronous command.
299   */
300  private long doRoleCommand(String serviceName, String roleName, RoleCommand roleCommand) {
301    URI uri = UriBuilder.fromUri(serverHostname)
302        .path("api")
303        .path(API_VERSION)
304        .path("clusters")
305        .path(clusterName)
306        .path("services")
307        .path(serviceName)
308        .path("roleCommands")
309        .path(roleCommand.toString())
310        .build();
311    String body = "{ \"items\": [ \"" + roleName + "\" ] }";
312    LOG.trace("Executing POST against {} with body {} ...", uri, body);
313    WebTarget webTarget = client.target(uri);
314    Invocation.Builder invocationBuilder =  webTarget.request(MediaType.APPLICATION_JSON);
315    Response response = invocationBuilder.post(Entity.json(body));
316    final int statusCode = response.getStatus();
317    final String responseBody = response.readEntity(String.class);
318    if (statusCode != Response.Status.OK.getStatusCode()) {
319      LOG.warn(
320        "RoleCommand failed with status code {} and response body {}", statusCode, responseBody);
321      throw new HTTPException(statusCode);
322    }
323
324    LOG.trace("POST against {} completed with status code {} and response body {}",
325      uri, statusCode, responseBody);
326    return parser.parse(responseBody)
327      .getAsJsonObject()
328      .get("items")
329      .getAsJsonArray()
330      .get(0)
331      .getAsJsonObject()
332      .get("id")
333      .getAsLong();
334  }
335
336  private HealthSummary getHealthSummary(String serviceName, String roleType, String hostId) {
337    return HealthSummary.fromString(
338      getRolePropertyValue(serviceName, roleType, hostId, "healthSummary"));
339  }
340
341  // This API uses a hostId to execute host-specific commands; get one from a hostname.
342  private String getHostId(String hostname) {
343    String hostId = null;
344    URI uri = UriBuilder.fromUri(serverHostname)
345      .path("api")
346      .path(API_VERSION)
347      .path("hosts")
348      .build();
349    JsonElement hosts = parser.parse(getFromURIGet(uri))
350      .getAsJsonObject()
351      .get("items");
352    if (hosts != null) {
353      // Iterate through the list of hosts, stopping once you've reached the requested hostname.
354      for (JsonElement host : hosts.getAsJsonArray()) {
355        if (host.getAsJsonObject().get("hostname").getAsString().equals(hostname)) {
356          hostId = host.getAsJsonObject().get("hostId").getAsString();
357          break;
358        }
359      }
360    }
361
362    return hostId;
363  }
364
365  private String getFromURIGet(URI uri) {
366    LOG.trace("Executing GET against {} ...", uri);
367    final Response response = client.target(uri)
368      .request(MediaType.APPLICATION_JSON_TYPE)
369      .get();
370    int statusCode = response.getStatus();
371    final String responseBody = response.readEntity(String.class);
372    if (statusCode != Response.Status.OK.getStatusCode()) {
373      LOG.warn(
374        "request failed with status code {} and response body {}", statusCode, responseBody);
375      throw new HTTPException(statusCode);
376    }
377    // This API folds information as the value to an "items" attribute.
378    LOG.trace("GET against {} completed with status code {} and response body {}",
379      uri, statusCode, responseBody);
380    return responseBody;
381  }
382
383  // This API assigns a unique role name to each host's instance of a role.
384  private String getRoleName(String serviceName, String roleType, String hostId) {
385    return getRolePropertyValue(serviceName, roleType, hostId, "name");
386  }
387
388  // Get the value of a property from a role on a particular host.
389  private String getRolePropertyValue(String serviceName, String roleType, String hostId,
390      String property) {
391    String roleValue = null;
392    URI uri = UriBuilder.fromUri(serverHostname)
393      .path("api")
394      .path(API_VERSION)
395      .path("clusters")
396      .path(clusterName)
397      .path("services")
398      .path(serviceName)
399      .path("roles")
400      .build();
401    JsonElement roles = parser.parse(getFromURIGet(uri))
402      .getAsJsonObject()
403      .get("items");
404    if (roles != null) {
405      // Iterate through the list of roles, stopping once the requested one is found.
406      for (JsonElement role : roles.getAsJsonArray()) {
407        JsonObject roleObj = role.getAsJsonObject();
408        if (roleObj.get("hostRef").getAsJsonObject().get("hostId").getAsString().equals(hostId) &&
409          roleObj.get("type").getAsString().toLowerCase(Locale.ROOT)
410            .equals(roleType.toLowerCase(Locale.ROOT))) {
411          roleValue = roleObj.get(property).getAsString();
412          break;
413        }
414      }
415    }
416
417    return roleValue;
418  }
419
420  private RoleState getRoleState(ServiceType service, String hostname) {
421    return executeWithRetries(() -> {
422      String serviceName = getServiceName(roleServiceType.get(service));
423      String hostId = getHostId(hostname);
424      RoleState state = getRoleState(serviceName, service.toString(), hostId);
425      // sometimes the response (usually the first) is null. retry those.
426      return Objects.requireNonNull(state);
427    });
428  }
429
430  private RoleState getRoleState(String serviceName, String roleType, String hostId) {
431    return RoleState.fromString(
432      getRolePropertyValue(serviceName, roleType, hostId, "roleState"));
433  }
434
435  // Convert a service (e.g. "HBASE," "HDFS") into a service name (e.g. "HBASE-1," "HDFS-1").
436  private String getServiceName(Service service) {
437    String serviceName = null;
438    URI uri = UriBuilder.fromUri(serverHostname).path("api").path(API_VERSION).path("clusters")
439      .path(clusterName).path("services").build();
440    JsonElement services = parser.parse(getFromURIGet(uri))
441      .getAsJsonObject()
442      .get("items");
443    if (services != null) {
444      // Iterate through the list of services, stopping once the requested one is found.
445      for (JsonElement serviceEntry : services.getAsJsonArray()) {
446        if (serviceEntry.getAsJsonObject().get("type").getAsString().equals(service.toString())) {
447          serviceName = serviceEntry.getAsJsonObject().get("name").getAsString();
448          break;
449        }
450      }
451    }
452
453    return serviceName;
454  }
455
456  private Optional<JsonObject> getCommand(final long commandId) {
457    final URI uri = UriBuilder.fromUri(serverHostname)
458      .path("api")
459      .path(API_VERSION)
460      .path("commands")
461      .path(Long.toString(commandId))
462      .build();
463    return Optional.ofNullable(getFromURIGet(uri))
464      .map(parser::parse)
465      .map(JsonElement::getAsJsonObject);
466  }
467
468  /**
469   * Return {@code true} if the {@code commandId} has finished processing.
470   */
471  private boolean hasCommandCompleted(final long commandId) {
472    return getCommand(commandId)
473      .map(val -> {
474        final boolean isActive = val.get("active").getAsBoolean();
475        if (isActive) {
476          LOG.debug("command {} is still active.", commandId);
477        }
478        return !isActive;
479      })
480      .orElse(false);
481  }
482
483  /**
484   * Return {@code true} if the {@code commandId} has finished successfully.
485   */
486  private boolean hasCommandCompletedSuccessfully(final long commandId) {
487    return getCommand(commandId)
488      .filter(val -> {
489        final boolean isActive = val.get("active").getAsBoolean();
490        if (isActive) {
491          LOG.debug("command {} is still active.", commandId);
492        }
493        return !isActive;
494      })
495      .map(val -> {
496        final boolean isSuccess = val.get("success").getAsBoolean();
497        LOG.debug("command {} completed as {}.", commandId, isSuccess);
498        return isSuccess;
499      })
500      .orElse(false);
501  }
502
503  /**
504   * Helper method for executing retryable work.
505   */
506  private <T> T executeWithRetries(final Callable<T> callable) {
507    final RetryCounter retryCounter = retryCounterFactory.create();
508    while (true) {
509      try {
510        return callable.call();
511      } catch (Exception e) {
512        if (retryCounter.shouldRetry()) {
513          LOG.debug("execution failed with exception. Retrying.", e);
514        } else {
515          throw new RuntimeException("retries exhausted", e);
516        }
517      }
518      try {
519        retryCounter.sleepUntilNextRetry();
520      } catch (InterruptedException e) {
521        throw new RuntimeException(e);
522      }
523    }
524  }
525
526  private void waitFor(final Callable<Boolean> predicate) {
527    final RetryCounter retryCounter = retryCounterFactory.create();
528    while (true) {
529      try {
530        if (Objects.equals(true, predicate.call())) {
531          return;
532        }
533      } catch (Exception e) {
534        if (retryCounter.shouldRetry()) {
535          LOG.debug("execution failed with exception. Retrying.", e);
536        } else {
537          throw new RuntimeException("retries exhausted", e);
538        }
539      }
540      try {
541        retryCounter.sleepUntilNextRetry();
542      } catch (InterruptedException e) {
543        throw new RuntimeException(e);
544      }
545    }
546  }
547
548  /*
549   * Some enums to guard against bad calls.
550   */
551
552  // The RoleCommand enum is used by the doRoleCommand method to guard against non-existent methods
553  // being invoked on a given role.
554  // TODO: Integrate zookeeper and hdfs related failure injections (Ref: HBASE-14261).
555  private enum RoleCommand {
556    START, STOP, RESTART;
557
558    // APIs tend to take commands in lowercase, so convert them to save the trouble later.
559    @Override
560    public String toString() {
561      return name().toLowerCase(Locale.ROOT);
562    }
563  }
564
565  /**
566   * Represents the configured run state of a role.
567   * @see <a href="https://archive.cloudera.com/cm6/6.3.0/generic/jar/cm_api/apidocs/json_ApiRoleState.html">
568   *   https://archive.cloudera.com/cm6/6.3.0/generic/jar/cm_api/apidocs/json_ApiRoleState.html</a>
569   */
570  private enum RoleState {
571    HISTORY_NOT_AVAILABLE, UNKNOWN, STARTING, STARTED, BUSY, STOPPING, STOPPED, NA;
572
573    public static RoleState fromString(final String value) {
574      if (StringUtils.isBlank(value)) {
575        return null;
576      }
577      return RoleState.valueOf(value.toUpperCase());
578    }
579  }
580
581  /**
582   * Represents of the high-level health status of a subject in the cluster.
583   * @see <a href="https://archive.cloudera.com/cm6/6.3.0/generic/jar/cm_api/apidocs/json_ApiHealthSummary.html">
584   *   https://archive.cloudera.com/cm6/6.3.0/generic/jar/cm_api/apidocs/json_ApiHealthSummary.html</a>
585   */
586  private enum HealthSummary {
587    DISABLED, HISTORY_NOT_AVAILABLE, NOT_AVAILABLE, GOOD, CONCERNING, BAD;
588
589    public static HealthSummary fromString(final String value) {
590      if (StringUtils.isBlank(value)) {
591        return null;
592      }
593      return HealthSummary.valueOf(value.toUpperCase());
594    }
595  }
596
597  // ClusterManager methods take a "ServiceType" object (e.g. "HBASE_MASTER," "HADOOP_NAMENODE").
598  // These "service types," which cluster managers call "roles" or "components," need to be mapped
599  // to their corresponding service (e.g. "HBase," "HDFS") in order to be controlled.
600  private static final Map<ServiceType, Service> roleServiceType = buildRoleServiceTypeMap();
601
602  private static Map<ServiceType, Service> buildRoleServiceTypeMap() {
603    final Map<ServiceType, Service> ret = new HashMap<>();
604    ret.put(ServiceType.HADOOP_NAMENODE, Service.HDFS);
605    ret.put(ServiceType.HADOOP_DATANODE, Service.HDFS);
606    ret.put(ServiceType.HADOOP_JOBTRACKER, Service.MAPREDUCE);
607    ret.put(ServiceType.HADOOP_TASKTRACKER, Service.MAPREDUCE);
608    ret.put(ServiceType.HBASE_MASTER, Service.HBASE);
609    ret.put(ServiceType.HBASE_REGIONSERVER, Service.HBASE);
610    return Collections.unmodifiableMap(ret);
611  }
612
613  enum Service {
614    HBASE, HDFS, MAPREDUCE
615  }
616}