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

mlbiscoc pushed a commit to branch feature/SOLR-17458-rebased
in repository https://gitbox.apache.org/repos/asf/solr.git

commit c022d6bb42730a0dd412820d12e783be2478aa94
Author: Sanjay Dutt <[email protected]>
AuthorDate: Fri Sep 26 20:51:15 2025 +0530

    SOLR-17806: Migrate SolrIndexSearcher metrics to OTEL (#3482)
    
    * Migrate SolrIndexSearcher metrics to OTEL
    
    * renaming metrics
    
    * metric renaming, adding descriptions
    
    * removing comment and renaming metric
    
    * switch to OTEL for statsCache
    
    * recording string metric is not required
    
    * Metrics cleanup
    
    * Fix broken searcher tests
    
    * Change solr_searcher_live_docs_cache to a counter
    
    * Feedback changes
    
    * Remove sort metric
    
    * Remove caching enabled metric
    
    ---------
    
    Co-authored-by: Matthew Biscocho <[email protected]>
    Co-authored-by: Matthew Biscocho <[email protected]>
---
 .../src/java/org/apache/solr/core/SolrCore.java    |   1 +
 .../org/apache/solr/search/SolrIndexSearcher.java  | 203 ++++++++++++---------
 .../org/apache/solr/search/stats/StatsCache.java   |  97 +++++++++-
 .../apache/solr/cloud/TestBaseStatsCacheCloud.java | 134 +++++++-------
 .../solr/handler/admin/StatsReloadRaceTest.java    |  34 ++--
 .../org/apache/solr/search/TestIndexSearcher.java  |  11 --
 .../apache/solr/search/TestMainQueryCaching.java   |  53 ++----
 7 files changed, 311 insertions(+), 222 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java 
b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index d42096c6b6e..ea2b9a08e8f 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1712,6 +1712,7 @@ public class SolrCore implements SolrInfoBean, Closeable {
       }
       cache = new LocalStatsCache();
     }
+    cache.initializeMetrics(solrMetricsContext, coreAttributes, null);
     return cache;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java 
b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 235160ec28b..51dd2c83930 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -16,6 +16,10 @@
  */
 package org.apache.solr.search;
 
+import static org.apache.solr.metrics.SolrCoreMetricManager.COLLECTION_ATTR;
+import static org.apache.solr.metrics.SolrCoreMetricManager.CORE_ATTR;
+import static org.apache.solr.metrics.SolrCoreMetricManager.REPLICA_ATTR;
+import static org.apache.solr.metrics.SolrCoreMetricManager.SHARD_ATTR;
 import static org.apache.solr.search.CpuAllowedLimit.TIMING_CONTEXT;
 
 import com.codahale.metrics.Gauge;
@@ -25,7 +29,6 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
@@ -96,8 +99,10 @@ import org.apache.solr.common.util.CollectionUtil;
 import org.apache.solr.common.util.EnvUtils;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.ExecutorUtil.MDCAwareThreadPoolExecutor;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.NodeConfig;
@@ -105,8 +110,10 @@ import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.index.SlowCompositeReaderWrapper;
-import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricsContext;
+import org.apache.solr.metrics.otel.OtelUnit;
+import org.apache.solr.metrics.otel.instruments.AttributedLongGauge;
+import org.apache.solr.metrics.otel.instruments.AttributedLongTimer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
@@ -170,6 +177,13 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
   private final LongAdder liveDocsNaiveCacheHitCount = new LongAdder();
   private final LongAdder liveDocsInsertsCount = new LongAdder();
   private final LongAdder liveDocsHitCount = new LongAdder();
+  private final List<AutoCloseable> toClose = new ArrayList<>();
+
+  // Synchronous gauge for caching enabled status
+  private AttributedLongGauge cachingEnabledGauge;
+
+  // Timer for warmup time histogram
+  private AttributedLongTimer warmupTimer;
 
   // map of generic caches - not synchronized since it's read-only after the 
