This is an automated email from the ASF dual-hosted git repository.
mlbiscoc pushed a commit to branch feature/SOLR-17458
in repository https://gitbox.apache.org/repos/asf/solr.git
The following commit(s) were added to refs/heads/feature/SOLR-17458 by this
push:
new 37ea562b8dc SOLR-17806: Migrate SolrIndexSearcher metrics to OTEL
(#3482)
37ea562b8dc is described below
commit 37ea562b8dcb27ad6ccc4ae084b8a34256271495
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 09e2747d7a6..976a9991b4b 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 be9733682b7..514c1b3d279 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);
}
@@ -2521,8 +2552,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 */
@@ -2603,95 +2635,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 25975ce80e8..bead43ca370 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 6ce6e1f5806..dff779ddcc3 100644
--- a/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
+++ b/solr/core/src/test/org/apache/solr/search/TestMainQueryCaching.java
@@ -27,8 +27,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;
@@ -93,33 +91,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
@@ -252,10 +235,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]) {
@@ -264,7 +245,6 @@ public class TestMainQueryCaching extends SolrTestCaseJ4 {
assertEquals(++expectCounters[0], queryCacheInsertCount);
expectCounters[1]++;
}
- assertEquals("Bad skip sort count", expectCounters[1],
coreToSortCount(core, "skip"));
}
@Test
@@ -405,13 +385,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")));
}
@@ -465,15 +443,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);