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