constructor.
   private final Map<String, SolrCache<?, ?>> cacheMap;
@@ -400,6 +414,7 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
     this.leafReader = SlowCompositeReaderWrapper.wrap(this.reader);
     this.core = core;
     this.statsCache = core.createStatsCache();
+    this.toClose.add(this.statsCache);
     this.schema = schema;
     this.name =
         "Searcher@"
@@ -606,8 +621,23 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
           core.getCoreAttributes().toBuilder().put(NAME_ATTR, 
cache.name()).build(),
           "solr_searcher_cache");
     }
-    // TODO SOLR-17458: Add Otel
-    initializeMetrics(solrMetricsContext, Attributes.empty(), STATISTICS_KEY);
+    Attributes baseAttributes;
+    if (core.getCoreContainer().isZooKeeperAware()) {
+      baseAttributes =
+          Attributes.builder()
+              .put(COLLECTION_ATTR, 
core.getCoreDescriptor().getCollectionName())
+              .put(CORE_ATTR, core.getCoreDescriptor().getName())
+              .put(SHARD_ATTR, 
core.getCoreDescriptor().getCloudDescriptor().getShardId())
+              .put(
+                  REPLICA_ATTR,
+                  Utils.parseMetricsReplicaName(
+                      core.getCoreDescriptor().getCollectionName(), 
core.getName()))
+              .build();
+    } else {
+      baseAttributes =
+          Attributes.builder().put(CORE_ATTR, 
core.getCoreDescriptor().getName()).build();
+    }
+    initializeMetrics(solrMetricsContext, baseAttributes, STATISTICS_KEY);
     registerTime = new Date();
   }
 
@@ -669,6 +699,7 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
 
     // do this at the end so it only gets done if there are no exceptions
     numCloses.incrementAndGet();
+    IOUtils.closeQuietly(toClose);
     assert ObjectReleaseTracker.release(this);
   }
 
@@ -2522,8 +2553,9 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
         log.debug("autowarming result for [{}]\n\t{}", this, cacheList[i]);
       }
     }
-    warmupTime =
+    this.warmupTime =
         TimeUnit.MILLISECONDS.convert(System.nanoTime() - warmingStartTime, 
TimeUnit.NANOSECONDS);
+    if (warmupTimer != null) warmupTimer.record(warmupTime);
   }
 
   /** return the named generic cache */
@@ -2604,95 +2636,84 @@ public class SolrIndexSearcher extends IndexSearcher 
implements Closeable, SolrI
     return solrMetricsContext;
   }
 
