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);
