ctubbsii commented on code in PR #5012:
URL: https://github.com/apache/accumulo/pull/5012#discussion_r1925646584


##########
core/src/main/scripts/generate-flatbuffers.sh:
##########
@@ -0,0 +1,52 @@
+#! /usr/bin/env bash
+#
+# 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.
+#
+
+# This code will generate the FlatBuffers code that is used
+# to serialize Micrometer Meter objects into a byte[] inside
+# the Thrift response object returned by the MetricServiceHandler
+# that is called by the Monitor.
+
+[[ -z $REQUIRED_FB_VERSION ]] && REQUIRED_FB_VERSION='24.3.25'
+
+# Test to see if we have thrift installed

Review Comment:
   ```suggestion
   # Test to see if we have flatc installed
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/metrics/MetricServiceHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.metrics;
+
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService;
+import org.apache.accumulo.core.metrics.thrift.MetricSource;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.net.HostAndPort;
+import com.google.flatbuffers.FlatBufferBuilder;
+
+import io.micrometer.core.instrument.Metrics;
+
+public class MetricServiceHandler implements MetricService.Iface {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(MetricServiceHandler.class);
+
+  private final MetricSource type;
+  private final String resourceGroup;
+  private final ServerContext ctx;
+
+  private String host;
+
+  public MetricServiceHandler(MetricSource source, String resourceGroup, 
ServerContext ctx) {
+    this.type = source;
+    this.resourceGroup = resourceGroup;
+    this.ctx = ctx;
+  }
+
+  public void setHost(HostAndPort host) {
+    this.host = host.toString();
+  }
+
+  @Override
+  public MetricResponse getMetrics(TInfo tinfo, TCredentials credentials) 
throws TException {
+
+    if (!(ctx.getSecurityOperation().isSystemUser(credentials)
+        && ctx.getSecurityOperation().authenticateUser(credentials, 
credentials))) {
+      throw new ThriftSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED);
+    }
+
+    final FlatBufferBuilder builder = new FlatBufferBuilder(1024);
+    final MetricResponseWrapper response = new MetricResponseWrapper(builder);

Review Comment:
   These are fine as is, but I just just pointing out that these situations are 
perfect for using `var`:
   
   ```suggestion
       final var builder = new FlatBufferBuilder(1024);
       final var response = new MetricResponseWrapper(builder);
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.monitor.next;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService.Client;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.transport.TTransportException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.eclipse.jetty.util.NanoTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.net.HostAndPort;
+
+public class InformationFetcher implements 
RemovalListener<ServerId,MetricResponse>, Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InformationFetcher.class);
+
+  public static class InstanceSummary {
+    private final String instanceName;
+    private final String instanceUUID;
+    private final Set<String> zooKeepers;
+    private final Set<String> volumes;
+    private final String version;
+
+    public InstanceSummary(String instanceName, String instanceUUID, 
Set<String> zooKeepers,
+        Set<String> volumes, String version) {
+      super();
+      this.instanceName = instanceName;
+      this.instanceUUID = instanceUUID;
+      this.zooKeepers = zooKeepers;
+      this.volumes = volumes;
+      this.version = version;
+    }
+
+    public String getInstanceName() {
+      return instanceName;
+    }
+
+    public String getInstanceUUID() {
+      return instanceUUID;
+    }
+
+    public Set<String> getZooKeepers() {
+      return zooKeepers;
+    }
+
+    public Set<String> getVolumes() {
+      return volumes;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+  }
+
+  private class MetricFetcher implements Runnable {
+
+    private final ServerContext ctx;
+    private final ServerId server;
+    private final SystemInformation summary;
+
+    private MetricFetcher(ServerContext ctx, ServerId server, 
SystemInformation summary) {
+      this.ctx = ctx;
+      this.server = server;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.METRICS,
+            HostAndPort.fromParts(server.getHost(), server.getPort()), ctx);
+        try {
+          MetricResponse response = 
metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds());
+          summary.processResponse(server, response);
+        } finally {
+          ThriftUtil.returnClient(metricsClient, ctx);
+        }
+      } catch (Exception e) {
+        LOG.warn("Error trying to get metrics from server: {}. Error message: 
{}", server,
+            e.getMessage());
+        summary.processError(server);
+      }
+    }
+
+  }
+
+  private class TableInformationFetcher implements Runnable {
+    private final ServerContext ctx;
+    private final String table;
+    private final SystemInformation summary;
+
+    private TableInformationFetcher(ServerContext ctx, String tableName,
+        SystemInformation summary) {
+      this.ctx = ctx;
+      this.table = tableName;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try (Stream<TabletInformation> tablets =
+          this.ctx.tableOperations().getTabletInformation(table, new Range())) 
{
+        tablets.forEach(t -> summary.processTabletInformation(table, t));
+      } catch (TableNotFoundException e) {
+        LOG.warn(
+            "TableNotFoundException thrown while trying to gather information 
for table: " + table,
+            e);
+      } catch (Exception e) {
+        LOG.warn("Interrupted while trying to gather information for table: 
{}", table);
+      }
+    }
+  }
+
+  private class CompactionListFetcher implements Runnable {
+
+    private final String coordinatorMissingMsg =
+        "Error getting the compaction coordinator client. Check that the 
Manager is running.";
+
+    private final SystemInformation summary;
+
+    public CompactionListFetcher(SystemInformation summary) {
+      this.summary = summary;
+    }
+
+    // Copied from Monitor
+    private Map<String,TExternalCompactionList> getLongRunningCompactions() {
+      Set<ServerId> managers = 
ctx.instanceOperations().getServers(ServerId.Type.MANAGER);
+      if (managers.isEmpty()) {
+        throw new IllegalStateException(coordinatorMissingMsg);
+      }
+      ServerId manager = managers.iterator().next();
+      HostAndPort hp = HostAndPort.fromParts(manager.getHost(), 
manager.getPort());
+      try {
+        CompactionCoordinatorService.Client client =
+            ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, hp, ctx);
+        try {
+          return client.getLongRunningCompactions(TraceUtil.traceInfo(), 
ctx.rpcCreds());
+        } catch (Exception e) {
+          throw new IllegalStateException("Unable to get running compactions 
from " + hp, e);
+        } finally {
+          if (client != null) {
+            ThriftUtil.returnClient(client, ctx);
+          }
+        }
+      } catch (TTransportException e) {
+        LOG.error("Unable to get Compaction coordinator at {}", hp);
+        throw new IllegalStateException(coordinatorMissingMsg, e);
+      }
+    }
+
+    @Override
+    public void run() {
+      try {
+        summary.processExternalCompactionList(getLongRunningCompactions());
+      } catch (Exception e) {
+        LOG.warn("Error gathering running compaction information. Error 
message: {}",
+            e.getMessage());

Review Comment:
   ```suggestion
           LOG.warn("Error gathering running compaction information.", e);
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.monitor.next;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService.Client;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.transport.TTransportException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.eclipse.jetty.util.NanoTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.net.HostAndPort;
+
+public class InformationFetcher implements 
RemovalListener<ServerId,MetricResponse>, Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InformationFetcher.class);
+
+  public static class InstanceSummary {
+    private final String instanceName;
+    private final String instanceUUID;
+    private final Set<String> zooKeepers;
+    private final Set<String> volumes;
+    private final String version;
+
+    public InstanceSummary(String instanceName, String instanceUUID, 
Set<String> zooKeepers,
+        Set<String> volumes, String version) {
+      super();
+      this.instanceName = instanceName;
+      this.instanceUUID = instanceUUID;
+      this.zooKeepers = zooKeepers;
+      this.volumes = volumes;
+      this.version = version;
+    }
+
+    public String getInstanceName() {
+      return instanceName;
+    }
+
+    public String getInstanceUUID() {
+      return instanceUUID;
+    }
+
+    public Set<String> getZooKeepers() {
+      return zooKeepers;
+    }
+
+    public Set<String> getVolumes() {
+      return volumes;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+  }
+
+  private class MetricFetcher implements Runnable {
+
+    private final ServerContext ctx;
+    private final ServerId server;
+    private final SystemInformation summary;
+
+    private MetricFetcher(ServerContext ctx, ServerId server, 
SystemInformation summary) {
+      this.ctx = ctx;
+      this.server = server;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.METRICS,
+            HostAndPort.fromParts(server.getHost(), server.getPort()), ctx);
+        try {
+          MetricResponse response = 
metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds());
+          summary.processResponse(server, response);
+        } finally {
+          ThriftUtil.returnClient(metricsClient, ctx);
+        }
+      } catch (Exception e) {
+        LOG.warn("Error trying to get metrics from server: {}. Error message: 
{}", server,
+            e.getMessage());
+        summary.processError(server);
+      }
+    }
+
+  }
+
+  private class TableInformationFetcher implements Runnable {
+    private final ServerContext ctx;
+    private final String table;
+    private final SystemInformation summary;
+
+    private TableInformationFetcher(ServerContext ctx, String tableName,
+        SystemInformation summary) {
+      this.ctx = ctx;
+      this.table = tableName;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try (Stream<TabletInformation> tablets =
+          this.ctx.tableOperations().getTabletInformation(table, new Range())) 
{
+        tablets.forEach(t -> summary.processTabletInformation(table, t));
+      } catch (TableNotFoundException e) {
+        LOG.warn(
+            "TableNotFoundException thrown while trying to gather information 
for table: " + table,
+            e);
+      } catch (Exception e) {
+        LOG.warn("Interrupted while trying to gather information for table: 
{}", table);

Review Comment:
   ```suggestion
           LOG.warn("Interrupted while trying to gather information for table: 
{}", table, e);
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.monitor.next;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService.Client;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.transport.TTransportException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.eclipse.jetty.util.NanoTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.net.HostAndPort;
+
+public class InformationFetcher implements 
RemovalListener<ServerId,MetricResponse>, Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InformationFetcher.class);
+
+  public static class InstanceSummary {
+    private final String instanceName;
+    private final String instanceUUID;
+    private final Set<String> zooKeepers;
+    private final Set<String> volumes;
+    private final String version;
+
+    public InstanceSummary(String instanceName, String instanceUUID, 
Set<String> zooKeepers,
+        Set<String> volumes, String version) {
+      super();
+      this.instanceName = instanceName;
+      this.instanceUUID = instanceUUID;
+      this.zooKeepers = zooKeepers;
+      this.volumes = volumes;
+      this.version = version;
+    }
+
+    public String getInstanceName() {
+      return instanceName;
+    }
+
+    public String getInstanceUUID() {
+      return instanceUUID;
+    }
+
+    public Set<String> getZooKeepers() {
+      return zooKeepers;
+    }
+
+    public Set<String> getVolumes() {
+      return volumes;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+  }
+
+  private class MetricFetcher implements Runnable {
+
+    private final ServerContext ctx;
+    private final ServerId server;
+    private final SystemInformation summary;
+
+    private MetricFetcher(ServerContext ctx, ServerId server, 
SystemInformation summary) {
+      this.ctx = ctx;
+      this.server = server;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.METRICS,
+            HostAndPort.fromParts(server.getHost(), server.getPort()), ctx);
+        try {
+          MetricResponse response = 
metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds());
+          summary.processResponse(server, response);
+        } finally {
+          ThriftUtil.returnClient(metricsClient, ctx);
+        }
+      } catch (Exception e) {
+        LOG.warn("Error trying to get metrics from server: {}. Error message: 
{}", server,
+            e.getMessage());
+        summary.processError(server);
+      }
+    }
+
+  }
+
+  private class TableInformationFetcher implements Runnable {
+    private final ServerContext ctx;
+    private final String table;
+    private final SystemInformation summary;
+
+    private TableInformationFetcher(ServerContext ctx, String tableName,
+        SystemInformation summary) {
+      this.ctx = ctx;
+      this.table = tableName;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try (Stream<TabletInformation> tablets =
+          this.ctx.tableOperations().getTabletInformation(table, new Range())) 
{
+        tablets.forEach(t -> summary.processTabletInformation(table, t));
+      } catch (TableNotFoundException e) {
+        LOG.warn(
+            "TableNotFoundException thrown while trying to gather information 
for table: " + table,
+            e);
+      } catch (Exception e) {
+        LOG.warn("Interrupted while trying to gather information for table: 
{}", table);
+      }
+    }
+  }
+
+  private class CompactionListFetcher implements Runnable {
+
+    private final String coordinatorMissingMsg =
+        "Error getting the compaction coordinator client. Check that the 
Manager is running.";
+
+    private final SystemInformation summary;
+
+    public CompactionListFetcher(SystemInformation summary) {
+      this.summary = summary;
+    }
+
+    // Copied from Monitor
+    private Map<String,TExternalCompactionList> getLongRunningCompactions() {
+      Set<ServerId> managers = 
ctx.instanceOperations().getServers(ServerId.Type.MANAGER);
+      if (managers.isEmpty()) {
+        throw new IllegalStateException(coordinatorMissingMsg);
+      }
+      ServerId manager = managers.iterator().next();
+      HostAndPort hp = HostAndPort.fromParts(manager.getHost(), 
manager.getPort());
+      try {
+        CompactionCoordinatorService.Client client =
+            ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, hp, ctx);
+        try {
+          return client.getLongRunningCompactions(TraceUtil.traceInfo(), 
ctx.rpcCreds());
+        } catch (Exception e) {
+          throw new IllegalStateException("Unable to get running compactions 
from " + hp, e);
+        } finally {
+          if (client != null) {
+            ThriftUtil.returnClient(client, ctx);
+          }
+        }
+      } catch (TTransportException e) {
+        LOG.error("Unable to get Compaction coordinator at {}", hp);
+        throw new IllegalStateException(coordinatorMissingMsg, e);
+      }
+    }
+
+    @Override
+    public void run() {
+      try {
+        summary.processExternalCompactionList(getLongRunningCompactions());
+      } catch (Exception e) {
+        LOG.warn("Error gathering running compaction information. Error 
message: {}",
+            e.getMessage());
+      }
+    }
+
+  }
+
+  private final String poolName = "MonitorMetricsThreadPool";
+  private final ThreadPoolExecutor pool = ThreadPools.getServerThreadPools()
+      .getPoolBuilder(poolName).numCoreThreads(10).withTimeOut(30, 
SECONDS).build();
+
+  private final ServerContext ctx;
+  private final Supplier<Long> connectionCount;
+  private final AtomicBoolean newConnectionEvent = new AtomicBoolean(false);
+  private final Cache<ServerId,MetricResponse> allMetrics;
+  private final AtomicReference<SystemInformation> summaryRef = new 
AtomicReference<>();
+
+  public InformationFetcher(ServerContext ctx, Supplier<Long> connectionCount) 
{
+    this.ctx = ctx;
+    this.connectionCount = connectionCount;
+    this.allMetrics = 
Caffeine.newBuilder().executor(pool).scheduler(Scheduler.systemScheduler())
+        
.expireAfterWrite(Duration.ofMinutes(10)).evictionListener(this::onRemoval).build();
+  }
+
+  public void newConnectionEvent() {
+    this.newConnectionEvent.compareAndSet(false, true);
+  }
+
+  // Protect against NPE and wait for initial data gathering
+  public SystemInformation getSummary() {
+    while (summaryRef.get() == null) {
+      Thread.onSpinWait();
+    }
+    return summaryRef.get();
+  }
+
+  public Cache<ServerId,MetricResponse> getAllMetrics() {
+    return allMetrics;
+  }
+
+  @Override
+  public void onRemoval(@Nullable ServerId server, @Nullable MetricResponse 
response,
+      RemovalCause cause) {
+    if (server == null) {
+      return;
+    }
+    LOG.info("{} has been evicted", server);
+    getSummary().processError(server);
+  }
+
+  @Override
+  public void run() {
+
+    long refreshTime = 0;
+
+    while (true) {
+
+      // Don't fetch new data if there are no connections.
+      // On an initial connection, no data may be displayed.
+      // If a connection has not been made in a while, stale data may be 
displayed.
+      // Only refresh every 5s (old monitor logic).
+      while (!newConnectionEvent.get() && connectionCount.get() == 0
+          && NanoTime.millisElapsed(refreshTime, NanoTime.now()) > 5000) {
+        Thread.onSpinWait();
+      }
+      // reset the connection event flag
+      newConnectionEvent.compareAndExchange(true, false);
+
+      LOG.info("Fetching metrics from servers");
+
+      final List<Future<?>> futures = new ArrayList<>();
+      final SystemInformation summary = new SystemInformation(allMetrics, 
this.ctx);
+
+      for (ServerId.Type type : ServerId.Type.values()) {
+        if (type == Type.MONITOR) {
+          continue;
+        }
+        for (ServerId server : this.ctx.instanceOperations().getServers(type)) 
{
+          futures.add(this.pool.submit(new MetricFetcher(this.ctx, server, 
summary)));
+        }
+      }
+      ThreadPools.resizePool(pool, () -> Math.max(20, (futures.size() / 20)), 
poolName);
+
+      // Fetch external compaction information from the Manager
+      futures.add(this.pool.submit(new CompactionListFetcher(summary)));
+
+      // Fetch Tablet / Tablet information from the metadata table
+      for (String tName : this.ctx.tableOperations().list()) {
+        futures.add(this.pool.submit(new TableInformationFetcher(this.ctx, 
tName, summary)));
+      }
+
+      long monitorFetchTimeout =
+          
ctx.getConfiguration().getTimeInMillis(Property.MONITOR_FETCH_TIMEOUT);
+      long allFuturesAdded = NanoTime.now();
+      boolean tookToLong = false;
+      while (!futures.isEmpty()) {
+
+        if (NanoTime.millisElapsed(allFuturesAdded, NanoTime.now()) > 
monitorFetchTimeout) {
+          tookToLong = true;
+        }
+
+        Iterator<Future<?>> iter = futures.iterator();
+        while (iter.hasNext()) {
+          Future<?> future = iter.next();
+          if (tookToLong && !future.isCancelled()) {
+            future.cancel(true);
+          } else if (future.isDone()) {
+            iter.remove();
+            try {
+              future.get();
+            } catch (CancellationException | InterruptedException | 
ExecutionException e) {
+              LOG.error("Error getting status from future: {}", 
e.getMessage());

Review Comment:
   ```suggestion
                 LOG.error("Error getting status from future", e);
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/metrics/MetricServiceHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.metrics;
+
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService;
+import org.apache.accumulo.core.metrics.thrift.MetricSource;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.net.HostAndPort;
+import com.google.flatbuffers.FlatBufferBuilder;
+
+import io.micrometer.core.instrument.Metrics;
+
+public class MetricServiceHandler implements MetricService.Iface {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(MetricServiceHandler.class);
+
+  private final MetricSource type;
+  private final String resourceGroup;
+  private final ServerContext ctx;
+
+  private String host;
+
+  public MetricServiceHandler(MetricSource source, String resourceGroup, 
ServerContext ctx) {
+    this.type = source;
+    this.resourceGroup = resourceGroup;
+    this.ctx = ctx;
+  }
+
+  public void setHost(HostAndPort host) {
+    this.host = host.toString();
+  }
+
+  @Override
+  public MetricResponse getMetrics(TInfo tinfo, TCredentials credentials) 
throws TException {
+
+    if (!(ctx.getSecurityOperation().isSystemUser(credentials)
+        && ctx.getSecurityOperation().authenticateUser(credentials, 
credentials))) {

Review Comment:
   It'd be good to put the security policy logic in a method like `public 
boolean canGetMetrics(TCredentials c);` in SecurityOperation, so this code is 
more readable.
   
   ```suggestion
       if (!ctx.getSecurityOperation().canGetMetrics(credentials)) {
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java:
##########
@@ -475,6 +485,14 @@ private ServletHolder getRestServlet() {
     return new ServletHolder(new ServletContainer(rc));
   }
 
+  private ServletHolder getRestV2Servlet() {
+    final ResourceConfig rc = new 
ResourceConfig().packages("org.apache.accumulo.monitor.next")

Review Comment:
   It looks like this servlet is being configured identically to the previous 
`/rest` servlet. If the classes are moved into the `.rest` package, then 
there's no need to set up a second servlet at all.



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java:
##########
@@ -811,4 +829,24 @@ public int getLivePort() {
   public ServiceLock getLock() {
     return monitorLock;
   }
+
+  public InformationFetcher getInformationFetcher() {
+    return fetcher;
+  }
+
+  @Override
+  public void onOpened(Connection connection) {
+    log.info("New connection event");

Review Comment:
   Probably should not log new connection events at info level. I'm guessing 
this was added during testing of this connection listener, but is no longer 
needed.
   
   ```suggestion
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/Endpoints.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.monitor.next;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import jakarta.inject.Inject;
+import jakarta.servlet.http.HttpServletRequest;
+import jakarta.ws.rs.GET;
+import jakarta.ws.rs.NotFoundException;
+import jakarta.ws.rs.Path;
+import jakarta.ws.rs.PathParam;
+import jakarta.ws.rs.Produces;
+import jakarta.ws.rs.core.Context;
+import jakarta.ws.rs.core.MediaType;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
+import org.apache.accumulo.core.metrics.flatbuffers.FMetric;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.monitor.Monitor;
+import org.apache.accumulo.monitor.next.InformationFetcher.InstanceSummary;
+import org.apache.accumulo.monitor.next.SystemInformation.ProcessSummary;
+import org.apache.accumulo.monitor.next.SystemInformation.TableSummary;
+
+import io.micrometer.core.instrument.Meter.Id;
+import io.micrometer.core.instrument.cumulative.CumulativeDistributionSummary;
+
+@Path("/")

Review Comment:
   If this is moved into the existing rest servlet, the package could be 
`org.apache.accumulo.monitor.rest.v2`, and this could be:
   
   ```suggestion
   @Path("/v2")
   ```
   
   That would make the paths to these endpoints be `/rest/v2`



##########
server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java:
##########
@@ -1048,14 +1172,14 @@ public TExternalCompactionList 
getRunningCompactions(TInfo tinfo, TCredentials c
    * @throws ThriftSecurityException permission error
    */
   @Override
-  public TExternalCompactionList getCompletedCompactions(TInfo tinfo, 
TCredentials credentials)

Review Comment:
   If we don't need this old type anymore, it could be removed.



##########
core/src/main/scripts/generate-flatbuffers.sh:
##########
@@ -0,0 +1,52 @@
+#! /usr/bin/env bash
+#
+# 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.
+#
+
+# This code will generate the FlatBuffers code that is used
+# to serialize Micrometer Meter objects into a byte[] inside
+# the Thrift response object returned by the MetricServiceHandler
+# that is called by the Monitor.
+
+[[ -z $REQUIRED_FB_VERSION ]] && REQUIRED_FB_VERSION='24.3.25'
+
+# Test to see if we have thrift installed
+if ! flatc --version 2>/dev/null | grep -qF "${REQUIRED_FB_VERSION}"; then
+  echo "****************************************************"
+  echo "*** flatc ${REQUIRED_FB_VERSION} is not available, check PATH"
+  echo "*** and ensure that 'flatc' is resolvable or install correct"
+  echo "*** version. Generated code will not be updated"
+  fail "****************************************************"
+
+  # git clone g...@github.com:google/flatbuffers.git
+  # cd flatbuffers
+  # git checkout v24.3.25
+  # cmake -G "Unix Makefiles"
+  # make -j
+  # make test
+  # sudo make install
+  # See FlatBuffers README and/or https://flatbuffers.dev for more information
+fi
+
+# Run this scrip from the accumulo/core directory, for exampe:

Review Comment:
   ```suggestion
   # Run this script from the accumulo/core directory, for example:
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.monitor.next;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService.Client;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.transport.TTransportException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.eclipse.jetty.util.NanoTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.net.HostAndPort;
+
+public class InformationFetcher implements 
RemovalListener<ServerId,MetricResponse>, Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InformationFetcher.class);
+
+  public static class InstanceSummary {
+    private final String instanceName;
+    private final String instanceUUID;
+    private final Set<String> zooKeepers;
+    private final Set<String> volumes;
+    private final String version;
+
+    public InstanceSummary(String instanceName, String instanceUUID, 
Set<String> zooKeepers,
+        Set<String> volumes, String version) {
+      super();
+      this.instanceName = instanceName;
+      this.instanceUUID = instanceUUID;
+      this.zooKeepers = zooKeepers;
+      this.volumes = volumes;
+      this.version = version;
+    }
+
+    public String getInstanceName() {
+      return instanceName;
+    }
+
+    public String getInstanceUUID() {
+      return instanceUUID;
+    }
+
+    public Set<String> getZooKeepers() {
+      return zooKeepers;
+    }
+
+    public Set<String> getVolumes() {
+      return volumes;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+  }
+
+  private class MetricFetcher implements Runnable {
+
+    private final ServerContext ctx;
+    private final ServerId server;
+    private final SystemInformation summary;
+
+    private MetricFetcher(ServerContext ctx, ServerId server, 
SystemInformation summary) {
+      this.ctx = ctx;
+      this.server = server;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.METRICS,
+            HostAndPort.fromParts(server.getHost(), server.getPort()), ctx);
+        try {
+          MetricResponse response = 
metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds());
+          summary.processResponse(server, response);
+        } finally {
+          ThriftUtil.returnClient(metricsClient, ctx);
+        }
+      } catch (Exception e) {
+        LOG.warn("Error trying to get metrics from server: {}. Error message: 
{}", server,
+            e.getMessage());

Review Comment:
   Might be useful to see the actual stack trace if there's a warning/error 
here:
   
   ```suggestion
           LOG.warn("Error trying to get metrics from server: {}", server, e);
   ```



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/Endpoints.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.monitor.next;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import java.lang.reflect.Method;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import jakarta.inject.Inject;
+import jakarta.servlet.http.HttpServletRequest;
+import jakarta.ws.rs.GET;
+import jakarta.ws.rs.NotFoundException;
+import jakarta.ws.rs.Path;
+import jakarta.ws.rs.PathParam;
+import jakarta.ws.rs.Produces;
+import jakarta.ws.rs.core.Context;
+import jakarta.ws.rs.core.MediaType;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
+import org.apache.accumulo.core.metrics.flatbuffers.FMetric;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.monitor.Monitor;
+import org.apache.accumulo.monitor.next.InformationFetcher.InstanceSummary;
+import org.apache.accumulo.monitor.next.SystemInformation.ProcessSummary;
+import org.apache.accumulo.monitor.next.SystemInformation.TableSummary;
+
+import io.micrometer.core.instrument.Meter.Id;
+import io.micrometer.core.instrument.cumulative.CumulativeDistributionSummary;
+
+@Path("/")
+public class Endpoints {
+
+  @Target(ElementType.METHOD)
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Description {
+    String value();
+  }
+
+  @Inject
+  private Monitor monitor;
+
+  private void validateResourceGroup(String resourceGroup) {
+    if 
(monitor.getInformationFetcher().getSummary().getResourceGroups().contains(resourceGroup))
 {
+      return;
+    }
+    throw new NotFoundException("Resource Group " + resourceGroup + " not 
found");
+  }
+
+  @GET
+  @Path("endpoints")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a list of the available endpoints and a description 
for each")
+  public Map<String,String> getEndpoints(@Context HttpServletRequest request) {
+
+    /*
+     * Attempted to use OpenAPI annotation for use with Swagger-UI, but ran 
into potential
+     * dependency convergence issues as we were using newer version of some of 
the same
+     * dependencies.
+     */
+    final String basePath = request.getRequestURL().toString();
+    final Map<String,String> documentation = new TreeMap<>();
+
+    for (Method m : Endpoints.class.getMethods()) {
+      if (m.isAnnotationPresent(Path.class)) {
+        Path pathAnnotation = m.getAnnotation(Path.class);
+        String path = basePath + "/" + pathAnnotation.value();
+        String description = "";
+        if (m.isAnnotationPresent(Description.class)) {
+          Description desc = m.getAnnotation(Description.class);
+          description = desc.value();
+        }
+        documentation.put(path, description);
+      }
+    }
+
+    return documentation;
+  }
+
+  @GET
+  @Path("groups")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a list of the resource groups that are in use")
+  public Set<String> getResourceGroups() {
+    return monitor.getInformationFetcher().getSummary().getResourceGroups();
+  }
+
+  @GET
+  @Path("problems")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a list of the servers that are potentially down")
+  public Collection<ServerId> getProblemHosts() {
+    return monitor.getInformationFetcher().getSummary().getProblemHosts();
+  }
+
+  @GET
+  @Path("metrics")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric responses for all servers")
+  public Collection<MetricResponse> getAll() {
+    return monitor.getInformationFetcher().getAllMetrics().asMap().values();
+  }
+
+  @GET
+  @Path("manager")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric response for the Manager")
+  public MetricResponse getManager() {
+    final ServerId s = 
monitor.getInformationFetcher().getSummary().getManager();
+    if (s == null) {
+      throw new NotFoundException("Manager not found");
+    }
+    return monitor.getInformationFetcher().getAllMetrics().asMap().get(s);
+  }
+
+  @GET
+  @Path("gc")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric response for the Garbage Collector")
+  public MetricResponse getGarbageCollector() {
+    final ServerId s = 
monitor.getInformationFetcher().getSummary().getGarbageCollector();
+    if (s == null) {
+      throw new NotFoundException("Garbage Collector not found");
+    }
+    return monitor.getInformationFetcher().getAllMetrics().asMap().get(s);
+  }
+
+  @GET
+  @Path("instance")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the instance name, instance id, version, zookeepers, 
and volumes")
+  public InstanceSummary getInstanceSummary() {
+    return new InstanceSummary(monitor.getContext().getInstanceName(),
+        monitor.getContext().instanceOperations().getInstanceId().canonical(),
+        Set.of(monitor.getContext().getZooKeepers().split(",")),
+        
monitor.getContext().getVolumeManager().getVolumes().stream().map(Object::toString)
+            .collect(Collectors.toSet()),
+        Constants.VERSION);
+  }
+
+  @GET
+  @Path("compactors/detail/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric responses for the Compactors in the 
supplied resource group")
+  public Collection<MetricResponse> getCompactors(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Set<ServerId> servers = monitor.getInformationFetcher().getSummary()
+        .getCompactorResourceGroupServers(resourceGroup);
+    if (servers == null) {
+      return List.of();
+    }
+    return 
monitor.getInformationFetcher().getAllMetrics().getAllPresent(servers).values();
+  }
+
+  @GET
+  @Path("compactors/summary/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for the 
Compactors in the supplied resource group")
+  public Map<Id,CumulativeDistributionSummary>
+      getCompactorResourceGroupMetricSummary(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Map<Id,CumulativeDistributionSummary> metrics = 
monitor.getInformationFetcher()
+        .getSummary().getCompactorResourceGroupMetricSummary(resourceGroup);
+    if (metrics == null) {
+      return Map.of();
+    }
+    return metrics;
+  }
+
+  @GET
+  @Path("compactors/summary")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for all 
Compactors")
+  public Map<Id,CumulativeDistributionSummary> getCompactorAllMetricSummary() {
+    return 
monitor.getInformationFetcher().getSummary().getCompactorAllMetricSummary();
+  }
+
+  @GET
+  @Path("sservers/detail/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric responses for the ScanServers in the 
supplied resource group")
+  public Collection<MetricResponse> getScanServers(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Set<ServerId> servers =
+        
monitor.getInformationFetcher().getSummary().getSServerResourceGroupServers(resourceGroup);
+    if (servers == null) {
+      return List.of();
+    }
+    return 
monitor.getInformationFetcher().getAllMetrics().getAllPresent(servers).values();
+  }
+
+  @GET
+  @Path("sservers/summary/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for the 
ScanServers in the supplied resource group")
+  public Map<Id,CumulativeDistributionSummary>
+      getScanServerResourceGroupMetricSummary(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Map<Id,CumulativeDistributionSummary> metrics = 
monitor.getInformationFetcher()
+        .getSummary().getSServerResourceGroupMetricSummary(resourceGroup);
+    if (metrics == null) {
+      return Map.of();
+    }
+    return metrics;
+  }
+
+  @GET
+  @Path("sservers/summary")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for all 
ScanServers")
+  public Map<Id,CumulativeDistributionSummary> getScanServerAllMetricSummary() 
{
+    return 
monitor.getInformationFetcher().getSummary().getSServerAllMetricSummary();
+  }
+
+  @GET
+  @Path("tservers/detail/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metric responses for the TabletServers in the 
supplied resource group")
+  public Collection<MetricResponse> getTabletServers(@PathParam("group") 
String resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Set<ServerId> servers =
+        
monitor.getInformationFetcher().getSummary().getTServerResourceGroupServers(resourceGroup);
+    if (servers == null) {
+      return List.of();
+    }
+    return 
monitor.getInformationFetcher().getAllMetrics().getAllPresent(servers).values();
+  }
+
+  @GET
+  @Path("tservers/summary/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for the 
TabletServers in the supplied resource group")
+  public Map<Id,CumulativeDistributionSummary>
+      getTabletServerResourceGroupMetricSummary(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    final Map<Id,CumulativeDistributionSummary> metrics = 
monitor.getInformationFetcher()
+        .getSummary().getTServerResourceGroupMetricSummary(resourceGroup);
+    if (metrics == null) {
+      return Map.of();
+    }
+    return metrics;
+  }
+
+  @GET
+  @Path("tservers/summary")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns an aggregate view of the metric responses for all 
TabletServers")
+  public Map<Id,CumulativeDistributionSummary> 
getTabletServerAllMetricSummary() {
+    return 
monitor.getInformationFetcher().getSummary().getTServerAllMetricSummary();
+  }
+
+  @GET
+  @Path("compactions/summary")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the metrics for all compaction queues")
+  public Map<String,List<FMetric>> getCompactionMetricSummary() {
+    return 
monitor.getInformationFetcher().getSummary().getCompactionMetricSummary();
+  }
+
+  @GET
+  @Path("compactions/detail")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a map of Compactor resource group to the 50 oldest 
running compactions")
+  public Map<String,List<TExternalCompaction>> getCompactions() {
+    Map<String,List<TExternalCompaction>> all =
+        monitor.getInformationFetcher().getSummary().getCompactions();
+    if (all == null) {
+      return Map.of();
+    }
+    return all;
+  }
+
+  @GET
+  @Path("compactions/detail/{group}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a list of the 50 oldest running compactions in the 
supplied resource group")
+  public List<TExternalCompaction> getCompactions(@PathParam("group") String 
resourceGroup) {
+    validateResourceGroup(resourceGroup);
+    List<TExternalCompaction> compactions =
+        
monitor.getInformationFetcher().getSummary().getCompactions(resourceGroup);
+    if (compactions == null) {
+      return List.of();
+    }
+    return compactions;
+  }
+
+  @GET
+  @Path("tables")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a map of table name to table details")
+  public Map<String,TableSummary> getTables() {
+    return monitor.getInformationFetcher().getSummary().getTables();
+  }
+
+  @GET
+  @Path("tables/{name}")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns table details for the supplied table name")
+  public TableSummary getTable(@PathParam("name") String tableName) {
+    TableSummary ts = 
monitor.getInformationFetcher().getSummary().getTables().get(tableName);
+    if (ts == null) {
+      throw new NotFoundException(tableName + " not found");
+    }
+    return ts;
+  }
+
+  @GET
+  @Path("tables/{name}/tablets")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns tablet details for the supplied table name")
+  public List<TabletInformation> getTablets(@PathParam("name") String 
tableName) {
+    List<TabletInformation> ti = 
monitor.getInformationFetcher().getSummary().getTablets(tableName);
+    if (ti == null) {
+      throw new NotFoundException(tableName + " not found");
+    }
+    return ti;
+  }
+
+  @GET
+  @Path("deployment")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a map of resource group to server type to process 
summary."
+      + " The process summary contains the number of configured, responding, 
and not responding servers")
+  public Map<String,Map<String,ProcessSummary>> getDeploymentOverview() {
+    return 
monitor.getInformationFetcher().getSummary().getDeploymentOverview();
+  }
+
+  @GET
+  @Path("suggestions")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns a list of suggestions")
+  public Set<String> getSuggestions() {
+    return monitor.getInformationFetcher().getSummary().getSuggestions();
+  }
+
+  @GET
+  @Path("lastUpdate")
+  @Produces(MediaType.APPLICATION_JSON)
+  @Description("Returns the timestamp of when the monitor information was last 
refreshed")
+  public long getTimestamp() {
+    return monitor.getInformationFetcher().getSummary().getTimestamp();
+  }
+
+  @GET
+  @Path("stats")
+  @Produces(MediaType.TEXT_PLAIN)
+  @Description("Returns connection statistics for the Jetty server")
+  public String getConnectionStatistics() {
+    return monitor.getConnectionStatisticsBean().dump();
+  }

Review Comment:
   Nice additions for testing the monitor



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java:
##########
@@ -359,6 +366,7 @@ public void run() {
         server = new EmbeddedWebServer(this, port);
         server.addServlet(getDefaultServlet(), "/resources/*");
         server.addServlet(getRestServlet(), "/rest/*");
+        server.addServlet(getRestV2Servlet(), "/rest-v2/*");

Review Comment:
   I don't think we need to version the rest endpoints. We can use the same 
path. If we do, though, I think the `/rest/v2/*` would be a more traditional 
way of versioning. In this case, though, maybe `/rest/metrics/`?
   
   In any case, these servlets should probably be flattened to one before 
release.



##########
server/monitor/src/main/java/org/apache/accumulo/monitor/next/InformationFetcher.java:
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.monitor.next;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TabletInformation;
+import org.apache.accumulo.core.client.admin.servers.ServerId;
+import org.apache.accumulo.core.client.admin.servers.ServerId.Type;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metrics.thrift.MetricResponse;
+import org.apache.accumulo.core.metrics.thrift.MetricService.Client;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.thrift.transport.TTransportException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.eclipse.jetty.util.NanoTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.net.HostAndPort;
+
+public class InformationFetcher implements 
RemovalListener<ServerId,MetricResponse>, Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(InformationFetcher.class);
+
+  public static class InstanceSummary {
+    private final String instanceName;
+    private final String instanceUUID;
+    private final Set<String> zooKeepers;
+    private final Set<String> volumes;
+    private final String version;
+
+    public InstanceSummary(String instanceName, String instanceUUID, 
Set<String> zooKeepers,
+        Set<String> volumes, String version) {
+      super();
+      this.instanceName = instanceName;
+      this.instanceUUID = instanceUUID;
+      this.zooKeepers = zooKeepers;
+      this.volumes = volumes;
+      this.version = version;
+    }
+
+    public String getInstanceName() {
+      return instanceName;
+    }
+
+    public String getInstanceUUID() {
+      return instanceUUID;
+    }
+
+    public Set<String> getZooKeepers() {
+      return zooKeepers;
+    }
+
+    public Set<String> getVolumes() {
+      return volumes;
+    }
+
+    public String getVersion() {
+      return version;
+    }
+  }
+
+  private class MetricFetcher implements Runnable {
+
+    private final ServerContext ctx;
+    private final ServerId server;
+    private final SystemInformation summary;
+
+    private MetricFetcher(ServerContext ctx, ServerId server, 
SystemInformation summary) {
+      this.ctx = ctx;
+      this.server = server;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Client metricsClient = ThriftUtil.getClient(ThriftClientTypes.METRICS,
+            HostAndPort.fromParts(server.getHost(), server.getPort()), ctx);
+        try {
+          MetricResponse response = 
metricsClient.getMetrics(TraceUtil.traceInfo(), ctx.rpcCreds());
+          summary.processResponse(server, response);
+        } finally {
+          ThriftUtil.returnClient(metricsClient, ctx);
+        }
+      } catch (Exception e) {
+        LOG.warn("Error trying to get metrics from server: {}. Error message: 
{}", server,
+            e.getMessage());
+        summary.processError(server);
+      }
+    }
+
+  }
+
+  private class TableInformationFetcher implements Runnable {
+    private final ServerContext ctx;
+    private final String table;
+    private final SystemInformation summary;
+
+    private TableInformationFetcher(ServerContext ctx, String tableName,
+        SystemInformation summary) {
+      this.ctx = ctx;
+      this.table = tableName;
+      this.summary = summary;
+    }
+
+    @Override
+    public void run() {
+      try (Stream<TabletInformation> tablets =
+          this.ctx.tableOperations().getTabletInformation(table, new Range())) 
{
+        tablets.forEach(t -> summary.processTabletInformation(table, t));
+      } catch (TableNotFoundException e) {
+        LOG.warn(
+            "TableNotFoundException thrown while trying to gather information 
for table: " + table,
+            e);
+      } catch (Exception e) {
+        LOG.warn("Interrupted while trying to gather information for table: 
{}", table);
+      }
+    }
+  }
+
+  private class CompactionListFetcher implements Runnable {
+
+    private final String coordinatorMissingMsg =
+        "Error getting the compaction coordinator client. Check that the 
Manager is running.";
+
+    private final SystemInformation summary;
+
+    public CompactionListFetcher(SystemInformation summary) {
+      this.summary = summary;
+    }
+
+    // Copied from Monitor
+    private Map<String,TExternalCompactionList> getLongRunningCompactions() {
+      Set<ServerId> managers = 
ctx.instanceOperations().getServers(ServerId.Type.MANAGER);
+      if (managers.isEmpty()) {
+        throw new IllegalStateException(coordinatorMissingMsg);
+      }
+      ServerId manager = managers.iterator().next();
+      HostAndPort hp = HostAndPort.fromParts(manager.getHost(), 
manager.getPort());
+      try {
+        CompactionCoordinatorService.Client client =
+            ThriftUtil.getClient(ThriftClientTypes.COORDINATOR, hp, ctx);
+        try {
+          return client.getLongRunningCompactions(TraceUtil.traceInfo(), 
ctx.rpcCreds());
+        } catch (Exception e) {
+          throw new IllegalStateException("Unable to get running compactions 
from " + hp, e);
+        } finally {
+          if (client != null) {
+            ThriftUtil.returnClient(client, ctx);
+          }
+        }
+      } catch (TTransportException e) {
+        LOG.error("Unable to get Compaction coordinator at {}", hp);

Review Comment:
   ```suggestion
           LOG.error("Unable to get Compaction coordinator at {}", hp, e);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscr...@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to