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 c488f788ad scan server metrics from #4458 based on #4459 (#4461)
c488f788ad is described below

commit c488f788ad3add4992c30556d7b5bc941d6f0a39
Author: EdColeman <d...@etcoleman.com>
AuthorDate: Thu May 2 21:52:23 2024 -0400

    scan server metrics from #4458 based on #4459 (#4461)
    
    * Add scan server metrics
    * scan metrics renaming to remove specific service in favor of tags 
(removes tserver from scan metric names)
    * enable stats on scan sever tablet metadata cache
    * add resource.group to sserver common tags instead of on individual metrics
    
    ---------
    
    Co-authored-by: Keith Turner <ktur...@apache.org>
---
 .../accumulo/core/metrics/MetricsProducer.java     | 102 ++++++++++++---------
 .../org/apache/accumulo/tserver/ScanServer.java    |  51 ++++++++++-
 .../apache/accumulo/tserver/ScanServerMetrics.java |  59 ++++++++++++
 .../accumulo/tserver/ThriftScanClientHandler.java  |   4 +-
 .../tserver/metrics/TabletServerScanMetrics.java   |  20 ++--
 .../apache/accumulo/tserver/ScanServerTest.java    |   4 +
 .../apache/accumulo/test/metrics/MetricsIT.java    |   7 +-
 7 files changed, 187 insertions(+), 60 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java 
b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
index 568b295f58..1bb2a1c10e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metrics/MetricsProducer.java
@@ -314,37 +314,6 @@ import io.micrometer.core.instrument.MeterRegistry;
  * <td></td>
  * </tr>
  * <tr>
