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