-  // TODO SOLR-17458: Migrate to Otel
   @Override
   public void initializeMetrics(
-      SolrMetricsContext parentContext, Attributes attributes, String scope) {
-    parentContext.gauge(() -> name, true, "searcherName", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(() -> cachingEnabled, true, "caching", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(() -> openTime, true, "openedAt", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(() -> warmupTime, true, "warmupTime", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(
-        () -> registerTime, true, "registeredAt", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(
-        fullSortCount::sum, true, "fullSortCount", 
Category.SEARCHER.toString(), scope);
-    parentContext.gauge(
-        skipSortCount::sum, true, "skipSortCount", 
Category.SEARCHER.toString(), scope);
-    final MetricsMap liveDocsCacheMetrics =
-        new MetricsMap(
-            (map) -> {
-              map.put("inserts", liveDocsInsertsCount.sum());
-              map.put("hits", liveDocsHitCount.sum());
-              map.put("naiveHits", liveDocsNaiveCacheHitCount.sum());
-            });
-    parentContext.gauge(
-        liveDocsCacheMetrics, true, "liveDocsCache", 
Category.SEARCHER.toString(), scope);
-    // reader stats
-    parentContext.gauge(
-        rgauge(parentContext.nullNumber(), () -> reader.numDocs()),
-        true,
-        "numDocs",
-        Category.SEARCHER.toString(),
-        scope);
-    parentContext.gauge(
-        rgauge(parentContext.nullNumber(), () -> reader.maxDoc()),
-        true,
-        "maxDoc",
-        Category.SEARCHER.toString(),
-        scope);
-    parentContext.gauge(
-        rgauge(parentContext.nullNumber(), () -> reader.maxDoc() - 
reader.numDocs()),
-        true,
-        "deletedDocs",
-        Category.SEARCHER.toString(),
-        scope);
-    parentContext.gauge(
-        rgauge(parentContext.nullString(), () -> reader.toString()),
-        true,
-        "reader",
-        Category.SEARCHER.toString(),
-        scope);
-    parentContext.gauge(
-        rgauge(parentContext.nullString(), () -> 
reader.directory().toString()),
-        true,
-        "readerDir",
-        Category.SEARCHER.toString(),
-        scope);
-    parentContext.gauge(
-        rgauge(parentContext.nullNumber(), () -> reader.getVersion()),
-        true,
-        "indexVersion",
-        Category.SEARCHER.toString(),
-        scope);
+      SolrMetricsContext solrMetricsContext, Attributes attributes, String 
scope) {
+    var baseAttributes =
+        attributes.toBuilder().put(CATEGORY_ATTR, 
Category.SEARCHER.toString()).build();
+
+    // warmupTime (ms) - timer for histogram tracking
+    warmupTimer =
+        new AttributedLongTimer(
+            solrMetricsContext.longHistogram(
+                "solr_searcher_warmup_time", "Searcher warmup time (ms)", 
OtelUnit.MILLISECONDS),
+            baseAttributes);
+
+    toClose.add(
+        solrMetricsContext.observableLongCounter(
+            "solr_searcher_live_docs_cache",
+            "LiveDocs cache metrics",
+            obs -> {
+              obs.record(
+                  liveDocsInsertsCount.sum(),
+                  baseAttributes.toBuilder().put(TYPE_ATTR, 
"inserts").build());
+              obs.record(
+                  liveDocsHitCount.sum(),
+                  baseAttributes.toBuilder().put(TYPE_ATTR, "hits").build());
+              obs.record(
+                  liveDocsNaiveCacheHitCount.sum(),
+                  baseAttributes.toBuilder().put(TYPE_ATTR, 
"naive_hits").build());
+            }));
+    // reader stats (numeric)
+    toClose.add(
+        solrMetricsContext.observableLongGauge(
+            "solr_searcher_index_num_docs",
+            "Number of live docs in the index",
+            obs -> {
+              try {
+                obs.record(reader.numDocs(), baseAttributes);
+              } catch (Exception ignore) {
+                // replacement for nullNumber
+              }
+            }));
+
+    toClose.add(
+        solrMetricsContext.observableLongGauge(
+            "solr_searcher_index_docs",
+            "Total number of docs in the index (including deletions)",
+            obs -> {
+              try {
+                obs.record(reader.maxDoc(), baseAttributes);
+              } catch (Exception ignore) {
+              }
+            }));
+    // indexVersion (numeric)
+    toClose.add(
+        solrMetricsContext.observableLongGauge(
+            "solr_searcher_index_version",
+            "Lucene index version",
+            obs -> {
+              try {
+                obs.record(reader.getVersion(), baseAttributes);
+              } catch (Exception ignore) {
+              }
+            }));
     // size of the currently opened commit
-    parentContext.gauge(
-        () -> {
-          try {
-            Collection<String> files = reader.getIndexCommit().getFileNames();
-            long total = 0;
-            for (String file : files) {
-              total += DirectoryFactory.sizeOf(reader.directory(), file);
-            }
-            return total;
-          } catch (Exception e) {
-            return parentContext.nullNumber();
-          }
-        },
-        true,
-        "indexCommitSize",
-        Category.SEARCHER.toString(),
-        scope);
-    // statsCache metrics
-    parentContext.gauge(
-        new MetricsMap(
-            map -> {
-              statsCache.getCacheMetrics().getSnapshot(map::putNoEx);
-              map.put("statsCacheImpl", statsCache.getClass().getSimpleName());
-            }),
-        true,
-        "statsCache",
-        Category.CACHE.toString(),
-        scope);
+    toClose.add(
+        solrMetricsContext.observableLongGauge(
+            "solr_searcher_index_commit_size_bytes",
+            "Size of the current index commit (bytes)",
+            obs -> {
+              try {
+                long total = 0L;
+                for (String file : reader.getIndexCommit().getFileNames()) {
+                  total += DirectoryFactory.sizeOf(reader.directory(), file);
+                }
+                obs.record(total, baseAttributes);
+              } catch (Exception e) {
+                // skip recording if unavailable (no nullNumber in OTel)
+              }
+            }));
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java 
b/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
index 833c40e9076..44dc7aca40e 100644
--- a/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
+++ b/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.search.stats;
 
+import io.opentelemetry.api.common.Attributes;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
@@ -31,10 +32,14 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.Weight;
+import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.ShardRequest;
 import org.apache.solr.handler.component.ShardResponse;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QueryCommand;
 import org.apache.solr.search.SolrCache;
@@ -51,7 +56,7 @@ import org.apache.solr.util.plugin.PluginInfoInitialized;
  * is aggregated), and on each core involved in a shard request (where this 
data is maintained and
  * updated from the aggregator's cache).
  */
-public abstract class StatsCache implements PluginInfoInitialized {
+public abstract class StatsCache implements PluginInfoInitialized, 
SolrInfoBean {
   // TODO: decouple use in response from use in request context for these keys
   /** Map of terms and {@link TermStats}. */
   public static final String TERM_STATS_KEY = "solr.stats.term";
@@ -65,6 +70,9 @@ public abstract class StatsCache implements 
PluginInfoInitialized {
   /** List of fields in the query. */
   public static final String FIELDS_KEY = "solr.stats.fields";
 
+  private SolrMetricsContext solrMetricsContext;
+  private AutoCloseable toClose;
+
   public static final class StatsCacheMetrics {
     public final LongAdder lookups = new LongAdder();
     public final LongAdder retrieveStats = new LongAdder();
@@ -298,4 +306,91 @@ public abstract class StatsCache implements 
PluginInfoInitialized {
       return super.collectionStatistics(field);
     }
   }
+
+  @Override
+  public String getName() {
+    return StatsCache.class.getName();
+  }
+
+  @Override
+  public String getDescription() {
+    return "A cache of global document frequency information for selected 
terms";
+  }
+
+  @Override
+  public Category getCategory() {
+    return Category.CACHE;
+  }
+
+  @Override
+  public SolrMetricsContext getSolrMetricsContext() {
+    return solrMetricsContext;
+  }
+
+  @Override
+  public void initializeMetrics(
+      SolrMetricsContext solrMetricsContext, Attributes attributes, String 
scope) {
+    this.solrMetricsContext = solrMetricsContext;
+    var cacheBaseAttribute =
+        attributes.toBuilder()
+            .put(SolrMetricProducer.CATEGORY_ATTR, Category.CACHE.toString())
+            .build();
+    this.toClose =
+        solrMetricsContext.observableLongGauge(
+            "solr_searcher_termstats_cache",
+            "Operation counts for the searcher term statistics cache, reported 
per operation type",
+            obs -> {
+              var cacheMetrics = getCacheMetrics();
+              obs.record(
+                  cacheMetrics.lookups.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "lookups")
+                      .build());
+              obs.record(
+                  cacheMetrics.missingGlobalFieldStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, 
"missing_global_field")
+                      .build());
+              obs.record(
+                  cacheMetrics.missingGlobalTermStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "missing_global_term")
+                      .build());
+              obs.record(
+                  cacheMetrics.mergeToGlobalStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "merge_to_global")
+                      .build());
+              obs.record(
+                  cacheMetrics.retrieveStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "retrieve")
+                      .build());
+              obs.record(
+                  cacheMetrics.returnLocalStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "return_local")
+                      .build());
+              obs.record(
+                  cacheMetrics.sendGlobalStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "send_global")
+                      .build());
+              obs.record(
+                  cacheMetrics.useCachedGlobalStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "use_cached_global")
+                      .build());
+              obs.record(
+                  cacheMetrics.receiveGlobalStats.sum(),
+                  cacheBaseAttribute.toBuilder()
+                      .put(SolrMetricProducer.TYPE_ATTR, "receive_global")
+                      .build());
+            });
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.closeQuietly(toClose);
+  }
 }
