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