This is an automated email from the ASF dual-hosted git repository.

edcoleman pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new e3c540d246 Add service status as command option to admin command 
(#4567)
e3c540d246 is described below

commit e3c540d246f00330f7e5a18aa9f07b2ff961527f
Author: EdColeman <d...@etcoleman.com>
AuthorDate: Fri May 24 13:18:01 2024 -0400

    Add service status as command option to admin command (#4567)
    
    Add service status as command option to admin command
    
      * Organizes service status by resource group.
---
 .../org/apache/accumulo/server/util/Admin.java     |  42 ++-
 .../accumulo/server/util/ServiceStatusCmd.java     | 368 +++++++++++++++++++
 .../util/serviceStatus/ServiceStatusReport.java    | 183 ++++++++++
 .../server/util/serviceStatus/StatusSummary.java   |  92 +++++
 .../accumulo/server/util/ServiceStatusCmdTest.java | 406 +++++++++++++++++++++
 .../serviceStatus/ServiceStatusReportTest.java     | 157 ++++++++
 6 files changed, 1235 insertions(+), 13 deletions(-)

diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java 
b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index ff9c4d9592..b8351ee43f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -283,6 +283,9 @@ public class Admin implements KeywordExecutable {
     JCommander cl = new JCommander(opts);
     cl.setProgramName("accumulo admin");
 
+    ServiceStatusCmd.Opts serviceStatusCommandOpts = new 
ServiceStatusCmd.Opts();
+    cl.addCommand("serviceStatus", serviceStatusCommandOpts);
+
     ChangeSecretCommand changeSecretCommand = new ChangeSecretCommand();
     cl.addCommand("changeSecret", changeSecretCommand);
 
@@ -398,6 +401,8 @@ public class Admin implements KeywordExecutable {
             tServerLocksOpts.delete);
       } else if (cl.getParsedCommand().equals("fate")) {
         executeFateOpsCommand(context, fateOpsCommand);
+      } else if (cl.getParsedCommand().equals("serviceStatus")) {
+        printServiceStatus(context, serviceStatusCommandOpts);
       } else {
         everything = cl.getParsedCommand().equals("stopAll");
 
@@ -425,6 +430,11 @@ public class Admin implements KeywordExecutable {
     }
   }
 
+  private static void printServiceStatus(ServerContext context, 
ServiceStatusCmd.Opts opts) {
+    ServiceStatusCmd ssc = new ServiceStatusCmd();
+    ssc.execute(context, opts);
+  }
+
   private static int ping(ClientContext context, List<String> args) {
 
     InstanceOperations io = context.instanceOperations();
@@ -590,21 +600,11 @@ public class Admin implements KeywordExecutable {
   private Map<String,String> siteConfig, systemConfig;
   private List<String> localUsers;
 
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "code runs in same security context as user who provided 
input")
   public void printConfig(ClientContext context, DumpConfigCommand opts) 
throws Exception {
 
-    File outputDirectory = null;
-    if (opts.directory != null) {
-      outputDirectory = new File(opts.directory);
-      if (!outputDirectory.isDirectory()) {
-        throw new IllegalArgumentException(
-            opts.directory + " does not exist on the local filesystem.");
-      }
-      if (!outputDirectory.canWrite()) {
-        throw new IllegalArgumentException(opts.directory + " is not 
writable");
-      }
-    }
+    @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
+        justification = "app is run in same security context as user providing 
the filename")
+    File outputDirectory = getOutputDirectory(opts.directory);
     defaultConfig = DefaultConfiguration.getInstance();
     siteConfig = context.instanceOperations().getSiteConfiguration();
     systemConfig = context.instanceOperations().getSystemConfiguration();
@@ -651,6 +651,22 @@ public class Admin implements KeywordExecutable {
     }
   }
 
+  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
+      justification = "app is run in same security context as user providing 
the filename")
+  private static File getOutputDirectory(final String directory) {
+    File outputDirectory = null;
+    if (directory != null) {
+      outputDirectory = new File(directory);
+      if (!outputDirectory.isDirectory()) {
+        throw new IllegalArgumentException(directory + " does not exist on the 
local filesystem.");
+      }
+      if (!outputDirectory.canWrite()) {
+        throw new IllegalArgumentException(directory + " is not writable");
+      }
+    }
+    return outputDirectory;
+  }
+
   private String getDefaultConfigValue(String key) {
     if (key == null) {
       return null;
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java
new file mode 100644
index 0000000000..204e536c93
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/ServiceStatusCmd.java
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.ServerServices;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.util.serviceStatus.ServiceStatusReport;
+import org.apache.accumulo.server.util.serviceStatus.StatusSummary;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.annotations.VisibleForTesting;
+
+public class ServiceStatusCmd {
+
+  // used when grouping by resource group when there is no group.
+  public static final String NO_GROUP_TAG = "NO_GROUP";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ServiceStatusCmd.class);
+
+  public ServiceStatusCmd() {}
+
+  /**
+   * Read the service statuses from ZooKeeper, build the status report and 
then output the report to
+   * stdout.
+   */
+  public void execute(final ServerContext context, final Opts opts) {
+
+    ZooReader zooReader = context.getZooReader();
+
+    final String zooRoot = context.getZooKeeperRoot();
+    LOG.trace("zooRoot: {}", zooRoot);
+
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = new 
TreeMap<>();
+
+    services.put(ServiceStatusReport.ReportKey.MANAGER, 
getManagerStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.MONITOR, 
getMonitorStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.T_SERVER, 
getTServerStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.S_SERVER, 
getScanServerStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.COORDINATOR,
+        getCoordinatorStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.COMPACTOR, 
getCompactorStatus(zooReader, zooRoot));
+    services.put(ServiceStatusReport.ReportKey.GC, getGcStatus(zooReader, 
zooRoot));
+
+    ServiceStatusReport report = new ServiceStatusReport(services, 
opts.noHosts);
+
+    if (opts.json) {
+      System.out.println(report.toJson());
+    } else {
+      StringBuilder sb = new StringBuilder(8192);
+      report.report(sb);
+      System.out.println(sb);
+    }
+  }
+
+  /**
+   * The manager paths in ZooKeeper are: {@code 
/accumulo/[IID]/managers/lock/zlock#[NUM]} with the
+   * lock data providing host:port.
+   */
+  @VisibleForTesting
+  StatusSummary getManagerStatus(final ZooReader zooReader, String zRootPath) {
+    String lockPath = zRootPath + Constants.ZMANAGER_LOCK;
+    return getStatusSummary(ServiceStatusReport.ReportKey.MANAGER, zooReader, 
lockPath);
+  }
+
+  /**
+   * The monitor paths in ZooKeeper are: {@code 
/accumulo/[IID]/monitor/lock/zlock#[NUM]} with the
+   * lock data providing host:port.
+   */
+  @VisibleForTesting
+  StatusSummary getMonitorStatus(final ZooReader zooReader, String zRootPath) {
+    String lockPath = zRootPath + Constants.ZMONITOR_LOCK;
+    return getStatusSummary(ServiceStatusReport.ReportKey.MONITOR, zooReader, 
lockPath);
+  }
+
+  /**
+   * The tserver paths in ZooKeeper are: {@code 
/accumulo/[IID]/tservers/[host:port]/zlock#[NUM]}
+   * with the lock data providing TSERV_CLIENT=host:port.
+   */
+  @VisibleForTesting
+  StatusSummary getTServerStatus(final ZooReader zooReader, String zRootPath) {
+    String lockPath = zRootPath + Constants.ZTSERVERS;
+    return getServerHostStatus(zooReader, lockPath, 
ServiceStatusReport.ReportKey.T_SERVER);
+  }
+
+  /**
+   * The sserver paths in ZooKeeper are: {@code 
/accumulo/[IID]/sservers/[host:port]/zlock#[NUM]}
+   * with the lock data providing [UUID],[GROUP]
+   */
+  @VisibleForTesting
+  StatusSummary getScanServerStatus(final ZooReader zooReader, String 
zRootPath) {
+    String lockPath = zRootPath + Constants.ZSSERVERS;
+    return getServerHostStatus(zooReader, lockPath, 
ServiceStatusReport.ReportKey.S_SERVER);
+  }
+
+  /**
+   * handles paths for tservers and servers with the lock stored beneath the 
host: port like:
+   * {@code /accumulo/IID/[tservers | sservers]/HOST:PORT/[LOCK]}
+   */
+  private StatusSummary getServerHostStatus(final ZooReader zooReader, String 
basePath,
+      ServiceStatusReport.ReportKey displayNames) {
+    AtomicInteger errorSum = new AtomicInteger(0);
+
+    // Set<String> hostNames = new TreeSet<>();
+    Set<String> groupNames = new TreeSet<>();
+    Map<String,Set<String>> hostsByGroups = new TreeMap<>();
+
+    var nodeNames = readNodeNames(zooReader, basePath);
+
+    nodeNames.getHosts().forEach(host -> {
+      var lock = readNodeNames(zooReader, basePath + "/" + host);
+      lock.getHosts().forEach(l -> {
+        var nodeData = readNodeData(zooReader, basePath + "/" + host + "/" + 
l);
+        int err = nodeData.getErrorCount();
+        if (err > 0) {
+          errorSum.addAndGet(nodeData.getErrorCount());
+        } else {
+          // process resource groups
+          String[] tokens = nodeData.getHosts().split(",");
+          if (tokens.length == 2) {
+            String groupName = tokens[1];
+            groupNames.add(groupName);
+            hostsByGroups.computeIfAbsent(groupName, s -> new 
TreeSet<>()).add(host);
+          } else {
+            hostsByGroups.computeIfAbsent(NO_GROUP_TAG, s -> new 
TreeSet<>()).add(host);
+          }
+        }
+
+      });
+      errorSum.addAndGet(lock.getFirst());
+    });
+    return new StatusSummary(displayNames, groupNames, hostsByGroups, 
errorSum.get());
+  }
+
+  /**
+   * The gc paths in ZooKeeper are: {@code 
/accumulo/[IID]/gc/lock/zlock#[NUM]} with the lock data
+   * providing GC_CLIENT=host:port
+   */
+  @VisibleForTesting
+  StatusSummary getGcStatus(final ZooReader zooReader, String zRootPath) {
+    String lockPath = zRootPath + Constants.ZGC_LOCK;
+    var temp = getStatusSummary(ServiceStatusReport.ReportKey.GC, zooReader, 
lockPath);
+    // remove GC_CLIENT= from displayed host:port
+    Set<String> hosts =
+        new 
TreeSet<>(stripServiceName(temp.getServiceByGroups().get(NO_GROUP_TAG)));
+
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put(NO_GROUP_TAG, hosts);
+
+    return new StatusSummary(temp.getServiceType(), temp.getResourceGroups(), 
hostByGroup,
+        temp.getErrorCount());
+
+  }
+
+  /**
+   * ServerServices writes lock data as [SERVICE]=host. This strips the 
[SERVICE]= from the string.
+   *
+   * @return a sort set of host names.
+   */
+  private Set<String> stripServiceName(Set<String> hostnames) {
+    return hostnames.stream().map(h -> {
+      if (h.contains(ServerServices.SEPARATOR_CHAR)) {
+        return h.substring(h.indexOf(ServerServices.SEPARATOR_CHAR) + 1);
+      }
+      return h;
+    }).collect(Collectors.toCollection(TreeSet::new));
+  }
+
+  /**
+   * The coordinator paths in ZooKeeper are: {@code 
/accumulo/[IID]/coordinators/lock/zlock#[NUM]}
+   * with the lock data providing host:port
+   */
+  @VisibleForTesting
+  StatusSummary getCoordinatorStatus(final ZooReader zooReader, String 
zRootPath) {
+    String lockPath = zRootPath + Constants.ZCOORDINATOR_LOCK;
+    return getStatusSummary(ServiceStatusReport.ReportKey.COORDINATOR, 
zooReader, lockPath);
+  }
+
+  /**
+   * The compactor paths in ZooKeeper are:
+   * {@code /accumulo/[IID]/compactors/[QUEUE_NAME]/host:port/zlock#[NUM]} 
with the host:port pulled
+   * from the path
+   */
+  @VisibleForTesting
+  StatusSummary getCompactorStatus(final ZooReader zooReader, String 
zRootPath) {
+    String lockPath = zRootPath + Constants.ZCOMPACTORS;
+    return getCompactorHosts(zooReader, lockPath);
+  }
+
+  /**
+   * Used to return status information when path is {@code 
/accumulo/IID/SERVICE_NAME/lock} like
+   * manager, monitor and others
+   *
+   * @return service status
+   */
+  private StatusSummary getStatusSummary(ServiceStatusReport.ReportKey 
displayNames,
+      ZooReader zooReader, String lockPath) {
+    var result = readAllNodesData(zooReader, lockPath);
+    Map<String,Set<String>> byGroup = new TreeMap<>();
+    byGroup.put(NO_GROUP_TAG, result.getHosts());
+    return new StatusSummary(displayNames, Set.of(), byGroup, 
result.getErrorCount());
+  }
+
+  /**
+   * Pull host:port from path {@code 
/accumulo/IID/compactors/[QUEUE][host:port]}
+   */
+  private StatusSummary getCompactorHosts(final ZooReader zooReader, final 
String zRootPath) {
+    final AtomicInteger errors = new AtomicInteger(0);
+
+    Map<String,Set<String>> hostsByGroups = new TreeMap<>();
+
+    // get group names
+    Result<Integer,Set<String>> queueNodes = readNodeNames(zooReader, 
zRootPath);
+    errors.addAndGet(queueNodes.getErrorCount());
+    Set<String> queues = new TreeSet<>(queueNodes.getHosts());
+
+    queues.forEach(group -> {
+      var hostNames = readNodeNames(zooReader, zRootPath + "/" + group);
+      errors.addAndGet(hostNames.getErrorCount());
+      Collection<String> hosts = hostNames.getHosts();
+      hosts.forEach(host -> {
+        hostsByGroups.computeIfAbsent(group, set -> new TreeSet<>()).add(host);
+      });
+    });
+
+    return new StatusSummary(ServiceStatusReport.ReportKey.COMPACTOR, queues, 
hostsByGroups,
+        errors.get());
+  }
+
+  /**
+   * Read the node names from ZooKeeper. Exceptions are counted but ignored.
+   *
+   * @return Result with error count, Set of the node names.
+   */
+  @VisibleForTesting
+  Result<Integer,Set<String>> readNodeNames(final ZooReader zooReader, final 
String path) {
+    Set<String> nodeNames = new TreeSet<>();
+    final AtomicInteger errorCount = new AtomicInteger(0);
+    try {
+      var children = zooReader.getChildren(path);
+      if (children != null) {
+        nodeNames.addAll(children);
+      }
+    } catch (KeeperException | InterruptedException ex) {
+      if (Thread.currentThread().isInterrupted()) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(ex);
+      }
+      errorCount.incrementAndGet();
+    }
+    return new Result<>(errorCount.get(), nodeNames);
+  }
+
+  /**
+   * Read the data from a ZooKeeper node, tracking if an error occurred. 
ZooKeeper's exceptions are
+   * counted but otherwise ignored.
+   *
+   * @return Pair with error count, the node data as String.
+   */
+  @VisibleForTesting
+  Result<Integer,String> readNodeData(final ZooReader zooReader, final String 
path) {
+    try {
+      byte[] data = zooReader.getData(path);
+      return new Result<>(0, new String(data, UTF_8));
+    } catch (KeeperException | InterruptedException ex) {
+      if (Thread.currentThread().isInterrupted()) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(ex);
+      }
+      LOG.info("Could not read locks from ZooKeeper for path {}", path, ex);
+      return new Result<>(1, "");
+    }
+  }
+
+  /**
+   * Read the data from all ZooKeeper nodes under a ptah, tracking if errors 
occurred. ZooKeeper's
+   * exceptions are counted but otherwise ignored.
+   *
+   * @return Pair with error count, the data from each node as a String.
+   */
+  @VisibleForTesting
+  Result<Integer,Set<String>> readAllNodesData(final ZooReader zooReader, 
final String path) {
+    Set<String> hosts = new TreeSet<>();
+    final AtomicInteger errorCount = new AtomicInteger(0);
+    try {
+      var locks = zooReader.getChildren(path);
+      locks.forEach(lock -> {
+        var nodeData = readNodeData(zooReader, path + "/" + lock);
+        int err = nodeData.getErrorCount();
+        if (err > 0) {
+          errorCount.addAndGet(nodeData.getErrorCount());
+        } else {
+          hosts.add(nodeData.getHosts());
+        }
+      });
+    } catch (KeeperException | InterruptedException ex) {
+      if (Thread.currentThread().isInterrupted()) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(ex);
+      }
+      LOG.info("Could not read node names from ZooKeeper for path {}", path, 
ex);
+      errorCount.incrementAndGet();
+    }
+    return new Result<>(errorCount.get(), hosts);
+  }
+
+  @Parameters(commandDescription = "show service status")
+  public static class Opts {
+    @Parameter(names = "--json", description = "provide output in json format 
(--noHosts ignored)")
+    boolean json = false;
+    @Parameter(names = "--noHosts",
+        description = "provide a summary of service counts without host 
details")
+    boolean noHosts = false;
+  }
+
+  /**
+   * Provides explicit method names instead of generic getFirst to get the 
error count and getSecond
+   * hosts information
+   *
+   * @param <A> errorCount
+   * @param <B> hosts
+   */
+  private static class Result<A extends Integer,B> extends Pair<A,B> {
+    public Result(A errorCount, B hosts) {
+      super(errorCount, hosts);
+    }
+
+    public A getErrorCount() {
+      return getFirst();
+    }
+
+    public B getHosts() {
+      return getSecond();
+    }
+  }
+}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReport.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReport.java
new file mode 100644
index 0000000000..0951831028
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReport.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util.serviceStatus;
+
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.gson.Gson;
+
+/**
+ * Wrapper for JSON formatted report.
+ */
+public class ServiceStatusReport {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ServiceStatusReport.class);
+
+  private static final Gson gson = new Gson();
+
+  private static final DateTimeFormatter rptTimeFmt =
+      DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'");
+  private static final String I2 = "  ";
+  private static final String I4 = "    ";
+  private static final String I6 = "      ";
+
+  private final String reportTime;
+  private final int zkReadErrors;
+  private final boolean noHosts;
+  private final Map<ReportKey,StatusSummary> summaries;
+
+  public ServiceStatusReport(final Map<ReportKey,StatusSummary> summaries, 
final boolean noHosts) {
+    reportTime = rptTimeFmt.format(ZonedDateTime.now(ZoneId.of("UTC")));
+    zkReadErrors = 
summaries.values().stream().map(StatusSummary::getErrorCount)
+        .reduce(Integer::sum).orElse(0);
+    this.noHosts = noHosts;
+    this.summaries = summaries;
+  }
+
+  public String getReportTime() {
+    return reportTime;
+  }
+
+  public int getTotalZkReadErrors() {
+    return zkReadErrors;
+  }
+
+  public Map<ReportKey,StatusSummary> getSummaries() {
+    return summaries;
+  }
+
+  public String toJson() {
+    return gson.toJson(this, ServiceStatusReport.class);
+  }
+
+  public static ServiceStatusReport fromJson(final String json) {
+    return gson.fromJson(json, ServiceStatusReport.class);
+  }
+
+  public String report(final StringBuilder sb) {
+    sb.append("Report time: 
").append(rptTimeFmt.format(ZonedDateTime.now(ZoneId.of("UTC"))))
+        .append("\n");
+    int zkErrors = 
summaries.values().stream().map(StatusSummary::getErrorCount)
+        .reduce(Integer::sum).orElse(0);
+    sb.append("ZooKeeper read errors: ").append(zkErrors).append("\n");
+
+    fmtServiceStatus(sb, ReportKey.MANAGER, summaries.get(ReportKey.MANAGER), 
noHosts);
+    fmtServiceStatus(sb, ReportKey.MONITOR, summaries.get(ReportKey.MONITOR), 
noHosts);
+    fmtServiceStatus(sb, ReportKey.GC, summaries.get(ReportKey.GC), noHosts);
+    fmtServiceStatus(sb, ReportKey.T_SERVER, 
summaries.get(ReportKey.T_SERVER), noHosts);
+    fmtResourceGroups(sb, ReportKey.S_SERVER, 
summaries.get(ReportKey.S_SERVER), noHosts);
+    fmtServiceStatus(sb, ReportKey.COORDINATOR, 
summaries.get(ReportKey.COORDINATOR), noHosts);
+    fmtResourceGroups(sb, ReportKey.COMPACTOR, 
summaries.get(ReportKey.COMPACTOR), noHosts);
+
+    sb.append("\n");
+    LOG.trace("fmtStatus - with hosts: {}", summaries);
+    return sb.toString();
+  }
+
+  private void fmtServiceStatus(final StringBuilder sb, final ReportKey 
displayNames,
+      final StatusSummary summary, boolean noHosts) {
+    if (summary == null) {
+      sb.append(displayNames).append(": unavailable").append("\n");
+      return;
+    }
+
+    fmtCounts(sb, summary);
+
+    // skip host info if requested
+    if (noHosts) {
+      return;
+    }
+    if (summary.getServiceCount() > 0) {
+      var hosts = summary.getServiceByGroups();
+      hosts.values().forEach(s -> s.forEach(h -> 
sb.append(I2).append(h).append("\n")));
+    }
+  }
+
+  private void fmtCounts(StringBuilder sb, StatusSummary summary) {
+    sb.append(summary.getDisplayName()).append(": count: 
").append(summary.getServiceCount());
+    if (summary.getErrorCount() > 0) {
+      sb.append(", (ZooKeeper errors: 
").append(summary.getErrorCount()).append(")\n");
+    } else {
+      sb.append("\n");
+    }
+  }
+
+  private void fmtResourceGroups(final StringBuilder sb, final ReportKey 
reportKey,
+      final StatusSummary summary, boolean noHosts) {
+    if (summary == null) {
+      sb.append(reportKey).append(": unavailable").append("\n");
+      return;
+    }
+
+    fmtCounts(sb, summary);
+
+    // skip host info if requested
+    if (noHosts) {
+      return;
+    }
+
+    if (!summary.getResourceGroups().isEmpty()) {
+      sb.append(I2).append("resource groups:\n");
+      summary.getResourceGroups().forEach(g -> 
sb.append(I4).append(g).append("\n"));
+
+      if (summary.getServiceCount() > 0) {
+        sb.append(I2).append("hosts (by group):\n");
+        var groups = summary.getServiceByGroups();
+        groups.forEach((g, h) -> {
+          sb.append(I4).append(g).append(" 
(").append(h.size()).append(")").append(":\n");
+          h.forEach(n -> {
+            sb.append(I6).append(n).append("\n");
+          });
+        });
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "ServiceStatusReport{reportTime='" + reportTime + '\'' + ", 
zkReadErrors=" + zkReadErrors
+        + ", noHosts=" + noHosts + ", status=" + summaries + '}';
+  }
+
+  public enum ReportKey {
+    COMPACTOR("Compactors"),
+    COORDINATOR("Coordinators"),
+    GC("Garbage Collectors"),
+    MANAGER("Managers"),
+    MONITOR("Monitors"),
+    S_SERVER("Scan Servers"),
+    T_SERVER("Tablet Servers");
+
+    private final String displayName;
+
+    ReportKey(final String name) {
+      this.displayName = name;
+    }
+
+    public String getDisplayName() {
+      return displayName;
+    }
+  }
+}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/StatusSummary.java
 
b/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/StatusSummary.java
new file mode 100644
index 0000000000..9ae9e87d5e
--- /dev/null
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/util/serviceStatus/StatusSummary.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util.serviceStatus;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+public class StatusSummary {
+
+  private final ServiceStatusReport.ReportKey serviceType;
+  private final Set<String> resourceGroups;
+  private final Map<String,Set<String>> serviceByGroups;
+  private final int serviceCount;
+  private final int errorCount;
+
+  public StatusSummary(ServiceStatusReport.ReportKey serviceType, final 
Set<String> resourceGroups,
+      final Map<String,Set<String>> serviceByGroups, final int errorCount) {
+    this.serviceType = serviceType;
+    this.resourceGroups = resourceGroups;
+    this.serviceByGroups = serviceByGroups;
+    this.serviceCount =
+        
serviceByGroups.values().stream().map(Set::size).reduce(Integer::sum).orElse(0);
+    this.errorCount = errorCount;
+  }
+
+  public ServiceStatusReport.ReportKey getServiceType() {
+    return serviceType;
+  }
+
+  public String getDisplayName() {
+    return serviceType.getDisplayName();
+  }
+
+  public Set<String> getResourceGroups() {
+    return resourceGroups;
+  }
+
+  public Map<String,Set<String>> getServiceByGroups() {
+    return serviceByGroups;
+  }
+
+  public int getServiceCount() {
+    return serviceCount;
+  }
+
+  public int getErrorCount() {
+    return errorCount;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof StatusSummary)) {
+      return false;
+    }
+    StatusSummary that = (StatusSummary) o;
+    return serviceCount == that.serviceCount && errorCount == that.errorCount
+        && serviceType == that.serviceType && Objects.equals(resourceGroups, 
that.resourceGroups)
+        && Objects.equals(serviceByGroups, that.serviceByGroups);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(serviceType, resourceGroups, serviceCount, 
serviceByGroups, errorCount);
+  }
+
+  @Override
+  public String toString() {
+    return "StatusSummary{serviceName=" + serviceType + ", resourceGroups=" + 
resourceGroups
+        + ", serviceCount=" + serviceCount + ", names=" + serviceByGroups + ", 
errorCount="
+        + errorCount + '}';
+  }
+}
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
new file mode 100644
index 0000000000..f4af749798
--- /dev/null
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
@@ -0,0 +1,406 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZGC_LOCK;
+import static org.apache.accumulo.server.util.ServiceStatusCmd.NO_GROUP_TAG;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.util.serviceStatus.ServiceStatusReport;
+import org.apache.accumulo.server.util.serviceStatus.StatusSummary;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServiceStatusCmdTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ServiceStatusCmdTest.class);
+
+  private ServerContext context;
+  private String zRoot;
+  private ZooReader zooReader;
+
+  @BeforeEach
+  public void populateContext() {
+    InstanceId iid = InstanceId.of(UUID.randomUUID());
+    zRoot = "/accumulo/" + iid.canonical();
+    context = createMock(ServerContext.class);
+    expect(context.getInstanceID()).andReturn(iid).anyTimes();
+    expect(context.getZooKeeperRoot()).andReturn(zRoot).anyTimes();
+
+    zooReader = createMock(ZooReader.class);
+
+    expect(context.getZooReader()).andReturn(zooReader).anyTimes();
+
+    replay(context);
+  }
+
+  @AfterEach
+  public void validateMocks() {
+    verify(context, zooReader);
+  }
+
+  @Test
+  void testManagerHosts() throws Exception {
+    String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001";
+    String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002";
+    String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003";
+
+    String host1 = "hostA:8080";
+    String host2 = "hostB:9090";
+    String host3 = "host1:9091";
+
+    String lockPath = zRoot + Constants.ZMANAGER_LOCK;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, 
lock2Name, lock3Name))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock1Name))).andReturn(host1.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock2Name))).andReturn(host2.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock3Name))).andReturn(host3.getBytes(UTF_8))
+        .anyTimes();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getManagerStatus(zooReader, zRoot);
+    LOG.info("manager status data: {}", status);
+
+    assertEquals(3, status.getServiceCount());
+
+    // expect sorted by name
+    Set<String> hosts = new TreeSet<>(List.of(host1, host2, host3));
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put(NO_GROUP_TAG, hosts);
+
+    StatusSummary expected =
+        new StatusSummary(ServiceStatusReport.ReportKey.MANAGER, Set.of(), 
hostByGroup, 0);
+
+    assertEquals(expected.hashCode(), status.hashCode());
+    assertEquals(expected.getDisplayName(), status.getDisplayName());
+    assertEquals(expected.getResourceGroups(), status.getResourceGroups());
+    assertEquals(expected.getServiceByGroups(), status.getServiceByGroups());
+    assertEquals(expected.getServiceCount(), status.getServiceCount());
+    assertEquals(expected.getErrorCount(), status.getErrorCount());
+    assertEquals(expected, status);
+  }
+
+  @Test
+  void testMonitorHosts() throws Exception {
+    String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001";
+    String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002";
+
+    String host1 = "hostA:8080";
+    String host2 = "host1:9091";
+
+    String lockPath = zRoot + Constants.ZMONITOR_LOCK;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, 
lock2Name)).anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock1Name))).andReturn(host1.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock2Name))).andReturn(host2.getBytes(UTF_8))
+        .anyTimes();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getMonitorStatus(zooReader, zRoot);
+    LOG.info("monitor status data: {}", status);
+
+    assertEquals(2, status.getServiceCount());
+
+    // expect sorted by name
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of(host1, host2)));
+
+    StatusSummary expected =
+        new StatusSummary(ServiceStatusReport.ReportKey.MONITOR, Set.of(), 
hostByGroup, 0);
+
+    assertEquals(expected.hashCode(), status.hashCode());
+    assertEquals(expected.getDisplayName(), status.getDisplayName());
+    assertEquals(expected.getResourceGroups(), status.getResourceGroups());
+    assertEquals(expected.getServiceByGroups(), status.getServiceByGroups());
+    assertEquals(expected.getServiceCount(), status.getServiceCount());
+    assertEquals(expected.getErrorCount(), status.getErrorCount());
+    assertEquals(expected, status);
+  }
+
+  @Test
+  void testTServerHosts() throws Exception {
+    String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001";
+    String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002";
+    String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003";
+
+    String host1 = "hostA:8080";
+    String host2 = "hostB:9090";
+    String host3 = "host1:9091";
+
+    String basePath = zRoot + Constants.ZTSERVERS;
+    expect(zooReader.getChildren(eq(basePath))).andReturn(List.of(host1, 
host2, host3)).anyTimes();
+
+    expect(zooReader.getChildren(eq(basePath + "/" + 
host1))).andReturn(List.of(lock1Name)).once();
+    expect(zooReader.getData(eq(basePath + "/" + host1 + "/" + lock1Name)))
+        .andReturn(("TSERV_CLIENT=" + host1).getBytes(UTF_8)).anyTimes();
+
+    expect(zooReader.getChildren(eq(basePath + "/" + 
host2))).andReturn(List.of(lock2Name)).once();
+    expect(zooReader.getData(eq(basePath + "/" + host2 + "/" + lock2Name)))
+        .andReturn(("TSERV_CLIENT=" + host2).getBytes(UTF_8)).anyTimes();
+
+    expect(zooReader.getChildren(eq(basePath + "/" + 
host3))).andReturn(List.of(lock3Name)).once();
+    expect(zooReader.getData(eq(basePath + "/" + host3 + "/" + lock3Name)))
+        .andReturn(("TSERV_CLIENT=" + host3).getBytes(UTF_8)).anyTimes();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getTServerStatus(zooReader, zRoot);
+    LOG.info("tserver status data: {}", status);
+
+    assertEquals(3, status.getServiceCount());
+
+    // expect sorted by name
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of(host1, host2, host3)));
+
+    StatusSummary expected =
+        new StatusSummary(ServiceStatusReport.ReportKey.T_SERVER, Set.of(), 
hostByGroup, 0);
+
+    assertEquals(expected.hashCode(), status.hashCode());
+    assertEquals(expected.getDisplayName(), status.getDisplayName());
+    assertEquals(expected.getResourceGroups(), status.getResourceGroups());
+    assertEquals(expected.getServiceByGroups(), status.getServiceByGroups());
+    assertEquals(expected.getServiceCount(), status.getServiceCount());
+    assertEquals(expected.getErrorCount(), status.getErrorCount());
+    assertEquals(expected, status);
+  }
+
+  @Test
+  void testScanServerHosts() throws Exception {
+    UUID uuid1 = UUID.randomUUID();
+    String lock1Name = "zlock#" + uuid1 + "#0000000001";
+    UUID uuid2 = UUID.randomUUID();
+    String lock2Name = "zlock#" + uuid2 + "#0000000022";
+    UUID uuid3 = UUID.randomUUID();
+    String lock3Name = "zlock#" + uuid3 + "#0000000033";
+    String lock4Name = "zlock#" + uuid3 + "#0000000044";
+
+    // UUID uuidLock = UUID.randomUUID();
+
+    String host1 = "host1:8080";
+    String host2 = "host2:9090";
+    String host3 = "host3:9091";
+    String host4 = "host4:9091";
+
+    String lockPath = zRoot + Constants.ZSSERVERS;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(host1, 
host2, host3, host4))
+        .anyTimes();
+
+    expect(zooReader.getChildren(eq(lockPath + "/" + 
host1))).andReturn(List.of(lock1Name)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + host1 + "/" + lock1Name)))
+        .andReturn((UUID.randomUUID() + ",rg1").getBytes(UTF_8)).once();
+
+    expect(zooReader.getChildren(eq(lockPath + "/" + 
host2))).andReturn(List.of(lock2Name)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + host2 + "/" + lock2Name)))
+        .andReturn((UUID.randomUUID() + ",default").getBytes(UTF_8)).once();
+
+    expect(zooReader.getChildren(eq(lockPath + "/" + 
host3))).andReturn(List.of(lock3Name)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + host3 + "/" + lock3Name)))
+        .andReturn((UUID.randomUUID() + ",rg1").getBytes(UTF_8)).once();
+
+    expect(zooReader.getChildren(eq(lockPath + "/" + 
host4))).andReturn(List.of(lock4Name)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + host4 + "/" + lock4Name)))
+        .andReturn((UUID.randomUUID() + ",default").getBytes(UTF_8)).once();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getScanServerStatus(zooReader, zRoot);
+    assertEquals(4, status.getServiceCount());
+
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put("default", new TreeSet<>(List.of("host2:9090", 
"host4:9091")));
+    hostByGroup.put("rg1", new TreeSet<>(List.of("host1:8080", "host3:9091")));
+
+    StatusSummary expected = new 
StatusSummary(ServiceStatusReport.ReportKey.S_SERVER,
+        Set.of("default", "rg1"), hostByGroup, 0);
+
+    assertEquals(expected, status);
+
+  }
+
+  @Test
+  void testCoordinatorHosts() throws Exception {
+    String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001";
+    String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000002";
+    String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000003";
+
+    String host1 = "hostA:8080";
+    String host2 = "hostB:9090";
+    String host3 = "host1:9091";
+
+    String lockPath = zRoot + Constants.ZCOORDINATOR_LOCK;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, 
lock2Name, lock3Name))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock1Name))).andReturn(host1.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock2Name))).andReturn(host2.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock3Name))).andReturn(host3.getBytes(UTF_8))
+        .anyTimes();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getCoordinatorStatus(zooReader, zRoot);
+    LOG.info("tserver status data: {}", status);
+
+    assertEquals(3, status.getServiceCount());
+
+    // expect sorted by name
+    Set<String> hosts = new TreeSet<>(List.of(host1, host2, host3));
+    Map<String,Set<String>> hostByGroup = new TreeMap<>();
+    hostByGroup.put(NO_GROUP_TAG, hosts);
+
+    StatusSummary expected =
+        new StatusSummary(ServiceStatusReport.ReportKey.COORDINATOR, Set.of(), 
hostByGroup, 0);
+
+    assertEquals(expected.hashCode(), status.hashCode());
+    assertEquals(expected.getDisplayName(), status.getDisplayName());
+    assertEquals(expected.getResourceGroups(), status.getResourceGroups());
+    assertEquals(expected.getServiceByGroups(), status.getServiceByGroups());
+    assertEquals(expected.getServiceCount(), status.getServiceCount());
+    assertEquals(expected.getErrorCount(), status.getErrorCount());
+    assertEquals(expected, status);
+  }
+
+  @Test
+  public void testCompactorStatus() throws Exception {
+    String lockPath = zRoot + Constants.ZCOMPACTORS;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of("q1", 
"q2")).once();
+
+    expect(zooReader.getChildren(eq(lockPath + "/q1")))
+        .andReturn(List.of("hostA:8080", "hostC:8081")).once();
+    expect(zooReader.getChildren(eq(lockPath + "/q2")))
+        .andReturn(List.of("hostB:9090", "hostD:9091")).once();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getCompactorStatus(zooReader, zRoot);
+    LOG.info("compactor group counts: {}", status);
+    assertEquals(2, status.getResourceGroups().size());
+  }
+
+  @Test
+  public void testGcHosts() throws Exception {
+
+    String lockPath = zRoot + ZGC_LOCK;
+    UUID uuid1 = UUID.randomUUID();
+    String lock1Name = "zlock#" + uuid1 + "#0000000001";
+    UUID uuid2 = UUID.randomUUID();
+    String lock2Name = "zlock#" + uuid2 + "#0000000022";
+
+    String host1 = "host1:8080";
+    String host2 = "host2:9090";
+
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, 
lock2Name)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + lock1Name)))
+        .andReturn(("GC_CLIENT=" + host1).getBytes(UTF_8)).once();
+    expect(zooReader.getData(eq(lockPath + "/" + lock2Name)))
+        .andReturn(("GC_CLIENT=" + host2).getBytes(UTF_8)).once();
+
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getGcStatus(zooReader, zRoot);
+    LOG.info("gc server counts: {}", status);
+    assertEquals(0, status.getResourceGroups().size());
+    assertEquals(2, status.getServiceCount());
+    assertEquals(0, status.getErrorCount());
+    assertEquals(1, status.getServiceByGroups().size());
+    assertEquals(2, status.getServiceByGroups().get(NO_GROUP_TAG).size());
+    assertEquals(new TreeSet<>(List.of(host1, host2)),
+        status.getServiceByGroups().get(NO_GROUP_TAG));
+  }
+
+  /**
+   * Simulates node being deleted after lock list is read from ZooKeeper. 
Expect that the no node
+   * error is skipped and available hosts are returned.
+   */
+  @Test
+  void zkNodeDeletedTest() throws Exception {
+    String lock1Name = "zlock#" + UUID.randomUUID() + "#0000000001";
+    String lock2Name = "zlock#" + UUID.randomUUID() + "#0000000022";
+    String lock3Name = "zlock#" + UUID.randomUUID() + "#0000000099";
+    String host2 = "hostZ:8080";
+    String host3 = "hostA:8080";
+
+    String lockPath = zRoot + Constants.ZMANAGER_LOCK;
+    expect(zooReader.getChildren(eq(lockPath))).andReturn(List.of(lock1Name, 
lock2Name, lock3Name))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + lock1Name)))
+        .andThrow(new KeeperException.NoNodeException("no node forced 
exception")).anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock2Name))).andReturn(host2.getBytes(UTF_8))
+        .anyTimes();
+    expect(zooReader.getData(eq(lockPath + "/" + 
lock3Name))).andReturn(host3.getBytes(UTF_8))
+        .anyTimes();
+    replay(zooReader);
+
+    ServiceStatusCmd cmd = new ServiceStatusCmd();
+    StatusSummary status = cmd.getManagerStatus(zooReader, zRoot);
+    LOG.info("manager status data: {}", status);
+
+    assertEquals(1, status.getServiceByGroups().size());
+    assertEquals(2, status.getServiceByGroups().get(NO_GROUP_TAG).size());
+    assertEquals(1, status.getErrorCount());
+
+    // host 1 missing - no node exception
+    Set<String> sortedHosts = new TreeSet<>(List.of(host3, host2));
+    assertEquals(sortedHosts, status.getServiceByGroups().get(NO_GROUP_TAG));
+  }
+
+  @Test
+  public void testServiceStatusCommandOpts() {
+    replay(zooReader); // needed for @AfterAll verify
+    ServiceStatusCmd.Opts opts = new ServiceStatusCmd.Opts();
+    assertFalse(opts.json);
+    assertFalse(opts.noHosts);
+  }
+
+}
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReportTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReportTest.java
new file mode 100644
index 0000000000..b90e2c8378
--- /dev/null
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/util/serviceStatus/ServiceStatusReportTest.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util.serviceStatus;
+
+import static org.apache.accumulo.server.util.ServiceStatusCmd.NO_GROUP_TAG;
+import static 
org.apache.accumulo.server.util.serviceStatus.ServiceStatusReport.ReportKey.MANAGER;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ServiceStatusReportTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ServiceStatusReport.class);
+
+  @Test
+  public void printOutputCountTest() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = 
buildHostStatus();
+    ServiceStatusReport report = new ServiceStatusReport(services, true);
+    StringBuilder sb = new StringBuilder(8192);
+    report.report(sb);
+    LOG.info("Report: \n{}", sb);
+    assertTrue(sb.length() > 0);
+  }
+
+  @Test
+  public void printOutputHostTest() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = 
buildHostStatus();
+    ServiceStatusReport report = new ServiceStatusReport(services, false);
+    StringBuilder sb = new StringBuilder(8192);
+    report.report(sb);
+    LOG.info("Report: \n{}", sb);
+    assertTrue(sb.length() > 0);
+  }
+
+  @Test
+  public void printJsonHostTest() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = 
buildHostStatus();
+    ServiceStatusReport report = new ServiceStatusReport(services, false);
+    var output = report.toJson();
+    LOG.info("{}", output);
+    assertFalse(output.isEmpty());
+  }
+
+  @Test
+  public void jsonRoundTripTest() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = new 
TreeMap<>();
+
+    Map<String,Set<String>> managerByGroup = new TreeMap<>();
+    managerByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of("hostZ:8080", 
"hostA:9090")));
+    StatusSummary managerSummary = new StatusSummary(MANAGER, Set.of(), 
managerByGroup, 1);
+    services.put(MANAGER, managerSummary);
+    ServiceStatusReport report = new ServiceStatusReport(services, false);
+    var encoded = report.toJson();
+
+    ServiceStatusReport decoded = ServiceStatusReport.fromJson(encoded);
+    assertNotNull(decoded.getReportTime());
+    assertEquals(1, decoded.getTotalZkReadErrors());
+    assertEquals(1, report.getSummaries().size());
+
+    var byGroup = report.getSummaries().get(MANAGER).getServiceByGroups();
+    assertEquals(new TreeSet<>(List.of("hostZ:8080", "hostA:9090")), 
byGroup.get(NO_GROUP_TAG));
+  }
+
+  /**
+   * validate reduce / sum is correct
+   */
+  @Test
+  public void sumTest() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = 
buildHostStatus();
+    int count =
+        
services.values().stream().map(StatusSummary::getErrorCount).reduce(Integer::sum).orElse(0);
+    assertEquals(4, count);
+  }
+
+  private Map<ServiceStatusReport.ReportKey,StatusSummary> buildHostStatus() {
+    final Map<ServiceStatusReport.ReportKey,StatusSummary> services = new 
TreeMap<>();
+
+    Map<String,Set<String>> managerByGroup = new TreeMap<>();
+    managerByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of("host1:8080", 
"host2:9090")));
+    StatusSummary managerSummary = new StatusSummary(MANAGER, Set.of(), 
managerByGroup, 1);
+    services.put(MANAGER, managerSummary);
+
+    Map<String,Set<String>> monitorByGroup = new TreeMap<>();
+    monitorByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of("host1:8080", 
"host2:9090")));
+    StatusSummary monitorSummary =
+        new StatusSummary(ServiceStatusReport.ReportKey.MONITOR, Set.of(), 
monitorByGroup, 0);
+    services.put(ServiceStatusReport.ReportKey.MONITOR, monitorSummary);
+
+    Map<String,Set<String>> gcByGroup = new TreeMap<>();
+    gcByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of("host1:8080", 
"host2:9090")));
+
+    StatusSummary gcSummary =
+        new StatusSummary(ServiceStatusReport.ReportKey.GC, Set.of(), 
gcByGroup, 0);
+    services.put(ServiceStatusReport.ReportKey.GC, gcSummary);
+
+    Map<String,Set<String>> tserverByGroup = new TreeMap<>();
+    tserverByGroup.put(NO_GROUP_TAG,
+        new TreeSet<>(List.of("host2:9090", "host4:9091", "host1:8080", 
"host3:9091")));
+
+    StatusSummary tserverSummary =
+        new StatusSummary(ServiceStatusReport.ReportKey.T_SERVER, Set.of(), 
tserverByGroup, 1);
+    services.put(ServiceStatusReport.ReportKey.T_SERVER, tserverSummary);
+
+    Map<String,Set<String>> sserverByGroup = new TreeMap<>();
+    sserverByGroup.put("default", new TreeSet<>(List.of("host2:9090")));
+    sserverByGroup.put("rg1", new TreeSet<>(List.of("host1:8080", 
"host3:9091")));
+    sserverByGroup.put("rg2", new TreeSet<>(List.of("host4:9091")));
+
+    StatusSummary scanServerSummary = new 
StatusSummary(ServiceStatusReport.ReportKey.S_SERVER,
+        new TreeSet<>(List.of("default", "rg1", "rg2")), sserverByGroup, 2);
+    services.put(ServiceStatusReport.ReportKey.S_SERVER, scanServerSummary);
+
+    Map<String,Set<String>> coordinatorByGroup = new TreeMap<>();
+    coordinatorByGroup.put(NO_GROUP_TAG, new TreeSet<>(List.of("host4:9090", 
"host2:9091")));
+    StatusSummary coordinatorSummary = new 
StatusSummary(ServiceStatusReport.ReportKey.COORDINATOR,
+        Set.of(), coordinatorByGroup, 0);
+    services.put(ServiceStatusReport.ReportKey.COORDINATOR, 
coordinatorSummary);
+
+    Map<String,Set<String>> compactorByGroup = new TreeMap<>();
+    compactorByGroup.put("q2", new TreeSet<>(List.of("host5:8080", 
"host2:9090", "host4:9091")));
+    compactorByGroup.put("q1", new TreeSet<>(List.of("host3:8080", 
"host1:9091")));
+
+    StatusSummary compactorSummary = new 
StatusSummary(ServiceStatusReport.ReportKey.COMPACTOR,
+        new TreeSet<>(List.of("q2", "q1")), compactorByGroup, 0);
+    services.put(ServiceStatusReport.ReportKey.COMPACTOR, compactorSummary);
+
+    return services;
+  }
+
+}

Reply via email to