diff --git 
a/solr/core/src/test/org/apache/solr/cloud/TestBaseStatsCacheCloud.java 
b/solr/core/src/test/org/apache/solr/cloud/TestBaseStatsCacheCloud.java
index 8f475e64541..607715f9aa4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestBaseStatsCacheCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestBaseStatsCacheCloud.java
@@ -16,19 +16,24 @@
  */
 package org.apache.solr.cloud;
 
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.function.Function;
 import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrRequest.METHOD;
+import org.apache.solr.client.solrj.SolrRequest.SolrRequestType;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.embedded.JettySolrRunner;
 import org.apache.solr.search.similarities.CustomSimilarityFactory;
@@ -135,66 +140,54 @@ public abstract class TestBaseStatsCacheCloud extends 
SolrCloudTestCase {
     StatsCache.StatsCacheMetrics statsCacheMetrics = new 
StatsCache.StatsCacheMetrics();
     for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
       try (SolrClient client = 
getHttpSolrClient(jettySolrRunner.getBaseUrl().toString())) {
-        NamedList<Object> metricsRsp =
-            client.request(
-                new GenericSolrRequest(
-                    SolrRequest.METHOD.GET,
-                    "/admin/metrics",
-                    params("group", "solr.core", "prefix", 
"CACHE.searcher.statsCache")));
-        assertNotNull(metricsRsp);
-        NamedList<Object> metricsPerReplica = (NamedList<Object>) 
metricsRsp.get("metrics");
-        assertNotNull("no metrics perReplica", metricsPerReplica);
-        // log.info("======= Node: " + jettySolrRunner.getBaseUrl());
-        // log.info("======= Metrics:\n" + 
Utils.toJSONString(metricsPerReplica));
-        metricsPerReplica.forEach(
-            (replica, metrics) -> {
-              Map<String, Object> values =
-                  (Map<String, Object>)
-                      ((NamedList<Object>) 
metrics).get("CACHE.searcher.statsCache");
-              values.forEach(
-                  (name, value) -> {
-                    long val = value instanceof Number ? ((Number) 
value).longValue() : 0;
-                    switch (name) {
-                      case "lookups":
-                        statsCacheMetrics.lookups.add(val);
-                        break;
-                      case "returnLocalStats":
-                        statsCacheMetrics.returnLocalStats.add(val);
-                        break;
-                      case "mergeToGlobalStats":
-                        statsCacheMetrics.mergeToGlobalStats.add(val);
-                        break;
-                      case "missingGlobalFieldStats":
-                        statsCacheMetrics.missingGlobalFieldStats.add(val);
-                        break;
-                      case "missingGlobalTermStats":
-                        statsCacheMetrics.missingGlobalTermStats.add(val);
-                        break;
-                      case "receiveGlobalStats":
-                        statsCacheMetrics.receiveGlobalStats.add(val);
-                        break;
-                      case "retrieveStats":
-                        statsCacheMetrics.retrieveStats.add(val);
-                        break;
-                      case "sendGlobalStats":
-                        statsCacheMetrics.sendGlobalStats.add(val);
-                        break;
-                      case "useCachedGlobalStats":
-                        statsCacheMetrics.useCachedGlobalStats.add(val);
-                        break;
-                      case "statsCacheImpl":
-                        assertTrue(
-                            "incorrect cache impl, expected"
-                                + getImplementationName()
-                                + " but was "
-                                + value,
-                            getImplementationName().endsWith((String) value));
-                        break;
-                      default:
-                        fail("Unexpected cache metrics: key=" + name + ", 
value=" + value);
-                    }
-                  });
-            });
+        var req =
+            new GenericSolrRequest(
+                METHOD.GET,
+                "/admin/metrics",
+                SolrRequestType.ADMIN,
+                SolrParams.of("wt", "prometheus"));
+        req.setResponseParser(new InputStreamResponseParser("prometheus"));
+
+        NamedList<Object> resp = client.request(req);
+        try (InputStream in = (InputStream) resp.get("stream")) {
+          String output = new String(in.readAllBytes(), 
StandardCharsets.UTF_8);
+
+          for (String line : output.lines().toList()) {
+            if (line.startsWith("solr_searcher_termstats_cache")) {
+              String type = extractTypeAttribute(line);
+              long value = extractMetricValue(line);
+              switch (type) {
+                case "lookups":
+                  statsCacheMetrics.lookups.add(value);
+                  break;
+                case "return_local":
+                  statsCacheMetrics.returnLocalStats.add(value);
+                  break;
+                case "merge_to_global":
+                  statsCacheMetrics.mergeToGlobalStats.add(value);
+                  break;
+                case "missing_global_field":
+                  statsCacheMetrics.missingGlobalFieldStats.add(value);
+                  break;
+                case "missing_global_term":
+                  statsCacheMetrics.missingGlobalTermStats.add(value);
+                  break;
+                case "receive_global":
+                  statsCacheMetrics.receiveGlobalStats.add(value);
+                  break;
+                case "retrieve":
+                  statsCacheMetrics.retrieveStats.add(value);
+                  break;
+                case "send_global":
+                  statsCacheMetrics.sendGlobalStats.add(value);
+                  break;
+                case "use_cached_global":
+                  statsCacheMetrics.useCachedGlobalStats.add(value);
+                  break;
+              }
+            }
+          }
+        }
       }
     }
     checkStatsCacheMetrics(statsCacheMetrics);