- * <td>queries</td>
- * <td>Gauge</td>
- * <td>{@value #METRICS_TSERVER_QUERIES}</td>
- * <td>Gauge</td>
- * <td></td>
- * </tr>
- * <tr>
- * <td>scannedRate</td>
- * <td>Gauge</td>
- * <td>{@value #METRICS_TSERVER_SCANNED_ENTRIES}</td>
- * <td>Gauge</td>
- * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
- * derived</td>
- * </tr>
- * <tr>
- * <td>queryRate</td>
- * <td>Gauge</td>
- * <td>{@value #METRICS_TSERVER_SCAN_RESULTS}</td>
- * <td>Gauge</td>
- * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
- * derived</td>
- * </tr>
- * <tr>
- * <td>queryByteRate</td>
- * <td>Gauge</td>
- * <td>{@value #METRICS_TSERVER_SCAN_RESULTS_BYTES}</td>
- * <td>Gauge</td>
- * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
- * derived</td>
- * </tr>
- * <tr>
  * <td>ingestRate</td>
  * <td>Gauge</td>
  * <td>{@value #METRICS_TSERVER_INGEST_MUTATIONS}</td>
@@ -367,6 +336,28 @@ import io.micrometer.core.instrument.MeterRegistry;
  * <td>Gauge</td>
  * <td></td>
  * </tr>
+ * <!-- scan server -->
+ * <tr>
+ * <th>N/A</th>
+ * <th>N/A</th>
+ * <th>{@value #METRICS_SCAN_RESERVATION_TIMER}</th>
+ * <th>Timer</th>
+ * <th>Time to reserve a tablets files for scan</th>
+ * </tr>
+ * <tr>
+ * <th>N/A</th>
+ * <th>N/A</th>
+ * <th>{@value #METRICS_SCAN_BUSY_TIMEOUT_COUNTER}</th>
+ * <th>Counter</th>
+ * <th>Count of the scans where a busy timeout happened</th>
+ * </tr>
+ * <tr>
+ * <th>N/A</th>
+ * <th>N/A</th>
+ * <th>{@value #METRICS_SCAN_TABLET_METADATA_CACHE}</th>
+ * <th>Cache</th>
+ * <th>scan server tablet cache metrics</th>
+ * </tr>
  * <!-- scans -->
  * <tr>
  * <td>scan</td>
@@ -418,12 +409,36 @@ import io.micrometer.core.instrument.MeterRegistry;
  * <td></td>
  * </tr>
  * <tr>
- * <td>N/A</td>
- * <td>N/A</td>
- * <td>{@value #METRICS_SCAN_BUSY_TIMEOUT}</td>
- * <td>Counter</td>
+ * <td>queries</td>
+ * <td>Gauge</td>
+ * <td>{@value #METRICS_SCAN_QUERIES}</td>
+ * <td>Gauge</td>
  * <td></td>
  * </tr>
+ * <tr>
+ * <td>scannedRate</td>
+ * <td>Gauge</td>
+ * <td>{@value #METRICS_SCAN_SCANNED_ENTRIES}</td>
+ * <td>Gauge</td>
+ * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
+ * derived</td>
+ * </tr>
+ * <tr>
+ * <td>queryRate</td>
+ * <td>Gauge</td>
+ * <td>{@value #METRICS_SCAN_QUERY_SCAN_RESULTS}</td>
+ * <td>Gauge</td>
+ * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
+ * derived</td>
+ * </tr>
+ * <tr>
+ * <td>queryByteRate</td>
+ * <td>Gauge</td>
+ * <td>{@value #METRICS_SCAN_QUERY_SCAN_RESULTS_BYTES}</td>
+ * <td>Gauge</td>
+ * <td>Prior to 2.1.0 this metric was reported as a rate, it is now the count 
and the rate can be
+ * derived</td>
+ * </tr>
  * <!-- major compactions -->
  * <tr>
  * <td>{i|e}_{compactionServiceName}_{executor_name}_queued</td>
@@ -605,7 +620,7 @@ public interface MetricsProducer {
   String METRICS_REPLICATION_PEERS = METRICS_REPLICATION_PREFIX + "peers";
   String METRICS_REPLICATION_THREADS = METRICS_REPLICATION_PREFIX + "threads";
 
-  String METRICS_SCAN_PREFIX = "accumulo.tserver.scans.";
+  String METRICS_SCAN_PREFIX = "accumulo.scan.";
   String METRICS_SCAN_TIMES = METRICS_SCAN_PREFIX + "times";
   String METRICS_SCAN_OPEN_FILES = METRICS_SCAN_PREFIX + "files.open";
   String METRICS_SCAN_RESULTS = METRICS_SCAN_PREFIX + "result";
@@ -613,7 +628,14 @@ public interface MetricsProducer {
   String METRICS_SCAN_START = METRICS_SCAN_PREFIX + "start";
   String METRICS_SCAN_CONTINUE = METRICS_SCAN_PREFIX + "continue";
   String METRICS_SCAN_CLOSE = METRICS_SCAN_PREFIX + "close";
-  String METRICS_SCAN_BUSY_TIMEOUT = METRICS_SCAN_PREFIX + "busy.timeout";
+  String METRICS_SCAN_BUSY_TIMEOUT_COUNTER = METRICS_SCAN_PREFIX + 
"busy.timeout.count";
+  String METRICS_SCAN_RESERVATION_TIMER = METRICS_SCAN_PREFIX + 
"reservation.timer";
+  String METRICS_SCAN_QUERIES = METRICS_SCAN_PREFIX + "queries";
+  String METRICS_SCAN_QUERY_SCAN_RESULTS = METRICS_SCAN_PREFIX + 
"query.results";
+  String METRICS_SCAN_QUERY_SCAN_RESULTS_BYTES = METRICS_SCAN_PREFIX + 
"query.results.bytes";
+  String METRICS_SCAN_SCANNED_ENTRIES = METRICS_SCAN_PREFIX + 
"query.scanned.entries";
+
+  String METRICS_SCAN_TABLET_METADATA_CACHE = METRICS_SCAN_PREFIX + 
"tablet.metadata.cache";
 
   String METRICS_TSERVER_PREFIX = "accumulo.tserver.";
   String METRICS_TSERVER_ENTRIES = METRICS_TSERVER_PREFIX + "entries";
@@ -629,14 +651,10 @@ public interface MetricsProducer {
   String METRICS_TSERVER_TABLETS_ONLINE = METRICS_TSERVER_PREFIX + 
"tablets.online";
   String METRICS_TSERVER_TABLETS_OPENING = METRICS_TSERVER_PREFIX + 
"tablets.opening";
   String METRICS_TSERVER_TABLETS_UNOPENED = METRICS_TSERVER_PREFIX + 
"tablets.unopened";
-  String METRICS_TSERVER_QUERIES = METRICS_TSERVER_PREFIX + "queries";
   String METRICS_TSERVER_TABLETS_FILES = METRICS_TSERVER_PREFIX + 
"tablets.files";
   String METRICS_TSERVER_HOLD = METRICS_TSERVER_PREFIX + "hold";
   String METRICS_TSERVER_INGEST_MUTATIONS = METRICS_TSERVER_PREFIX + 
"ingest.mutations";
   String METRICS_TSERVER_INGEST_BYTES = METRICS_TSERVER_PREFIX + 
"ingest.bytes";
-  String METRICS_TSERVER_SCAN_RESULTS = METRICS_TSERVER_PREFIX + 
"scan.results";
-  String METRICS_TSERVER_SCAN_RESULTS_BYTES = METRICS_TSERVER_PREFIX + 
"scan.results.bytes";
-  String METRICS_TSERVER_SCANNED_ENTRIES = METRICS_TSERVER_PREFIX + 
"scan.scanned.entries";
 
   String METRICS_THRIFT_PREFIX = "accumulo.thrift.";
   String METRICS_THRIFT_EXECUTE = METRICS_THRIFT_PREFIX + "execute";
@@ -669,7 +687,7 @@ public interface MetricsProducer {
           fields.put((String) f.get(MetricsProducer.class), f.getName());
         } catch (IllegalArgumentException | IllegalAccessException e) {
           // this shouldn't happen, but let's log it anyway
-          LOG.error("Error getting metric value for field: " + f.getName());
+          LOG.error("Error getting metric value for field: {}", f.getName());
         }
       }
     }
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
index bcba26aa13..94186ed38f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
@@ -76,6 +76,7 @@ import org.apache.accumulo.core.spi.scan.ScanServerSelector;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
 import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
 import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
 import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
@@ -122,6 +123,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 
+import io.micrometer.core.instrument.Tag;
+
 public class ScanServer extends AbstractServer
     implements TabletScanClientService.Iface, TabletHostingServer {
 
@@ -199,6 +202,7 @@ public class ScanServer extends AbstractServer
   private volatile boolean serverStopRequested = false;
   private ServiceLock scanServerLock;
   protected TabletServerScanMetrics scanMetrics;
+  private ScanServerMetrics scanServerMetrics;
 
   private ZooCache managerLockCache;
 
@@ -243,7 +247,7 @@ public class ScanServer extends AbstractServer
       }
       tabletMetadataCache =
           Caffeine.newBuilder().expireAfterWrite(cacheExpiration, 
TimeUnit.MILLISECONDS)
-              
.scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+              
.scheduler(Scheduler.systemScheduler()).recordStats().build(tabletMetadataLoader);
     }
 
     delegate = newThriftScanClientHandler(new WriteTracker());
@@ -338,6 +342,7 @@ public class ScanServer extends AbstractServer
       // Don't use the normal ServerServices lock content, instead put the 
server UUID here.
       byte[] lockContent = (serverLockUUID.toString() + "," + 
groupName).getBytes(UTF_8);
 
+      // wait for 120 seconds with 5 second delay
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zLockPath.toString(), new byte[0], 
NodeExistsPolicy.SKIP);
 
@@ -371,10 +376,12 @@ public class ScanServer extends AbstractServer
 
     MetricsInfo metricsInfo = getContext().getMetricsInfo();
     metricsInfo.addServiceTags(getApplicationName(), clientAddress);
+    metricsInfo.addCommonTags(List.of(Tag.of("resource.group", groupName)));
 
     scanMetrics = new TabletServerScanMetrics();
+    scanServerMetrics = new ScanServerMetrics(tabletMetadataCache);
 
-    metricsInfo.addMetricsProducers(scanMetrics);
+    metricsInfo.addMetricsProducers(scanMetrics, scanServerMetrics);
     metricsInfo.init();
     // We need to set the compaction manager so that we don't get an NPE in 
CompactableImpl.close
 
@@ -657,6 +664,19 @@ public class ScanServer extends AbstractServer
     }
   }
 
+  @VisibleForTesting
+  ScanReservation reserveFilesInstrumented(Map<KeyExtent,List<TRange>> extents)
+      throws AccumuloException {
+    long start = System.nanoTime();
+    try {
+      return reserveFiles(extents);
+    } finally {
+      scanServerMetrics.getReservationTimer().record(System.nanoTime() - start,
+          TimeUnit.NANOSECONDS);
+    }
+
+  }
+
   protected ScanReservation reserveFiles(Map<KeyExtent,List<TRange>> extents)
       throws AccumuloException {
 
@@ -687,6 +707,16 @@ public class ScanServer extends AbstractServer
     return new ScanReservation(tabletsMetadata, myReservationId, failures);
   }
 
+  private ScanReservation reserveFilesInstrumented(long scanId) throws 
NoSuchScanIDException {
+    long start = System.nanoTime();
+    try {
+      return reserveFiles(scanId);
+    } finally {
+      scanServerMetrics.getReservationTimer().record(System.nanoTime() - start,
+          TimeUnit.NANOSECONDS);
+    }
+  }
+
   protected ScanReservation reserveFiles(long scanId) throws 
NoSuchScanIDException {
     var session = (ScanSession) sessionManager.getSession(scanId);
     if (session == null) {
@@ -875,7 +905,7 @@ public class ScanServer extends AbstractServer
 
     KeyExtent extent = getKeyExtent(textent);
     try (ScanReservation reservation =
-        reserveFiles(Map.of(extent, Collections.singletonList(range)))) {
+        reserveFilesInstrumented(Map.of(extent, 
Collections.singletonList(range)))) {
 
       if (reservation.getFailures().containsKey(textent)) {
         throw new NotServingTabletException(extent.toThrift());
@@ -889,7 +919,9 @@ public class ScanServer extends AbstractServer
           busyTimeout);
 
       return is;
-
+    } catch (ScanServerBusyException be) {
+      scanServerMetrics.incrementBusy();
+      throw be;
     } catch (AccumuloException | IOException e) {
       LOG.error("Error starting scan", e);
       throw new RuntimeException(e);
@@ -905,6 +937,9 @@ public class ScanServer extends AbstractServer
     try (ScanReservation reservation = reserveFiles(scanID)) {
       Preconditions.checkState(reservation.getFailures().isEmpty());
       return delegate.continueScan(tinfo, scanID, busyTimeout);
+    } catch (ScanServerBusyException be) {
+      scanServerMetrics.incrementBusy();
+      throw be;
     }
   }
 
@@ -933,7 +968,7 @@ public class ScanServer extends AbstractServer
       batch.put(extent, entry.getValue());
     }
 
-    try (ScanReservation reservation = reserveFiles(batch)) {
+    try (ScanReservation reservation = reserveFilesInstrumented(batch)) {
 
       HashMap<KeyExtent,TabletBase> tablets = new HashMap<>();
       reservation.getTabletMetadataExtents().forEach(extent -> {
@@ -950,6 +985,9 @@ public class ScanServer extends AbstractServer
 
       LOG.trace("started scan: {}", ims.getScanID());
       return ims;
+    } catch (ScanServerBusyException be) {
+      scanServerMetrics.incrementBusy();
+      throw be;
     } catch (TException e) {
       LOG.error("Error starting scan", e);
       throw e;
@@ -967,6 +1005,9 @@ public class ScanServer extends AbstractServer
     try (ScanReservation reservation = reserveFiles(scanID)) {
       Preconditions.checkState(reservation.getFailures().isEmpty());
       return delegate.continueMultiScan(tinfo, scanID, busyTimeout);
+    } catch (ScanServerBusyException be) {
+      scanServerMetrics.incrementBusy();
+      throw be;
     }
   }
 
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServerMetrics.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServerMetrics.java
new file mode 100644
index 0000000000..1a516b597b
--- /dev/null
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServerMetrics.java
@@ -0,0 +1,59 @@
+/*
+ * 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.tserver;
+
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsProducer;
+
+import com.github.benmanes.caffeine.cache.LoadingCache;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.MeterRegistry;
+import io.micrometer.core.instrument.Timer;
+import io.micrometer.core.instrument.binder.cache.CaffeineCacheMetrics;
+
+public class ScanServerMetrics implements MetricsProducer {
+
+  private Timer reservationTimer;
+  private Counter busyTimeoutCount;
+
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+
+  public ScanServerMetrics(final LoadingCache<KeyExtent,TabletMetadata> 
tabletMetadataCache) {
+    this.tabletMetadataCache = tabletMetadataCache;
+  }
+
+  @Override
+  public void registerMetrics(MeterRegistry registry) {
+    reservationTimer = 
Timer.builder(MetricsProducer.METRICS_SCAN_RESERVATION_TIMER)
+        .description("Time to reserve a tablets files for 
scan").register(registry);
+    busyTimeoutCount = Counter.builder(METRICS_SCAN_BUSY_TIMEOUT_COUNTER)
+        .description("The number of scans where a busy timeout 
happened").register(registry);
+    CaffeineCacheMetrics.monitor(registry, tabletMetadataCache, 
METRICS_SCAN_TABLET_METADATA_CACHE);
+  }
+
+  public Timer getReservationTimer() {
+    return reservationTimer;
+  }
+
+  public void incrementBusy() {
+    busyTimeoutCount.increment();
+  }
+}
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
index d1ff17347a..8a99b2315e 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
@@ -284,7 +284,7 @@ public class ThriftScanClientHandler implements 
TabletScanClientService.Iface {
       server.getSessionManager().removeSession(scanID);
       TabletBase tablet = 
scanSession.getTabletResolver().getTablet(scanSession.extent);
       if (busyTimeout > 0) {
-        server.getScanMetrics().incrementScanBusyTimeout(1.0D);
+        server.getScanMetrics().incrementBusy(1.0D);
         throw new ScanServerBusyException();
       } else if (tablet == null || tablet.isClosed()) {
         throw new NotServingTabletException(scanSession.extent.toThrift());
@@ -495,7 +495,7 @@ public class ThriftScanClientHandler implements 
TabletScanClientService.Iface {
     } catch (CancellationException ce) {
       server.getSessionManager().removeSession(scanID);
       if (busyTimeout > 0) {
-        server.getScanMetrics().incrementScanBusyTimeout(1.0D);
+        server.getScanMetrics().incrementBusy(1.0D);
         throw new ScanServerBusyException();
       } else {
         log.warn("Failed to get multiscan result", ce);
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
index 492a6a8c80..8e066dd7f7 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
@@ -39,7 +39,7 @@ public class TabletServerScanMetrics implements 
MetricsProducer {
   private Counter startScanCalls;
   private Counter continueScanCalls;
   private Counter closeScanCalls;
-  private Counter busyTimeoutReturned;
+  private Counter busyTimeoutCount;
 
   private final LongAdder lookupCount = new LongAdder();
   private final LongAdder queryResultCount = new LongAdder();
@@ -114,8 +114,8 @@ public class TabletServerScanMetrics implements 
MetricsProducer {
     closeScanCalls.increment(value);
   }
 
-  public void incrementScanBusyTimeout(double value) {
-    busyTimeoutReturned.increment(value);
+  public void incrementBusy(double value) {
+    busyTimeoutCount.increment(value);
   }
 
   @Override
@@ -133,17 +133,19 @@ public class TabletServerScanMetrics implements 
MetricsProducer {
         .description("calls to continue a scan / 
multiscan").register(registry);
     closeScanCalls = Counter.builder(METRICS_SCAN_CLOSE)
         .description("calls to close a scan / multiscan").register(registry);
-    busyTimeoutReturned = Counter.builder(METRICS_SCAN_BUSY_TIMEOUT)
-        .description("times that a scan has timed out in the 
queue").register(registry);
-    Gauge.builder(METRICS_TSERVER_QUERIES, this, 
TabletServerScanMetrics::getLookupCount)
+    busyTimeoutCount = Counter.builder(METRICS_SCAN_BUSY_TIMEOUT_COUNTER)
+        .description("The number of scans where a busy timeout 
happened").register(registry);
+    Gauge.builder(METRICS_SCAN_QUERIES, this, 
TabletServerScanMetrics::getLookupCount)
         .description("Number of queries").register(registry);
-    Gauge.builder(METRICS_TSERVER_SCAN_RESULTS, this, 
TabletServerScanMetrics::getQueryResultCount)
+    Gauge
+        .builder(METRICS_SCAN_QUERY_SCAN_RESULTS, this,
+            TabletServerScanMetrics::getQueryResultCount)
         .description("Query rate (entries/sec)").register(registry);
     Gauge
-        .builder(METRICS_TSERVER_SCAN_RESULTS_BYTES, this,
+        .builder(METRICS_SCAN_QUERY_SCAN_RESULTS_BYTES, this,
             TabletServerScanMetrics::getQueryByteCount)
         .description("Query rate (bytes/sec)").register(registry);
-    Gauge.builder(METRICS_TSERVER_SCANNED_ENTRIES, this, 
TabletServerScanMetrics::getScannedCount)
+    Gauge.builder(METRICS_SCAN_SCANNED_ENTRIES, this, 
TabletServerScanMetrics::getScannedCount)
         .description("Scanned rate").register(registry);
   }
 
diff --git 
a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java 
b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
index fdb79e1b00..b7f64e0524 100644
--- 
a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
+++ 
b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
@@ -101,6 +101,10 @@ public class ScanServerTest {
       return reservation;
     }
 
+    @Override
+    ScanReservation reserveFilesInstrumented(Map<KeyExtent,List<TRange>> 
extents) {
+      return reservation;
+    }
   }
 
   private ThriftScanClientHandler handler;
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java 
b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
index 4c8c3bea26..55622d0793 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
@@ -100,8 +100,11 @@ public class MetricsIT extends ConfigurableMacBase 
implements MetricsProducer {
     cluster.stop();
 
     Set<String> unexpectedMetrics = Set.of(METRICS_SCAN_YIELDS, 
METRICS_UPDATE_ERRORS,
-        METRICS_REPLICATION_QUEUE, METRICS_COMPACTOR_MAJC_STUCK, 
METRICS_SCAN_BUSY_TIMEOUT);
-    Set<String> flakyMetrics = Set.of(METRICS_GC_WAL_ERRORS, 
METRICS_FATE_TYPE_IN_PROGRESS);
+        METRICS_REPLICATION_QUEUE, METRICS_COMPACTOR_MAJC_STUCK, 
METRICS_SCAN_BUSY_TIMEOUT_COUNTER);
+    // add sserver as flaky until scan server included in mini tests.
+    Set<String> flakyMetrics = Set.of(METRICS_GC_WAL_ERRORS, 
METRICS_FATE_TYPE_IN_PROGRESS,
+        METRICS_SCAN_BUSY_TIMEOUT_COUNTER, METRICS_SCAN_RESERVATION_TIMER,
+        METRICS_SCAN_TABLET_METADATA_CACHE);
 
     Map<String,String> expectedMetricNames = this.getMetricFields();
     flakyMetrics.forEach(expectedMetricNames::remove); // might not see these

Reply via email to