@@ -246,4 +239,23 @@ public abstract class TestBaseStatsCacheCloud extends 
SolrCloudTestCase {
     ureq.process(client, collectionName);
     client.commit(collectionName);
   }
+
+  /**
+   * Extract type label value from Prometheus format line
+   * "solr_searcher_stats_cache{...type="lookups",...}" -> "lookups"
+   */
+  private String extractTypeAttribute(String line) {
+    int typeStart = line.indexOf("type=\"");
+    int typeEnd = line.indexOf('"', typeStart += 6);
+    return line.substring(typeStart, typeEnd);
+  }
+
+  /**
+   * Extract numeric value from Prometheus format line. 
"solr_searcher_stats_cache{...} 123.0" ->
+   * 123
+   */
+  private long extractMetricValue(String line) {
+    String valueStr = line.substring(line.lastIndexOf(' ') + 1);
+    return (long) Double.parseDouble(valueStr);
+  }
 }
diff --git 
a/solr/core/src/test/org/apache/solr/handler/admin/StatsReloadRaceTest.java 
b/solr/core/src/test/org/apache/solr/handler/admin/StatsReloadRaceTest.java
index 7cdbf10f3e2..7f88b6c8d5c 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/StatsReloadRaceTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/StatsReloadRaceTest.java
@@ -23,8 +23,9 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
-import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.SolrMetricTestUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -126,34 +127,27 @@ public class StatsReloadRaceTest extends SolrTestCaseJ4 {
     boolean found = false;
     int count = 10;
     while (!found && count-- > 0) {
-      h.getCoreContainer()
-          .getRequestHandler("/admin/metrics")
-          .handleRequest(req("prefix", "SEARCHER", "registry", registry, 
"compact", "true"), rsp);
-
-      NamedList<?> values = rsp.getValues();
       // this is not guaranteed to exist right away after core reload - 
there's a
       // small window between core load and before searcher metrics are 
registered,
       // so we may have to check a few times, and then fail softly if reload 
is not complete yet
-      NamedList<?> metrics = (NamedList<?>) values.get("metrics");
-      if (metrics == null) {
-        if (softFail) {
-          return;
-        } else {
-          fail("missing 'metrics' element in handler's output: " + 
values.asMap(5).toString());
-        }
-      }
-      metrics = (NamedList<?>) metrics.get(registry);
-      if (metrics.get(key) != null) {
+      SolrCore core = h.getCore();
+      var indexVersion =
+          SolrMetricTestUtils.getGaugeDatapoint(
+              core,
+              "solr_searcher_index_version",
+              SolrMetricTestUtils.newStandaloneLabelsBuilder(core)
+                  .label("category", "SEARCHER")
+                  .build());
+      if (indexVersion == null) {
+        Thread.sleep(500);
+      } else {
         found = true;
-        assertTrue(metrics.get(key) instanceof Long);
         break;
-      } else {
-        Thread.sleep(500);
       }
     }
     if (softFail && !found) {
       return;
     }
-    assertTrue("Key " + key + " not found in registry " + registry, found);
+    assertTrue("solr_searcher_index_version metric not found", found);
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java 
b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
index 649d7bbd914..4f7e0eb0809 100644
--- a/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
+++ b/solr/core/src/test/org/apache/solr/search/TestIndexSearcher.java
@@ -16,11 +16,8 @@
  */
 package org.apache.solr.search;
 
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Metric;
 import java.io.IOException;
 import java.lang.reflect.Array;
-import java.util.Date;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
@@ -137,18 +134,10 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
     int baseRefCount = r3.getRefCount();
     assertEquals(1, baseRefCount);
 
-    Map<String, Metric> metrics = 
h.getCore().getCoreMetricManager().getRegistry().getMetrics();
-    @SuppressWarnings({"unchecked"})
-    Gauge<Date> g = (Gauge<Date>) 
metrics.get("SEARCHER.searcher.registeredAt");
-    Date sr3SearcherRegAt = g.getValue();
     assertU(commit()); // nothing has changed
     SolrQueryRequest sr4 = req("q", "foo");
     assertSame(
         "nothing changed, searcher should be the same", sr3.getSearcher(), 
sr4.getSearcher());
-    assertEquals(
-        "nothing changed, searcher should not have been re-registered",
-        sr3SearcherRegAt.toInstant(),
-        g.getValue().toInstant());
     IndexReader r4 = sr4.getSearcher().getRawReader();
 
     // force an index change so the registered searcher won't be the one we 
are testing (and
diff --git 
a/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java 
b/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
index 60cd7b9ab90..23349bf643b 100644
--- a/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
+++ b/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
@@ -28,8 +28,6 @@ import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.metrics.MetricsMap;
-import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.SolrMetricTestUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -100,33 +98,18 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
     return (long) SolrMetricTestUtils.getCacheSearcherOpsInserts(core, 
cacheName).getValue();
   }
 
-  private static long coreToSortCount(SolrCore core, String skipOrFull) {
+  private static long coreToMatchAllDocsCount(SolrCore core, String type) {
     return (long)
-        ((SolrMetricManager.GaugeWrapper<?>)
-                core.getCoreMetricManager()
-                    .getRegistry()
-                    .getMetrics()
-                    .get("SEARCHER.searcher." + skipOrFull + "SortCount"))
-            .getGauge()
+        SolrMetricTestUtils.getCounterDatapoint(
+                core,
+                "solr_searcher_live_docs_cache",
+                SolrMetricTestUtils.newStandaloneLabelsBuilder(core)
+                    .label("category", "SEARCHER")
+                    .label("type", type)
+                    .build())
             .getValue();
   }
 
-  // NOCOMMIT: Fix this once SolrIndexSearcher is migrated for OTEL
-  private static long coreToMatchAllDocsInsertCount(SolrCore core) {
-    return (long) coreToLiveDocsCacheMetrics(core).get("inserts");
-  }
-
-  private static Map<String, Object> coreToLiveDocsCacheMetrics(SolrCore core) 
{
-    return ((MetricsMap)
-            ((SolrMetricManager.GaugeWrapper<?>)
-                    core.getCoreMetricManager()
-                        .getRegistry()
-                        .getMetrics()
-                        .get("SEARCHER.searcher.liveDocsCache"))
-                .getGauge())
-        .getValue();
-  }
-
   private static final String SCORING_QUERY = "str:d*";
 
   // wrapped as a ConstantScoreQuery
@@ -259,10 +242,8 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
     Map<?, ?> res = (Map<?, ?>) fromJSONString(response);
     Map<?, ?> body = (Map<?, ?>) (res.get("response"));
     SolrCore core = h.getCore();
-    assertEquals("Bad matchAllDocs insert count", 1, 
coreToMatchAllDocsInsertCount(core));
-    assertEquals(
-        "Bad filterCache insert count", 0, coreToInserts(core, 
SolrMetricTestUtils.FILTER_CACHE));
-    assertEquals("Bad full sort count", 0, coreToSortCount(core, "full"));
+    assertEquals("Bad matchAllDocs insert count", 1, 
coreToMatchAllDocsCount(core, "inserts"));
+    assertEquals("Bad filterCache insert count", 0, coreToInserts(core, 
"filterCache"));
     assertEquals("Should have exactly " + ALL_DOCS, ALL_DOCS, (long) 
(body.get("numFound")));
     long queryCacheInsertCount = coreToInserts(core, 
SolrMetricTestUtils.QUERY_RESULT_CACHE);
     if (queryCacheInsertCount == expectCounters[0]) {
@@ -271,7 +252,6 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
       assertEquals(++expectCounters[0], queryCacheInsertCount);
       expectCounters[1]++;
     }
-    assertEquals("Bad skip sort count", expectCounters[1], 
coreToSortCount(core, "skip"));
   }
 
   @Test
@@ -412,13 +392,11 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
     assertEquals(
         "Bad matchAllDocs insert count",
         (matchAllDocs ? 1 : 0),
-        coreToMatchAllDocsInsertCount(core));
+        coreToMatchAllDocsCount(core, "inserts"));
     assertEquals(
         "Bad filterCache insert count",
         expectFilterCacheInsertCount,
-        coreToInserts(core, SolrMetricTestUtils.FILTER_CACHE));
-    assertEquals("Bad full sort count", expectFullSortCount, 
coreToSortCount(core, "full"));
-    assertEquals("Bad skip sort count", expectSkipSortCount, 
coreToSortCount(core, "skip"));
+        coreToInserts(core, "filterCache"));
     assertEquals(
         "Should have exactly " + expectNumFound, expectNumFound, (long) 
(body.get("numFound")));
   }
@@ -472,15 +450,14 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
       assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS));
     }
     final SolrCore core = h.getCore();
-    Map<String, Object> liveDocsCacheMetrics = 
coreToLiveDocsCacheMetrics(core);
 
     // the one and only liveDocs computation
-    long inserts = (long) liveDocsCacheMetrics.get("inserts");
+    long inserts = coreToMatchAllDocsCount(core, "inserts");
 
     // hits during the initial phase
-    long hits = (long) liveDocsCacheMetrics.get("hits");
+    long hits = coreToMatchAllDocsCount(core, "hits");
 
-    long naiveHits = (long) liveDocsCacheMetrics.get("naiveHits");
+    long naiveHits = coreToMatchAllDocsCount(core, "naive_hits");
 
     assertEquals(1, inserts);
     assertEquals(nThreads - 1, hits + naiveHits);


Reply via email to