From a99dd0627622d60d8965c1f6d7d161f423e3a47b Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Thu, 15 Jan 2026 11:47:05 +0100 Subject: [PATCH 01/15] WIP. --- gradle/libs.versions.toml | 2 + solr/cross-dc-manager/build.gradle | 6 +- solr/cross-dc-manager/gradle.lockfile | 24 +-- .../crossdc/manager/consumer/Consumer.java | 13 +- .../manager/consumer/ConsumerMetrics.java | 34 ++++ .../consumer/KafkaCrossDcConsumer.java | 35 ++-- .../manager/consumer/PrometheusMetrics.java | 151 ++++++++++++++++++ .../crossdc/manager/consumer/ThreadDump.java | 128 +++++++++++++++ .../manager/consumer/ThreadDumpServlet.java | 71 ++++++++ .../solr/crossdc/manager/consumer/Util.java | 48 ++---- .../SolrMessageProcessor.java | 30 ++-- .../manager/DeleteByQueryToIdTest.java | 2 +- .../manager/SimpleSolrIntegrationTest.java | 4 +- .../manager/consumer/NoopConsumerMetrics.java | 60 +++++++ .../SolrMessageProcessorTest.java | 3 +- .../TestMessageProcessor.java | 5 +- 16 files changed, 526 insertions(+), 90 deletions(-) create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java create mode 100644 solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index c9616523957d..6d1255659ecb 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -498,6 +498,8 @@ ow2-asm-commons = { module = "org.ow2.asm:asm-commons", version.ref = "ow2-asm" ow2-asm-tree = { module = "org.ow2.asm:asm-tree", version.ref = "ow2-asm" } # @keep transitive dependency for version alignment perfmark-api = { module = "io.perfmark:perfmark-api", version.ref = "perfmark" } +prometheus-metrics-core = { module = "io.prometheus:prometheus-metrics-core", version.ref = "prometheus-metrics" } +prometheus-metrics-exporter-servlet-jakarta = { module = "io.prometheus:prometheus-metrics-exporter-servlet-jakarta", version.ref = "prometheus-metrics" } prometheus-metrics-expositionformats = { module = "io.prometheus:prometheus-metrics-exposition-formats", version.ref = "prometheus-metrics" } prometheus-metrics-model = { module = "io.prometheus:prometheus-metrics-model", version.ref = "prometheus-metrics" } prometheus-simpleclient = { module = "io.prometheus:simpleclient", version.ref = "prometheus-simpleclient" } diff --git a/solr/cross-dc-manager/build.gradle b/solr/cross-dc-manager/build.gradle index e6b120190f2a..7c86b619dc7a 100644 --- a/solr/cross-dc-manager/build.gradle +++ b/solr/cross-dc-manager/build.gradle @@ -28,8 +28,10 @@ dependencies { implementation project(':solr:solrj-zookeeper') implementation project(':solr:modules:cross-dc') - implementation libs.dropwizard.metrics.core - implementation libs.dropwizard.metrics.servlets + implementation libs.prometheus.metrics.core + implementation libs.prometheus.metrics.exporter.servlet.jakarta + implementation libs.prometheus.metrics.expositionformats + implementation libs.prometheus.metrics.model implementation libs.eclipse.jetty.server implementation libs.eclipse.jetty.ee10.servlet implementation libs.slf4j.api diff --git a/solr/cross-dc-manager/gradle.lockfile b/solr/cross-dc-manager/gradle.lockfile index 6d9a1f755696..6fbf5d69c42d 100644 --- a/solr/cross-dc-manager/gradle.lockfile +++ b/solr/cross-dc-manager/gradle.lockfile @@ -4,8 +4,8 @@ com.carrotsearch.randomizedtesting:randomizedtesting-runner:2.8.3=jarValidation,testCompileClasspath,testRuntimeClasspath com.carrotsearch:hppc:0.10.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.fasterxml.jackson.core:jackson-annotations:2.20=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -com.fasterxml.jackson.core:jackson-core:2.20.1=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -com.fasterxml.jackson.core:jackson-databind:2.20.1=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +com.fasterxml.jackson.core:jackson-core:2.20.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +com.fasterxml.jackson.core:jackson-databind:2.20.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.20.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.fasterxml.jackson.dataformat:jackson-dataformat-csv:2.20.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.20.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath @@ -32,7 +32,6 @@ com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava=anno com.google.j2objc:j2objc-annotations:3.1=annotationProcessor,compileClasspath,errorprone,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testAnnotationProcessor,testCompileClasspath,testRuntimeClasspath com.google.protobuf:protobuf-java-util:3.25.8=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath com.google.protobuf:protobuf-java:3.25.8=annotationProcessor,errorprone,jarValidation,runtimeClasspath,runtimeLibs,testAnnotationProcessor,testRuntimeClasspath -com.helger:profiler:1.1.1=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath com.j256.simplemagic:simplemagic:1.17=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.jayway.jsonpath:json-path:2.9.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.lmax:disruptor:3.4.4=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath @@ -48,13 +47,9 @@ commons-digester:commons-digester:2.1=jarValidation,runtimeClasspath,runtimeLibs commons-io:commons-io:2.20.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath commons-validator:commons-validator:1.7=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.dropwizard.metrics:metrics-annotation:4.2.26=jarValidation,testRuntimeClasspath -io.dropwizard.metrics:metrics-core:4.2.26=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.dropwizard.metrics:metrics-healthchecks:4.2.26=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.dropwizard.metrics:metrics-jakarta-servlets:4.2.26=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.dropwizard.metrics:metrics-core:4.2.26=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.dropwizard.metrics:metrics-jetty12-ee10:4.2.26=jarValidation,testRuntimeClasspath io.dropwizard.metrics:metrics-jetty12:4.2.26=jarValidation,testRuntimeClasspath -io.dropwizard.metrics:metrics-json:4.2.26=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.dropwizard.metrics:metrics-jvm:4.2.26=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath io.github.eisop:dataflow-errorprone:3.41.0-eisop1=annotationProcessor,errorprone,testAnnotationProcessor io.github.java-diff-utils:java-diff-utils:4.12=annotationProcessor,errorprone,testAnnotationProcessor io.netty:netty-buffer:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath @@ -82,8 +77,17 @@ io.opentelemetry:opentelemetry-sdk-common:1.56.0=jarValidation,runtimeClasspath, io.opentelemetry:opentelemetry-sdk-metrics:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-sdk-trace:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-sdk:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.prometheus:prometheus-metrics-exposition-formats:1.1.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.prometheus:prometheus-metrics-model:1.1.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +io.prometheus:prometheus-metrics-config:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-core:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-exporter-common:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-exporter-servlet-jakarta:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-exposition-formats:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-model:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-shaded-protobuf:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-tracer-common:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-tracer-initializer:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-tracer-otel-agent:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-tracer-otel:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath io.sgr:s2-geometry-library-java:1.0.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.swagger.core.v3:swagger-annotations-jakarta:2.2.22=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath jakarta.annotation:jakarta.annotation-api:2.1.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java index 7e4219f49226..64bb61255232 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java @@ -20,15 +20,14 @@ import static org.apache.solr.crossdc.common.KafkaCrossDcConf.TOPIC_NAME; import static org.apache.solr.crossdc.common.KafkaCrossDcConf.ZK_CONNECT_STRING; -import com.codahale.metrics.SharedMetricRegistries; -import io.dropwizard.metrics.servlets.MetricsServlet; -import io.dropwizard.metrics.servlets.ThreadDumpServlet; import java.lang.invoke.MethodHandles; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; + +import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.util.ExecutorUtil; @@ -46,8 +45,6 @@ public class Consumer { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static final String METRICS_REGISTRY = "metrics"; - private Server server; private CrossDcConsumer crossDcConsumer; @@ -98,11 +95,9 @@ public void start(Map properties) { ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); context.setContextPath("/"); server.setHandler(context); + context.addServlet(ThreadDumpServlet.class, "/threads/*"); - context.addServlet(MetricsServlet.class, "/metrics/*"); - context.setAttribute( - "com.codahale.metrics.servlets.MetricsServlet.registry", - SharedMetricRegistries.getOrCreate(METRICS_REGISTRY)); + context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); for (ServletMapping mapping : context.getServletHandler().getServletMappings()) { if (log.isInfoEnabled()) { log.info(" - {}", mapping.getPathSpecs()[0]); diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java new file mode 100644 index 000000000000..2e189d818fd6 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -0,0 +1,34 @@ +package org.apache.solr.crossdc.manager.consumer; + +import io.prometheus.metrics.core.datapoints.Timer; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.crossdc.common.MirroredSolrRequest; + +public interface ConsumerMetrics { + + interface ConsumerTimer { + double observeDuration(); + + default void close() { + observeDuration(); + } + } + + void incrementCollapsedCounter(); + + void incrementInputCounter(String type, String subType); + + void incrementInputCounter(String type, String subType, int delta); + + void incrementOutputCounter(String type, String result); + + void incrementOutputCounter(String type, String result, int delta); + + void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest); + + void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs); + + void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs); + + ConsumerTimer startOutputTimeTimer(String requestType); +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index bf1ba691f263..3da208587c87 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -16,8 +16,6 @@ */ package org.apache.solr.crossdc.manager.consumer; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.SharedMetricRegistries; import java.lang.invoke.MethodHandles; import java.time.Duration; import java.util.Arrays; @@ -73,9 +71,6 @@ public class KafkaCrossDcConsumer extends Consumer.CrossDcConsumer { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final MetricRegistry metrics = - SharedMetricRegistries.getOrCreate(Consumer.METRICS_REGISTRY); - private final KafkaConsumer> kafkaConsumer; private final CountDownLatch startLatch; KafkaMirroringSink kafkaMirroringSink; @@ -86,6 +81,7 @@ public class KafkaCrossDcConsumer extends Consumer.CrossDcConsumer { private final CrossDcConf.CollapseUpdates collapseUpdates; private final int maxCollapseRecords; private final SolrMessageProcessor messageProcessor; + protected PrometheusMetrics metrics; protected SolrClientSupplier solrClientSupplier; @@ -164,6 +160,7 @@ public CloudSolrClient get() { * @param startLatch To inform the caller when the Consumer has started */ public KafkaCrossDcConsumer(KafkaCrossDcConf conf, CountDownLatch startLatch) { + this.metrics = new PrometheusMetrics(); this.topicNames = conf.get(KafkaCrossDcConf.TOPIC_NAME).split(","); this.maxAttempts = conf.getInt(KafkaCrossDcConf.MAX_ATTEMPTS); @@ -239,7 +236,7 @@ protected SolrClientSupplier createSolrClientSupplier(KafkaCrossDcConf conf) { } protected SolrMessageProcessor createSolrMessageProcessor() { - return new SolrMessageProcessor(solrClientSupplier, resubmitRequest -> 0L); + return new SolrMessageProcessor(metrics, solrClientSupplier, resubmitRequest -> 0L); } public KafkaConsumer> createKafkaConsumer(Properties properties) { @@ -345,7 +342,10 @@ boolean pollAndProcessRequests() { MirroredSolrRequest req = requestRecord.value(); SolrRequest solrReq = req.getSolrRequest(); MirroredSolrRequest.Type type = req.getType(); - metrics.counter(MetricRegistry.name(type.name(), "input")).inc(); + if (type != MirroredSolrRequest.Type.UPDATE) { + metrics.incrementInputCounter(type.name(), solrReq.getPath()); + } + ModifiableSolrParams params = new ModifiableSolrParams(solrReq.getParams()); if (log.isTraceEnabled()) { log.trace("-- picked type={}, params={}", req.getType(), params); @@ -398,7 +398,7 @@ boolean pollAndProcessRequests() { if (collapseUpdates == CrossDcConf.CollapseUpdates.PARTIAL && hasDeletes) { throw new RuntimeException("Can't collapse requests with deletions."); } - metrics.counter(MetricRegistry.name(type.name(), "collapsed")).inc(); + metrics.incrementCollapsedCounter(); currentCollapsed++; } UpdateRequest update = (UpdateRequest) solrReq; @@ -408,19 +408,19 @@ boolean pollAndProcessRequests() { List docs = update.getDocuments(); if (docs != null) { updateReqBatch.add(docs); - metrics.counter(MetricRegistry.name(type.name(), "add")).inc(docs.size()); + metrics.incrementInputCounter(type.name(), "add"); } List deletes = update.getDeleteById(); if (deletes != null) { updateReqBatch.deleteById(deletes); - metrics.counter(MetricRegistry.name(type.name(), "dbi")).inc(deletes.size()); + metrics.incrementInputCounter(type.name(), "dbi"); } List deleteByQuery = update.getDeleteQuery(); if (deleteByQuery != null) { for (String delByQuery : deleteByQuery) { updateReqBatch.deleteByQuery(delByQuery); } - metrics.counter(MetricRegistry.name(type.name(), "dbq")).inc(deleteByQuery.size()); + metrics.incrementInputCounter(type.name(), "dbq", deleteByQuery.size()); } } else { // non-update requests should be sent immediately @@ -505,6 +505,7 @@ public void sendBatch( final IQueueHandler.Result> result = messageProcessor.handleItem(mirroredSolrRequest); + metrics.recordOutputBatchSize(type, solrReqBatch); processResult(type, result); } catch (MirroringException e) { // We don't really know what to do here @@ -532,10 +533,10 @@ protected void processResult( "Sending message to dead letter queue because of max attempts limit with current value = {}", attempt); kafkaMirroringSink.submitToDlq(item); - metrics.counter(MetricRegistry.name(type.name(), "failed-dlq")).inc(); + metrics.incrementOutputCounter(type.name(), "failed-dlq"); } else { kafkaMirroringSink.submit(item); - metrics.counter(MetricRegistry.name(type.name(), "failed-resubmit")).inc(); + metrics.incrementOutputCounter(type.name(), "failed-resubmit"); } break; case HANDLED: @@ -543,18 +544,18 @@ protected void processResult( if (log.isTraceEnabled()) { log.trace("result=handled"); } - metrics.counter(MetricRegistry.name(type.name(), "handled")).inc(); + metrics.incrementOutputCounter(type.name(), "handled"); break; case NOT_HANDLED_SHUTDOWN: if (log.isTraceEnabled()) { log.trace("result=nothandled_shutdown"); } - metrics.counter(MetricRegistry.name(type.name(), "nothandled_shutdown")).inc(); + metrics.incrementOutputCounter(type.name(), "nothandled_shutdown"); break; case FAILED_RETRY: log.error( "Unexpected response while processing request. We never expect {}.", result.status()); - metrics.counter(MetricRegistry.name(type.name(), "failed-retry")).inc(); + metrics.incrementOutputCounter(type.name(), "failed-retry"); break; default: if (log.isTraceEnabled()) { @@ -585,7 +586,7 @@ public final void shutdown() { } catch (Exception e) { log.warn("Exception closing Solr client on shutdown", e); } finally { - Util.logMetrics(metrics); + Util.logMetrics(metrics.getRegistry()); } } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java new file mode 100644 index 000000000000..637da7d3b145 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.solr.crossdc.manager.consumer; + +import io.prometheus.metrics.core.datapoints.Timer; +import io.prometheus.metrics.core.metrics.Counter; +import io.prometheus.metrics.core.metrics.Histogram; +import io.prometheus.metrics.model.registry.PrometheusRegistry; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.crossdc.common.MirroredSolrRequest; + +public class PrometheusMetrics implements ConsumerMetrics { + + protected PrometheusRegistry registry; + protected Counter input; + protected Counter collapsed; + protected Counter output; + protected Histogram outputBatchSizeHistogram; + protected Histogram outputTimeHistogram; + protected Histogram outputBackoffHistogram; + protected Histogram outputFirstAttemptHistogram; + + public PrometheusMetrics() { + register(new PrometheusRegistry()); + } + + protected void register(PrometheusRegistry registry) { + this.registry = registry; + input = Counter.builder() + .name("consumer_input_total") + .help("Total number of input messages") + .labelNames("type", "subtype") + .register(registry); + + collapsed = Counter.builder() + .name("consumer_collapsed_total") + .help("Total number of collapsed messages") + .register(registry); + + output = Counter.builder() + .name("consumer_output_total") + .help("Total number of output requests") + .labelNames("type", "result") + .register(registry); + + outputBatchSizeHistogram = Histogram.builder() + .name("consumer_output_batch_size_histogram") + .help("Histogram of output batch sizes") + .labelNames("type", "subtype") + .register(registry); + + outputBackoffHistogram = Histogram.builder() + .name("consumer_output_backoff_histogram") + .help("Histogram of output backoff sleep times") + .labelNames("type") + .register(registry); + + outputTimeHistogram = Histogram.builder() + .name("consumer_output_time_histogram") + .help("Histogram of output request times") + .labelNames("type") + .register(registry); + + outputFirstAttemptHistogram = Histogram.builder() + .name("consumer_output_first_attempt_histogram") + .help("Histogram of first attempt request times") + .labelNames("type") + .register(registry); + } + + public PrometheusRegistry getRegistry() { + return registry; + } + + @Override + public void incrementCollapsedCounter() { + collapsed.inc(); + } + + @Override + public void incrementInputCounter(String type, String subType) { + incrementInputCounter(type, subType, 1); + } + + @Override + public void incrementInputCounter(String type, String subType, int delta) { + input.labelValues(type, subType).inc(delta); + } + + @Override + public void incrementOutputCounter(String type, String result) { + incrementOutputCounter(type, result, 1); + } + + @Override + public void incrementOutputCounter(String type, String result, int delta) { + input.labelValues(type, result).inc(delta); + } + + @Override + public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { + if (type != MirroredSolrRequest.Type.UPDATE) { + outputBatchSizeHistogram.labelValues(type.name(), solrRequest.getPath()).observe(1); + return; + } + UpdateRequest req = (UpdateRequest) solrRequest; + int addCount = req.getDocuments() == null ? 0 : req.getDocuments().size(); + int dbiCount = req.getDeleteById() == null ? 0 : req.getDeleteById().size(); + int dbqCount = req.getDeleteQuery() == null ? 0 : req.getDeleteQuery().size(); + if (addCount > 0) { + outputBatchSizeHistogram.labelValues(type.name(), "add").observe(addCount); + } + if (dbiCount > 0) { + outputBatchSizeHistogram.labelValues(type.name(), "dbi").observe(dbiCount); + } + if (dbqCount > 0) { + outputBatchSizeHistogram.labelValues(type.name(), "dbq").observe(dbqCount); + } + } + + @Override + public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { + outputBackoffHistogram.labelValues(type.name()).observe(backoffTimeMs); + } + + @Override + public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { + outputFirstAttemptHistogram.labelValues(type.name()).observe(firstAttemptTimeNs); + } + + @Override + public ConsumerTimer startOutputTimeTimer(String requestType) { + final Timer timer = outputTimeHistogram.labelValues(requestType).startTimer(); + return () -> timer.observeDuration(); + } +} \ No newline at end of file diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java new file mode 100644 index 000000000000..826952ceec62 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.solr.crossdc.manager.consumer; + +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.lang.management.LockInfo; +import java.lang.management.MonitorInfo; +import java.lang.management.ThreadInfo; +import java.lang.management.ThreadMXBean; +import java.nio.charset.StandardCharsets; + +/** + * A convenience class for getting a thread dump. + * + *

Copy of the code in https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jvm/src/main/java/io/dropwizard/metrics5/jvm/ThreadDump.java

+ */ +public class ThreadDump { + private final ThreadMXBean threadMXBean; + + public ThreadDump(ThreadMXBean threadMXBean) { + this.threadMXBean = threadMXBean; + } + + /** + * Dumps all of the threads' current information, including synchronization, to an output stream. + * + * @param out an output stream + */ + public void dump(OutputStream out) { + dump(true, true, out); + } + + /** + * Dumps all of the threads' current information, optionally including synchronization, to an output stream. + * + * Having control over including synchronization info allows using this method (and its wrappers, i.e. + * ThreadDumpServlet) in environments where getting object monitor and/or ownable synchronizer usage is not + * supported. It can also speed things up. + * + * See {@link ThreadMXBean#dumpAllThreads(boolean, boolean)} + * + * @param lockedMonitors dump all locked monitors if true + * @param lockedSynchronizers dump all locked ownable synchronizers if true + * @param out an output stream + */ + public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStream out) { + final ThreadInfo[] threads = this.threadMXBean.dumpAllThreads(lockedMonitors, lockedSynchronizers); + final PrintWriter writer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); + + for (int ti = threads.length - 1; ti >= 0; ti--) { + final ThreadInfo t = threads[ti]; + writer.printf("\"%s\" id=%d state=%s", + t.getThreadName(), + t.getThreadId(), + t.getThreadState()); + final LockInfo lock = t.getLockInfo(); + if (lock != null && t.getThreadState() != Thread.State.BLOCKED) { + writer.printf("%n - waiting on <0x%08x> (a %s)", + lock.getIdentityHashCode(), + lock.getClassName()); + writer.printf("%n - locked <0x%08x> (a %s)", + lock.getIdentityHashCode(), + lock.getClassName()); + } else if (lock != null && t.getThreadState() == Thread.State.BLOCKED) { + writer.printf("%n - waiting to lock <0x%08x> (a %s)", + lock.getIdentityHashCode(), + lock.getClassName()); + } + + if (t.isSuspended()) { + writer.print(" (suspended)"); + } + + if (t.isInNative()) { + writer.print(" (running in native)"); + } + + writer.println(); + if (t.getLockOwnerName() != null) { + writer.printf(" owned by %s id=%d%n", t.getLockOwnerName(), t.getLockOwnerId()); + } + + final StackTraceElement[] elements = t.getStackTrace(); + final MonitorInfo[] monitors = t.getLockedMonitors(); + + for (int i = 0; i < elements.length; i++) { + final StackTraceElement element = elements[i]; + writer.printf(" at %s%n", element); + for (int j = 1; j < monitors.length; j++) { + final MonitorInfo monitor = monitors[j]; + if (monitor.getLockedStackDepth() == i) { + writer.printf(" - locked %s%n", monitor); + } + } + } + writer.println(); + + final LockInfo[] locks = t.getLockedSynchronizers(); + if (locks.length > 0) { + writer.printf(" Locked synchronizers: count = %d%n", locks.length); + for (LockInfo l : locks) { + writer.printf(" - %s%n", l); + } + writer.println(); + } + } + + writer.println(); + writer.flush(); + } + +} \ No newline at end of file diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java new file mode 100644 index 000000000000..cf6f51c8f592 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.solr.crossdc.manager.consumer; + +import jakarta.servlet.ServletException; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.management.ManagementFactory; + +/** + * An HTTP servlets which outputs a {@code text/plain} dump of all threads in + * the VM. Only responds to {@code GET} requests. + *

Copy of the code from https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jakarta-servlets/src/main/java/io/dropwizard/metrics5/servlets/ThreadDumpServlet.java

+ */ +public class ThreadDumpServlet extends HttpServlet { + + private static final long serialVersionUID = -2690343532336103046L; + private static final String CONTENT_TYPE = "text/plain"; + + private transient ThreadDump threadDump; + + @Override + public void init() throws ServletException { + try { + // Some PaaS like Google App Engine blacklist java.lang.managament + this.threadDump = new ThreadDump(ManagementFactory.getThreadMXBean()); + } catch (NoClassDefFoundError ncdfe) { + this.threadDump = null; // we won't be able to provide thread dump + } + } + + @Override + protected void doGet(HttpServletRequest req, + HttpServletResponse resp) throws ServletException, IOException { + final boolean includeMonitors = getParam(req.getParameter("monitors"), true); + final boolean includeSynchronizers = getParam(req.getParameter("synchronizers"), true); + + resp.setStatus(HttpServletResponse.SC_OK); + resp.setContentType(CONTENT_TYPE); + resp.setHeader("Cache-Control", "must-revalidate,no-cache,no-store"); + if (threadDump == null) { + resp.getWriter().println("Sorry your runtime environment does not allow to dump threads."); + return; + } + try (OutputStream output = resp.getOutputStream()) { + threadDump.dump(includeMonitors, includeSynchronizers, output); + } + } + + private static Boolean getParam(String initParam, boolean defaultValue) { + return initParam == null ? defaultValue : Boolean.parseBoolean(initParam); + } +} \ No newline at end of file diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index 4c964c1a739d..6eddade50c01 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -16,16 +16,16 @@ */ package org.apache.solr.crossdc.manager.consumer; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.lang.invoke.MethodHandles; -import java.util.Map; +import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.Set; + +import io.prometheus.metrics.expositionformats.ExpositionFormats; +import io.prometheus.metrics.model.registry.PrometheusRegistry; +import io.prometheus.metrics.model.snapshots.MetricSnapshots; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListTopicsOptions; @@ -42,32 +42,14 @@ public class Util { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @SuppressWarnings("rawtypes") - public static void logMetrics(MetricRegistry metricRegistry) { - log.info("Metrics Registry:"); - for (Map.Entry entry : metricRegistry.getGauges().entrySet()) { - if (log.isInfoEnabled()) { - log.info("Gauge {}: {}", entry.getKey(), entry.getValue().getValue()); - } - } - for (Map.Entry entry : metricRegistry.getCounters().entrySet()) { - if (log.isInfoEnabled()) { - log.info("Counter {}: {}", entry.getKey(), entry.getValue().getCount()); - } - } - for (Map.Entry entry : metricRegistry.getHistograms().entrySet()) { - if (log.isInfoEnabled()) { - log.info("Histogram {}: {}", entry.getKey(), entry.getValue().getSnapshot().toString()); - } - } - for (Map.Entry entry : metricRegistry.getMeters().entrySet()) { - if (log.isInfoEnabled()) { - log.info("Meter {}: {}", entry.getKey(), entry.getValue().getCount()); - } - } - for (Map.Entry entry : metricRegistry.getTimers().entrySet()) { - if (log.isInfoEnabled()) { - log.info("Timer {}: {}", entry.getKey(), entry.getValue().getSnapshot().toString()); - } + public static void logMetrics(PrometheusRegistry registry) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + MetricSnapshots snapshots = registry.scrape(); + try { + ExpositionFormats.init().getPrometheusTextFormatWriter().write(baos, snapshots); + log.info(baos.toString(StandardCharsets.UTF_8)); + } catch (IOException e) { + log.error("Error while logging metrics", e); } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index f0ac2b182a27..a4d7acb9f2c9 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -16,14 +16,13 @@ */ package org.apache.solr.crossdc.manager.messageprocessor; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.SharedMetricRegistries; -import com.codahale.metrics.Timer; import java.lang.invoke.MethodHandles; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; + +import io.prometheus.metrics.core.datapoints.Timer; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.UpdateRequest; @@ -40,6 +39,7 @@ import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; import org.apache.solr.crossdc.common.SolrExceptionUtil; import org.apache.solr.crossdc.manager.consumer.Consumer; +import org.apache.solr.crossdc.manager.consumer.ConsumerMetrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.MDC; @@ -57,16 +57,15 @@ public class SolrMessageProcessor extends MessageProcessor implements IQueueHandler> { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final MetricRegistry metrics = - SharedMetricRegistries.getOrCreate(Consumer.METRICS_REGISTRY); - + final ConsumerMetrics metrics; final Supplier clientSupplier; private static final String VERSION_FIELD = "_version_"; - public SolrMessageProcessor( + public SolrMessageProcessor(ConsumerMetrics metrics, Supplier clientSupplier, ResubmitBackoffPolicy resubmitBackoffPolicy) { super(resubmitBackoffPolicy); + this.metrics = metrics; this.clientSupplier = clientSupplier; } @@ -145,7 +144,7 @@ private void maybeBackoff(MirroredSolrRequest request, SolrException solrExce sleepTimeMs = Math.max(1, Long.parseLong(backoffTimeSuggested)); } log.info("Consumer backoff. sleepTimeMs={}", sleepTimeMs); - metrics.meter(MetricRegistry.name(request.getType().name(), "backoff")).mark(sleepTimeMs); + metrics.recordOutputBackoffSize(request.getType(), sleepTimeMs); uncheckedSleep(sleepTimeMs); } @@ -197,11 +196,14 @@ private Result> processMirroredSolrRequest( } Result> result; SolrResponseBase response; - Timer.Context ctx = metrics.timer(MetricRegistry.name(type.name(), "outputTime")).time(); + ConsumerMetrics.ConsumerTimer timer = metrics.startOutputTimeTimer(type.name()); try { response = (SolrResponseBase) request.process(clientSupplier.get()); } finally { - ctx.stop(); + // unit tests might not care + if (timer != null) { + timer.close(); + } } int status = response.getStatus(); @@ -211,7 +213,7 @@ private Result> processMirroredSolrRequest( } if (status != 0) { - metrics.counter(MetricRegistry.name(type.name(), "outputErrors")).inc(); + metrics.incrementOutputCounter(type.name(), "solrError"); throw new SolrException(SolrException.ErrorCode.getErrorCode(status), "response=" + response); } @@ -306,9 +308,7 @@ private void logFirstAttemptLatency(MirroredSolrRequest mirroredSolrRequest) if (mirroredSolrRequest.getAttempt() == 1) { final long latency = System.nanoTime() - mirroredSolrRequest.getSubmitTimeNanos(); log.debug("First attempt latency = {} ns", latency); - metrics - .timer(MetricRegistry.name(mirroredSolrRequest.getType().name(), "outputLatency")) - .update(latency, TimeUnit.NANOSECONDS); + metrics.recordOutputFirstAttemptSize(mirroredSolrRequest.getType(), latency); } } @@ -377,7 +377,7 @@ private void backoffIfNeeded( if (result.status().equals(ResultStatus.FAILED_RESUBMIT)) { final long backoffMs = getResubmitBackoffPolicy().getBackoffTimeMs(result.getItem()); if (backoffMs > 0L) { - metrics.meter(MetricRegistry.name(type.name(), "backoff")).mark(backoffMs); + metrics.recordOutputBackoffSize(type, backoffMs); try { Thread.sleep(backoffMs); } catch (final InterruptedException ex) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java index 421f7b1c0c17..eb14acb3c00a 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java @@ -164,7 +164,7 @@ protected CrossDcConsumer getCrossDcConsumer( return new KafkaCrossDcConsumer(conf, startLatch) { @Override protected SolrMessageProcessor createSolrMessageProcessor() { - return new SolrMessageProcessor(solrClientSupplier, resubmitRequest -> 0L) { + return new SolrMessageProcessor(metrics, solrClientSupplier, resubmitRequest -> 0L) { @Override public Result> handleItem( MirroredSolrRequest mirroredSolrRequest) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java index 06f5add80c03..a4837ab324f4 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.solr.crossdc.manager; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import java.util.Map; @@ -26,6 +27,7 @@ import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.crossdc.common.MirroredSolrRequest; +import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; import org.apache.solr.crossdc.manager.messageprocessor.SolrMessageProcessor; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -51,7 +53,7 @@ public static void beforeSimpleSolrIntegrationTest() throws Exception { CloudSolrClient cloudClient1 = cluster1.getSolrClient(); - processor = new SolrMessageProcessor(() -> cloudClient1, null); + processor = new SolrMessageProcessor(mock(PrometheusMetrics.class), () -> cloudClient1, null); CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLLECTION, 1, 1); diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java new file mode 100644 index 000000000000..e91131e73707 --- /dev/null +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java @@ -0,0 +1,60 @@ +package org.apache.solr.crossdc.manager.consumer; + +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.crossdc.common.MirroredSolrRequest; + +public class NoopConsumerMetrics implements ConsumerMetrics { + public static final NoopConsumerMetrics INSTANCE = new NoopConsumerMetrics(); + + @Override + public void incrementCollapsedCounter() { + + } + + @Override + public void incrementInputCounter(String type, String subType) { + + } + + @Override + public void incrementInputCounter(String type, String subType, int delta) { + + } + + @Override + public void incrementOutputCounter(String type, String result) { + + } + + @Override + public void incrementOutputCounter(String type, String result, int delta) { + + } + + @Override + public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { + + } + + @Override + public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { + + } + + @Override + public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { + + } + + private static final ConsumerTimer NOOP_TIMER = new ConsumerTimer() { + @Override + public double observeDuration() { + return 0; + } + }; + + @Override + public ConsumerTimer startOutputTimeTimer(String requestType) { + return NOOP_TIMER; + } +} diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java index 2a1b552ab260..1a9d7168a128 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java @@ -36,6 +36,7 @@ import org.apache.solr.crossdc.common.IQueueHandler; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; +import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -55,7 +56,7 @@ public static void ensureWorkingMockito() { public void setUp() { client = mock(CloudSolrClient.class); resubmitBackoffPolicy = mock(ResubmitBackoffPolicy.class); - solrMessageProcessor = new SolrMessageProcessor(() -> client, resubmitBackoffPolicy); + solrMessageProcessor = new SolrMessageProcessor(mock(PrometheusMetrics.class), () -> client, resubmitBackoffPolicy); } /** Should handle MirroredSolrRequest and return a failed result with no retry */ diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java index a3c162ce7614..0a686f650110 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java @@ -38,6 +38,8 @@ import org.apache.solr.crossdc.common.IQueueHandler; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; +import org.apache.solr.crossdc.manager.consumer.ConsumerMetrics; +import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; @@ -70,7 +72,8 @@ public static void ensureWorkingMockito() { public void setUp() { MockitoAnnotations.initMocks(this); - processor = Mockito.spy(new SolrMessageProcessor(() -> solrClient, backoffPolicy)); + ConsumerMetrics metrics = Mockito.mock(PrometheusMetrics.class); + processor = Mockito.spy(new SolrMessageProcessor(metrics, () -> solrClient, backoffPolicy)); Mockito.doNothing().when(processor).uncheckedSleep(anyLong()); } From aff9064dfe1d61c8b281c9cbdd7de6b531c2954e Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 19 Jan 2026 14:23:57 +0100 Subject: [PATCH 02/15] SOLR-18060: Add Prometheus metrics to CrossDC Consumer. --- changelog/unreleased/solr-18060.yml | 9 ++ .../crossdc/manager/consumer/Consumer.java | 6 +- .../manager/consumer/ConsumerMetrics.java | 1 - .../consumer/KafkaCrossDcConsumer.java | 6 +- .../manager/consumer/PrometheusMetrics.java | 89 ++++++++++--------- .../crossdc/manager/consumer/ThreadDump.java | 43 +++++---- .../manager/consumer/ThreadDumpServlet.java | 16 ++-- .../solr/crossdc/manager/consumer/Util.java | 7 +- .../SolrMessageProcessor.java | 10 +-- .../manager/DeleteByQueryToIdTest.java | 3 +- .../manager/consumer/NoopConsumerMetrics.java | 60 ------------- .../SolrMessageProcessorTest.java | 4 +- .../metrics-healthchecks-4.2.26.jar.sha1 | 1 - .../metrics-jakarta-servlets-4.2.26.jar.sha1 | 1 - solr/licenses/metrics-json-4.2.26.jar.sha1 | 1 - solr/licenses/metrics-jvm-4.2.26.jar.sha1 | 1 - solr/licenses/profiler-1.1.1.jar.sha1 | 1 - .../prometheus-metrics-config-1.1.0.jar.sha1 | 1 + .../prometheus-metrics-core-1.1.0.jar.sha1 | 1 + ...eus-metrics-exporter-common-1.1.0.jar.sha1 | 1 + ...cs-exporter-servlet-jakarta-1.1.0.jar.sha1 | 1 + ...eus-metrics-shaded-protobuf-1.1.0.jar.sha1 | 1 + ...theus-metrics-tracer-common-1.1.0.jar.sha1 | 1 + ...-metrics-tracer-initializer-1.1.0.jar.sha1 | 1 + ...metheus-metrics-tracer-otel-1.1.0.jar.sha1 | 1 + ...s-metrics-tracer-otel-agent-1.1.0.jar.sha1 | 1 + .../SolrKafkaTestsIgnoredThreadsFilter.java | 5 ++ 27 files changed, 119 insertions(+), 154 deletions(-) create mode 100644 changelog/unreleased/solr-18060.yml delete mode 100644 solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java delete mode 100644 solr/licenses/metrics-healthchecks-4.2.26.jar.sha1 delete mode 100644 solr/licenses/metrics-jakarta-servlets-4.2.26.jar.sha1 delete mode 100644 solr/licenses/metrics-json-4.2.26.jar.sha1 delete mode 100644 solr/licenses/metrics-jvm-4.2.26.jar.sha1 delete mode 100644 solr/licenses/profiler-1.1.1.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 create mode 100644 solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 diff --git a/changelog/unreleased/solr-18060.yml b/changelog/unreleased/solr-18060.yml new file mode 100644 index 000000000000..773805a8bbf4 --- /dev/null +++ b/changelog/unreleased/solr-18060.yml @@ -0,0 +1,9 @@ +# See https://github.com/apache/solr/blob/main/dev-docs/changelog.adoc +title: CrossDC Consumer - add Prometheus metrics +type: added +authors: + - name: Andrzej Bialecki + nick: ab +links: + - name: SOLR-18060 + url: https://issues.apache.org/jira/browse/SOLR-18060 diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java index 64bb61255232..12d13fdbe835 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java @@ -26,8 +26,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; - -import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.util.ExecutorUtil; @@ -96,8 +94,8 @@ public void start(Map properties) { context.setContextPath("/"); server.setHandler(context); - context.addServlet(ThreadDumpServlet.class, "/threads/*"); - context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); + // context.addServlet(ThreadDumpServlet.class, "/threads/*"); + // context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); for (ServletMapping mapping : context.getServletHandler().getServletMappings()) { if (log.isInfoEnabled()) { log.info(" - {}", mapping.getPathSpecs()[0]); diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java index 2e189d818fd6..010dee9a847f 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -1,6 +1,5 @@ package org.apache.solr.crossdc.manager.consumer; -import io.prometheus.metrics.core.datapoints.Timer; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.crossdc.common.MirroredSolrRequest; diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index 3da208587c87..20419cf4bfab 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -81,7 +81,7 @@ public class KafkaCrossDcConsumer extends Consumer.CrossDcConsumer { private final CrossDcConf.CollapseUpdates collapseUpdates; private final int maxCollapseRecords; private final SolrMessageProcessor messageProcessor; - protected PrometheusMetrics metrics; + protected ConsumerMetrics metrics; protected SolrClientSupplier solrClientSupplier; @@ -586,7 +586,9 @@ public final void shutdown() { } catch (Exception e) { log.warn("Exception closing Solr client on shutdown", e); } finally { - Util.logMetrics(metrics.getRegistry()); + if (metrics instanceof PrometheusMetrics) { + Util.logMetrics(((PrometheusMetrics) metrics).getRegistry()); + } } } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java index 637da7d3b145..873995a53696 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java @@ -41,46 +41,53 @@ public PrometheusMetrics() { protected void register(PrometheusRegistry registry) { this.registry = registry; - input = Counter.builder() - .name("consumer_input_total") - .help("Total number of input messages") - .labelNames("type", "subtype") - .register(registry); - - collapsed = Counter.builder() - .name("consumer_collapsed_total") - .help("Total number of collapsed messages") - .register(registry); - - output = Counter.builder() - .name("consumer_output_total") - .help("Total number of output requests") - .labelNames("type", "result") - .register(registry); - - outputBatchSizeHistogram = Histogram.builder() - .name("consumer_output_batch_size_histogram") - .help("Histogram of output batch sizes") - .labelNames("type", "subtype") - .register(registry); - - outputBackoffHistogram = Histogram.builder() - .name("consumer_output_backoff_histogram") - .help("Histogram of output backoff sleep times") - .labelNames("type") - .register(registry); - - outputTimeHistogram = Histogram.builder() - .name("consumer_output_time_histogram") - .help("Histogram of output request times") - .labelNames("type") - .register(registry); - - outputFirstAttemptHistogram = Histogram.builder() - .name("consumer_output_first_attempt_histogram") - .help("Histogram of first attempt request times") - .labelNames("type") - .register(registry); + input = + Counter.builder() + .name("consumer_input_total") + .help("Total number of input messages") + .labelNames("type", "subtype") + .register(registry); + + collapsed = + Counter.builder() + .name("consumer_collapsed_total") + .help("Total number of collapsed messages") + .register(registry); + + output = + Counter.builder() + .name("consumer_output_total") + .help("Total number of output requests") + .labelNames("type", "result") + .register(registry); + + outputBatchSizeHistogram = + Histogram.builder() + .name("consumer_output_batch_size_histogram") + .help("Histogram of output batch sizes") + .labelNames("type", "subtype") + .register(registry); + + outputBackoffHistogram = + Histogram.builder() + .name("consumer_output_backoff_histogram") + .help("Histogram of output backoff sleep times") + .labelNames("type") + .register(registry); + + outputTimeHistogram = + Histogram.builder() + .name("consumer_output_time_histogram") + .help("Histogram of output request times") + .labelNames("type") + .register(registry); + + outputFirstAttemptHistogram = + Histogram.builder() + .name("consumer_output_first_attempt_histogram") + .help("Histogram of first attempt request times") + .labelNames("type") + .register(registry); } public PrometheusRegistry getRegistry() { @@ -148,4 +155,4 @@ public ConsumerTimer startOutputTimeTimer(String requestType) { final Timer timer = outputTimeHistogram.labelValues(requestType).startTimer(); return () -> timer.observeDuration(); } -} \ No newline at end of file +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java index 826952ceec62..b93b9d441f61 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java @@ -28,7 +28,9 @@ /** * A convenience class for getting a thread dump. * - *

Copy of the code in https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jvm/src/main/java/io/dropwizard/metrics5/jvm/ThreadDump.java

+ *

Copy of the code in + * https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jvm/src/main/java/io/dropwizard/metrics5/jvm/ThreadDump.java + * */ public class ThreadDump { private final ThreadMXBean threadMXBean; @@ -47,40 +49,38 @@ public void dump(OutputStream out) { } /** - * Dumps all of the threads' current information, optionally including synchronization, to an output stream. + * Dumps all of the threads' current information, optionally including synchronization, to an + * output stream. * - * Having control over including synchronization info allows using this method (and its wrappers, i.e. - * ThreadDumpServlet) in environments where getting object monitor and/or ownable synchronizer usage is not - * supported. It can also speed things up. + *

Having control over including synchronization info allows using this method (and its + * wrappers, i.e. ThreadDumpServlet) in environments where getting object monitor and/or ownable + * synchronizer usage is not supported. It can also speed things up. * - * See {@link ThreadMXBean#dumpAllThreads(boolean, boolean)} + *

See {@link ThreadMXBean#dumpAllThreads(boolean, boolean)} * * @param lockedMonitors dump all locked monitors if true * @param lockedSynchronizers dump all locked ownable synchronizers if true * @param out an output stream */ public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStream out) { - final ThreadInfo[] threads = this.threadMXBean.dumpAllThreads(lockedMonitors, lockedSynchronizers); + final ThreadInfo[] threads = + this.threadMXBean.dumpAllThreads(lockedMonitors, lockedSynchronizers); final PrintWriter writer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); for (int ti = threads.length - 1; ti >= 0; ti--) { final ThreadInfo t = threads[ti]; - writer.printf("\"%s\" id=%d state=%s", - t.getThreadName(), - t.getThreadId(), - t.getThreadState()); + writer.printf( + "\"%s\" id=%d state=%s", t.getThreadName(), t.getThreadId(), t.getThreadState()); final LockInfo lock = t.getLockInfo(); if (lock != null && t.getThreadState() != Thread.State.BLOCKED) { - writer.printf("%n - waiting on <0x%08x> (a %s)", - lock.getIdentityHashCode(), - lock.getClassName()); - writer.printf("%n - locked <0x%08x> (a %s)", - lock.getIdentityHashCode(), - lock.getClassName()); + writer.printf( + "%n - waiting on <0x%08x> (a %s)", lock.getIdentityHashCode(), lock.getClassName()); + writer.printf( + "%n - locked <0x%08x> (a %s)", lock.getIdentityHashCode(), lock.getClassName()); } else if (lock != null && t.getThreadState() == Thread.State.BLOCKED) { - writer.printf("%n - waiting to lock <0x%08x> (a %s)", - lock.getIdentityHashCode(), - lock.getClassName()); + writer.printf( + "%n - waiting to lock <0x%08x> (a %s)", + lock.getIdentityHashCode(), lock.getClassName()); } if (t.isSuspended()) { @@ -124,5 +124,4 @@ public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStre writer.println(); writer.flush(); } - -} \ No newline at end of file +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java index cf6f51c8f592..7967365d0f61 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java @@ -20,15 +20,17 @@ import jakarta.servlet.http.HttpServlet; import jakarta.servlet.http.HttpServletRequest; import jakarta.servlet.http.HttpServletResponse; - import java.io.IOException; import java.io.OutputStream; import java.lang.management.ManagementFactory; /** - * An HTTP servlets which outputs a {@code text/plain} dump of all threads in - * the VM. Only responds to {@code GET} requests. - *

Copy of the code from https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jakarta-servlets/src/main/java/io/dropwizard/metrics5/servlets/ThreadDumpServlet.java

+ * An HTTP servlets which outputs a {@code text/plain} dump of all threads in the VM. Only responds + * to {@code GET} requests. + * + *

Copy of the code from + * https://github.com/dropwizard/metrics/blob/release/5.0.x/metrics-jakarta-servlets/src/main/java/io/dropwizard/metrics5/servlets/ThreadDumpServlet.java + * */ public class ThreadDumpServlet extends HttpServlet { @@ -48,8 +50,8 @@ public void init() throws ServletException { } @Override - protected void doGet(HttpServletRequest req, - HttpServletResponse resp) throws ServletException, IOException { + protected void doGet(HttpServletRequest req, HttpServletResponse resp) + throws ServletException, IOException { final boolean includeMonitors = getParam(req.getParameter("monitors"), true); final boolean includeSynchronizers = getParam(req.getParameter("synchronizers"), true); @@ -68,4 +70,4 @@ protected void doGet(HttpServletRequest req, private static Boolean getParam(String initParam, boolean defaultValue) { return initParam == null ? defaultValue : Boolean.parseBoolean(initParam); } -} \ No newline at end of file +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index 6eddade50c01..d34e3e0f6402 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -16,16 +16,15 @@ */ package org.apache.solr.crossdc.manager.consumer; +import io.prometheus.metrics.expositionformats.ExpositionFormats; +import io.prometheus.metrics.model.registry.PrometheusRegistry; +import io.prometheus.metrics.model.snapshots.MetricSnapshots; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.Set; - -import io.prometheus.metrics.expositionformats.ExpositionFormats; -import io.prometheus.metrics.model.registry.PrometheusRegistry; -import io.prometheus.metrics.model.snapshots.MetricSnapshots; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListTopicsOptions; diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index a4d7acb9f2c9..66262e48d6c7 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -19,10 +19,7 @@ import java.lang.invoke.MethodHandles; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; - -import io.prometheus.metrics.core.datapoints.Timer; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.UpdateRequest; @@ -38,7 +35,6 @@ import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; import org.apache.solr.crossdc.common.SolrExceptionUtil; -import org.apache.solr.crossdc.manager.consumer.Consumer; import org.apache.solr.crossdc.manager.consumer.ConsumerMetrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,8 +58,10 @@ public class SolrMessageProcessor extends MessageProcessor private static final String VERSION_FIELD = "_version_"; - public SolrMessageProcessor(ConsumerMetrics metrics, - Supplier clientSupplier, ResubmitBackoffPolicy resubmitBackoffPolicy) { + public SolrMessageProcessor( + ConsumerMetrics metrics, + Supplier clientSupplier, + ResubmitBackoffPolicy resubmitBackoffPolicy) { super(resubmitBackoffPolicy); this.metrics = metrics; this.clientSupplier = clientSupplier; diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java index eb14acb3c00a..752d5f643410 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java @@ -164,7 +164,8 @@ protected CrossDcConsumer getCrossDcConsumer( return new KafkaCrossDcConsumer(conf, startLatch) { @Override protected SolrMessageProcessor createSolrMessageProcessor() { - return new SolrMessageProcessor(metrics, solrClientSupplier, resubmitRequest -> 0L) { + return new SolrMessageProcessor( + metrics, solrClientSupplier, resubmitRequest -> 0L) { @Override public Result> handleItem( MirroredSolrRequest mirroredSolrRequest) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java deleted file mode 100644 index e91131e73707..000000000000 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/NoopConsumerMetrics.java +++ /dev/null @@ -1,60 +0,0 @@ -package org.apache.solr.crossdc.manager.consumer; - -import org.apache.solr.client.solrj.SolrRequest; -import org.apache.solr.crossdc.common.MirroredSolrRequest; - -public class NoopConsumerMetrics implements ConsumerMetrics { - public static final NoopConsumerMetrics INSTANCE = new NoopConsumerMetrics(); - - @Override - public void incrementCollapsedCounter() { - - } - - @Override - public void incrementInputCounter(String type, String subType) { - - } - - @Override - public void incrementInputCounter(String type, String subType, int delta) { - - } - - @Override - public void incrementOutputCounter(String type, String result) { - - } - - @Override - public void incrementOutputCounter(String type, String result, int delta) { - - } - - @Override - public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { - - } - - @Override - public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { - - } - - @Override - public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { - - } - - private static final ConsumerTimer NOOP_TIMER = new ConsumerTimer() { - @Override - public double observeDuration() { - return 0; - } - }; - - @Override - public ConsumerTimer startOutputTimeTimer(String requestType) { - return NOOP_TIMER; - } -} diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java index 1a9d7168a128..20a72d2fa0fa 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java @@ -56,7 +56,9 @@ public static void ensureWorkingMockito() { public void setUp() { client = mock(CloudSolrClient.class); resubmitBackoffPolicy = mock(ResubmitBackoffPolicy.class); - solrMessageProcessor = new SolrMessageProcessor(mock(PrometheusMetrics.class), () -> client, resubmitBackoffPolicy); + solrMessageProcessor = + new SolrMessageProcessor( + mock(PrometheusMetrics.class), () -> client, resubmitBackoffPolicy); } /** Should handle MirroredSolrRequest and return a failed result with no retry */ diff --git a/solr/licenses/metrics-healthchecks-4.2.26.jar.sha1 b/solr/licenses/metrics-healthchecks-4.2.26.jar.sha1 deleted file mode 100644 index 4d1ab22e72a7..000000000000 --- a/solr/licenses/metrics-healthchecks-4.2.26.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -50819fda1745b03673eff3bdcddf914999045673 diff --git a/solr/licenses/metrics-jakarta-servlets-4.2.26.jar.sha1 b/solr/licenses/metrics-jakarta-servlets-4.2.26.jar.sha1 deleted file mode 100644 index 023b4931bb15..000000000000 --- a/solr/licenses/metrics-jakarta-servlets-4.2.26.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da590186279c40e3187b50ca05ad9a6c4503db33 diff --git a/solr/licenses/metrics-json-4.2.26.jar.sha1 b/solr/licenses/metrics-json-4.2.26.jar.sha1 deleted file mode 100644 index 1a1b3db2247b..000000000000 --- a/solr/licenses/metrics-json-4.2.26.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1eac3853bb964647b38d7e1d7b66e515443437d6 diff --git a/solr/licenses/metrics-jvm-4.2.26.jar.sha1 b/solr/licenses/metrics-jvm-4.2.26.jar.sha1 deleted file mode 100644 index b6e5eda90439..000000000000 --- a/solr/licenses/metrics-jvm-4.2.26.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a7df0386df8c8938dea6cfacdc67972b9fd1a01e diff --git a/solr/licenses/profiler-1.1.1.jar.sha1 b/solr/licenses/profiler-1.1.1.jar.sha1 deleted file mode 100644 index 329a0deb7d36..000000000000 --- a/solr/licenses/profiler-1.1.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c92a582728b09b47de38b1b97bd2e5b0c8cd553c diff --git a/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 new file mode 100644 index 000000000000..954a48dcf646 --- /dev/null +++ b/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 @@ -0,0 +1 @@ +4588ed1d1bbdcebb2663ccef1162f4cc4c5eb9fb diff --git a/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 new file mode 100644 index 000000000000..aefaedc39a60 --- /dev/null +++ b/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 @@ -0,0 +1 @@ +b25424c069a44fce42b55626512718d49c9dcc5d diff --git a/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 new file mode 100644 index 000000000000..d087383b5789 --- /dev/null +++ b/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 @@ -0,0 +1 @@ +2c719d8a7481f4dcc797c7450863bbb65a664e0d diff --git a/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 new file mode 100644 index 000000000000..e0b7056acce8 --- /dev/null +++ b/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 @@ -0,0 +1 @@ +3550df21bc7be5358bbe8b76e1430309dafb9c26 diff --git a/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 new file mode 100644 index 000000000000..c593438438de --- /dev/null +++ b/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 @@ -0,0 +1 @@ +5a18603c6281609b0e9e1e7f21d647c6ce994adb diff --git a/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 new file mode 100644 index 000000000000..942ecf85c06f --- /dev/null +++ b/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 @@ -0,0 +1 @@ +daf7dbdb867221e418be0771663eae50a9199228 diff --git a/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 new file mode 100644 index 000000000000..f308f25736cd --- /dev/null +++ b/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 @@ -0,0 +1 @@ +169f3dae0c95b4154696407de0e1213373f074e1 diff --git a/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 new file mode 100644 index 000000000000..c28f7197ac4f --- /dev/null +++ b/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 @@ -0,0 +1 @@ +25c2d488b084a9757a7b5e23cbce8dc812147e54 diff --git a/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 new file mode 100644 index 000000000000..45a83f1dda19 --- /dev/null +++ b/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 @@ -0,0 +1 @@ +393255f85063e08684131ac75af5e65aef070a60 diff --git a/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java b/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java index 0f45c8916bd4..00ca4442d3da 100644 --- a/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java +++ b/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java @@ -45,6 +45,11 @@ public boolean reject(Thread t) { return true; } + // Prometheus Scheduler doesn't provide any method to shut down its worker threads + if (t.isDaemon()) { + return true; + } + return false; } } From a42205a95e6ab936d0034d63b7d83876ddfaf978 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 19 Jan 2026 15:08:07 +0100 Subject: [PATCH 03/15] Add unit test, fix some bugs. --- solr/cross-dc-manager/build.gradle | 1 + .../crossdc/manager/consumer/Consumer.java | 6 ++- .../manager/consumer/PrometheusMetrics.java | 4 +- .../manager/SolrAndKafkaIntegrationTest.java | 42 ++++++++++++++++++- 4 files changed, 48 insertions(+), 5 deletions(-) diff --git a/solr/cross-dc-manager/build.gradle b/solr/cross-dc-manager/build.gradle index 7c86b619dc7a..5aea2d3fe4cc 100644 --- a/solr/cross-dc-manager/build.gradle +++ b/solr/cross-dc-manager/build.gradle @@ -51,6 +51,7 @@ dependencies { testImplementation project(':solr:solrj-jetty') testImplementation libs.apache.lucene.testframework testImplementation libs.carrotsearch.randomizedtesting.runner + testImplementation libs.commonsio.commonsio testImplementation libs.junit.junit // The explicit dependency on bytebuddy is required for Java 25 support // Once Mockito upgrades its dependency on ByteBuddy to 1.16.1, we should diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java index 12d13fdbe835..64bb61255232 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java @@ -26,6 +26,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; + +import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.util.ExecutorUtil; @@ -94,8 +96,8 @@ public void start(Map properties) { context.setContextPath("/"); server.setHandler(context); - // context.addServlet(ThreadDumpServlet.class, "/threads/*"); - // context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); + context.addServlet(ThreadDumpServlet.class, "/threads/*"); + context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); for (ServletMapping mapping : context.getServletHandler().getServletMappings()) { if (log.isInfoEnabled()) { log.info(" - {}", mapping.getPathSpecs()[0]); diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java index 873995a53696..ac8532c2fc7f 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java @@ -36,7 +36,7 @@ public class PrometheusMetrics implements ConsumerMetrics { protected Histogram outputFirstAttemptHistogram; public PrometheusMetrics() { - register(new PrometheusRegistry()); + register(PrometheusRegistry.defaultRegistry); } protected void register(PrometheusRegistry registry) { @@ -116,7 +116,7 @@ public void incrementOutputCounter(String type, String result) { @Override public void incrementOutputCounter(String type, String result, int delta) { - input.labelValues(type, result).inc(delta); + output.labelValues(type, result).inc(delta); } @Override diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java index 3770884b372a..7196aa28a784 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java @@ -22,9 +22,11 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering; +import java.io.InputStream; import java.lang.invoke.MethodHandles; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -32,6 +34,7 @@ import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import org.apache.commons.io.IOUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -40,15 +43,20 @@ import org.apache.lucene.tests.util.QuickPatchThreadsFilter; import org.apache.solr.SolrIgnoredThreadsFilter; import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.jetty.HttpJettySolrClient; import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.GenericSolrRequest; import org.apache.solr.client.solrj.request.SolrQuery; import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.client.solrj.response.InputStreamResponseParser; import org.apache.solr.client.solrj.response.QueryResponse; import org.apache.solr.cloud.MiniSolrCloudCluster; import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.util.ExecutorUtil; +import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.ObjectReleaseTracker; import org.apache.solr.common.util.SolrNamedThreadFactory; import org.apache.solr.crossdc.common.KafkaCrossDcConf; @@ -101,7 +109,6 @@ public void beforeSolrAndKafkaIntegrationTest() throws Exception { uceh = Thread.getDefaultUncaughtExceptionHandler(); Thread.setDefaultUncaughtExceptionHandler( (t, e) -> log.error("Uncaught exception in thread {}", t, e)); - System.setProperty(PORT, "-1"); consumer = new Consumer(); Properties config = new Properties(); @@ -332,6 +339,39 @@ public void testParallelUpdatesToCluster2() throws Exception { assertCluster2EventuallyHasDocs(COLLECTION, "*:*", 5000); } + @Test + @SuppressWarnings({"unchecked"}) + public void testMetrics() throws Exception { + CloudSolrClient client = solrCluster1.getSolrClient(); + SolrInputDocument doc = new SolrInputDocument(); + doc.addField("id", String.valueOf(new Date().getTime())); + doc.addField("text", "some test"); + + client.add(COLLECTION, doc); + + client.commit(COLLECTION); + + System.out.println("Sent producer record"); + + assertCluster2EventuallyHasDocs(COLLECTION, "*:*", 1); + + String baseUrl = "http://localhost:" + KafkaCrossDcConf.DEFAULT_PORT; + HttpJettySolrClient httpJettySolrClient = + new HttpJettySolrClient.Builder(baseUrl).useHttp1_1(true).build(); + try { + GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/metrics"); + req.setResponseParser(new InputStreamResponseParser("test/plain")); + NamedList rsp = httpJettySolrClient.request(req); + String content = + IOUtils.toString( + (InputStream) rsp.get(InputStreamResponseParser.STREAM_KEY), StandardCharsets.UTF_8); + assertTrue(content.contains("consumer_output_total{result=\"handled\",type=\"UPDATE\"} 1.0")); + } finally { + httpJettySolrClient.close(); + client.close(); + } + } + private void assertCluster2EventuallyHasDocs(String collection, String query, int expectedNumDocs) throws Exception { assertClusterEventuallyHasDocs( From 5eef27145de012db21c59f7c1ea6768bba1ba51a Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 20 Jan 2026 13:25:10 +0100 Subject: [PATCH 04/15] Tidy. --- .../org/apache/solr/crossdc/manager/consumer/Consumer.java | 3 +-- .../solr/crossdc/manager/SolrAndKafkaIntegrationTest.java | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java index 64bb61255232..af8b3e3d9d08 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java @@ -20,14 +20,13 @@ import static org.apache.solr.crossdc.common.KafkaCrossDcConf.TOPIC_NAME; import static org.apache.solr.crossdc.common.KafkaCrossDcConf.ZK_CONNECT_STRING; +import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import java.lang.invoke.MethodHandles; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; - -import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.util.ExecutorUtil; diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java index 7196aa28a784..152c9425e175 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java @@ -18,7 +18,6 @@ import static org.apache.solr.crossdc.common.KafkaCrossDcConf.DEFAULT_MAX_REQUEST_SIZE; import static org.apache.solr.crossdc.common.KafkaCrossDcConf.INDEX_UNMIRRORABLE_DOCS; -import static org.apache.solr.crossdc.common.KafkaCrossDcConf.PORT; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering; From c569ff62d4def7391eb546bfefeca6d809805d47 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 20 Jan 2026 14:12:22 +0100 Subject: [PATCH 05/15] Add license and notice. --- .../prometheus-metrics-LICENSE-ASL.txt | 201 ++++++++++++++++++ solr/licenses/prometheus-metrics-NOTICE.txt | 11 + 2 files changed, 212 insertions(+) create mode 100644 solr/licenses/prometheus-metrics-LICENSE-ASL.txt create mode 100644 solr/licenses/prometheus-metrics-NOTICE.txt diff --git a/solr/licenses/prometheus-metrics-LICENSE-ASL.txt b/solr/licenses/prometheus-metrics-LICENSE-ASL.txt new file mode 100644 index 000000000000..f49a4e16e68b --- /dev/null +++ b/solr/licenses/prometheus-metrics-LICENSE-ASL.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. \ No newline at end of file diff --git a/solr/licenses/prometheus-metrics-NOTICE.txt b/solr/licenses/prometheus-metrics-NOTICE.txt new file mode 100644 index 000000000000..cbd3cd95befd --- /dev/null +++ b/solr/licenses/prometheus-metrics-NOTICE.txt @@ -0,0 +1,11 @@ +Prometheus instrumentation library for JVM applications +Copyright 2012-2015 The Prometheus Authors + +This product includes software developed at +Boxever Ltd. (http://www.boxever.com/). + +This product includes software developed at +SoundCloud Ltd. (http://soundcloud.com/). + +This product includes software developed as part of the +Ocelli project by Netflix Inc. (https://github.com/Netflix/ocelli/). \ No newline at end of file From 82531d591370b6535731962d06a1c228bbf429c6 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 20 Jan 2026 14:41:04 +0100 Subject: [PATCH 06/15] Fix a logging call. --- .../java/org/apache/solr/crossdc/manager/consumer/Util.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index d34e3e0f6402..2c4695704f02 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -46,7 +46,9 @@ public static void logMetrics(PrometheusRegistry registry) { MetricSnapshots snapshots = registry.scrape(); try { ExpositionFormats.init().getPrometheusTextFormatWriter().write(baos, snapshots); - log.info(baos.toString(StandardCharsets.UTF_8)); + if (log.isInfoEnabled()) { + log.info("### Final Consumer metrics:\n{}", baos.toString(StandardCharsets.UTF_8)); + } } catch (IOException e) { log.error("Error while logging metrics", e); } From a7eee83e0a07d6b118d3c91489b44774f9951873 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 20 Jan 2026 16:49:10 +0100 Subject: [PATCH 07/15] Remove unused licenses. --- solr/licenses/metrics-jvm-LICENSE-ASL.txt | 203 ------------- solr/licenses/metrics-jvm-NOTICE.txt | 12 - solr/licenses/profiler-LICENSE-ASL.txt | 345 ---------------------- solr/licenses/profiler-NOTICE.txt | 0 4 files changed, 560 deletions(-) delete mode 100644 solr/licenses/metrics-jvm-LICENSE-ASL.txt delete mode 100644 solr/licenses/metrics-jvm-NOTICE.txt delete mode 100644 solr/licenses/profiler-LICENSE-ASL.txt delete mode 100644 solr/licenses/profiler-NOTICE.txt diff --git a/solr/licenses/metrics-jvm-LICENSE-ASL.txt b/solr/licenses/metrics-jvm-LICENSE-ASL.txt deleted file mode 100644 index ccb320c7daae..000000000000 --- a/solr/licenses/metrics-jvm-LICENSE-ASL.txt +++ /dev/null @@ -1,203 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright 2010-2012 Coda Hale and Yammer, Inc. - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - diff --git a/solr/licenses/metrics-jvm-NOTICE.txt b/solr/licenses/metrics-jvm-NOTICE.txt deleted file mode 100644 index b4c6298472f2..000000000000 --- a/solr/licenses/metrics-jvm-NOTICE.txt +++ /dev/null @@ -1,12 +0,0 @@ -Metrics -Copyright 2010-2013 Coda Hale and Yammer, Inc. - -This product includes software developed by Coda Hale and Yammer, Inc. - -This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, -LongAdder), which was released with the following comments: - - Written by Doug Lea with assistance from members of JCP JSR-166 - Expert Group and released to the public domain, as explained at - http://creativecommons.org/publicdomain/zero/1.0/ - diff --git a/solr/licenses/profiler-LICENSE-ASL.txt b/solr/licenses/profiler-LICENSE-ASL.txt deleted file mode 100644 index a6fe25e5c8d8..000000000000 --- a/solr/licenses/profiler-LICENSE-ASL.txt +++ /dev/null @@ -1,345 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -------------------------------------------------------------------------------- -This project bundles some components that are also licensed under the Apache -License Version 2.0: - -audience-annotations-0.12.0 -caffeine-2.9.3 -commons-beanutils-1.9.4 -commons-cli-1.4 -commons-collections-3.2.2 -commons-digester-2.1 -commons-io-2.11.0 -commons-lang3-3.12.0 -commons-logging-1.2 -commons-validator-1.7 -error_prone_annotations-2.10.0 -jackson-annotations-2.16.2 -jackson-core-2.16.2 -jackson-databind-2.16.2 -jackson-dataformat-csv-2.16.2 -jackson-datatype-jdk8-2.16.2 -jackson-jaxrs-base-2.16.2 -jackson-jaxrs-json-provider-2.16.2 -jackson-module-afterburner-2.16.2 -jackson-module-jaxb-annotations-2.16.2 -jackson-module-scala_2.13-2.16.2 -jackson-module-scala_2.12-2.16.2 -jakarta.validation-api-2.0.2 -javassist-3.29.2-GA -jetty-client-9.4.54.v20240208 -jetty-continuation-9.4.54.v20240208 -jetty-http-9.4.54.v20240208 -jetty-io-9.4.54.v20240208 -jetty-security-9.4.54.v20240208 -jetty-server-9.4.54.v20240208 -jetty-servlet-9.4.54.v20240208 -jetty-servlets-9.4.54.v20240208 -jetty-util-9.4.54.v20240208 -jetty-util-ajax-9.4.54.v20240208 -jose4j-0.9.4 -lz4-java-1.8.0 -maven-artifact-3.9.6 -metrics-core-4.1.12.1 -metrics-core-2.2.0 -netty-buffer-4.1.111.Final -netty-codec-4.1.111.Final -netty-common-4.1.111.Final -netty-handler-4.1.111.Final -netty-resolver-4.1.111.Final -netty-transport-4.1.111.Final -netty-transport-classes-epoll-4.1.111.Final -netty-transport-native-epoll-4.1.111.Final -netty-transport-native-unix-common-4.1.111.Final -opentelemetry-proto-1.0.0-alpha -plexus-utils-3.5.1 -reflections-0.10.2 -reload4j-1.2.25 -rocksdbjni-7.9.2 -scala-collection-compat_2.12-2.10.0 -scala-collection-compat_2.13-2.10.0 -scala-library-2.12.19 -scala-library-2.13.14 -scala-logging_2.12-3.9.5 -scala-logging_2.13-3.9.5 -scala-reflect-2.12.19 -scala-reflect-2.13.14 -scala-java8-compat_2.12-1.0.2 -scala-java8-compat_2.13-1.0.2 -snappy-java-1.1.10.5 -swagger-annotations-2.2.8 -zookeeper-3.8.4 -zookeeper-jute-3.8.4 - -=============================================================================== -This product bundles various third-party components under other open source -licenses. This section summarizes those components and their licenses. -See licenses/ for text of these licenses. - ---------------------------------------- -Eclipse Distribution License - v 1.0 -see: licenses/eclipse-distribution-license-1.0 - -jakarta.activation-api-1.2.2 -jakarta.xml.bind-api-2.3.3 - ---------------------------------------- -Eclipse Public License - v 2.0 -see: licenses/eclipse-public-license-2.0 - -jakarta.annotation-api-1.3.5 -jakarta.ws.rs-api-2.1.6 -hk2-api-2.6.1 -hk2-locator-2.6.1 -hk2-utils-2.6.1 -osgi-resource-locator-1.0.3 -aopalliance-repackaged-2.6.1 -jakarta.inject-2.6.1 -jersey-client-2.39.1 -jersey-common-2.39.1 -jersey-container-servlet-2.39.1 -jersey-container-servlet-core-2.39.1 -jersey-hk2-2.39.1 -jersey-server-2.39.1 - ---------------------------------------- -CDDL 1.1 + GPLv2 with classpath exception -see: licenses/CDDL+GPL-1.1 - -javax.activation-api-1.2.0 -javax.annotation-api-1.3.2 -javax.servlet-api-3.1.0 -javax.ws.rs-api-2.1.1 -jaxb-api-2.3.1 -activation-1.1.1 - ---------------------------------------- -MIT License - -argparse4j-0.7.0, see: licenses/argparse-MIT -checker-qual-3.19.0, see: licenses/checker-qual-MIT -jopt-simple-5.0.4, see: licenses/jopt-simple-MIT -slf4j-api-1.7.36, see: licenses/slf4j-MIT -slf4j-reload4j-1.7.36, see: licenses/slf4j-MIT -pcollections-4.0.1, see: licenses/pcollections-MIT - ---------------------------------------- -BSD 2-Clause - -zstd-jni-1.5.6-3 see: licenses/zstd-jni-BSD-2-clause - ---------------------------------------- -BSD 3-Clause - -jline-3.25.1, see: licenses/jline-BSD-3-clause -jsr305-3.0.2, see: licenses/jsr305-BSD-3-clause -paranamer-2.8, see: licenses/paranamer-BSD-3-clause -protobuf-java-3.23.4, see: licenses/protobuf-java-BSD-3-clause - ---------------------------------------- -Do What The F*ck You Want To Public License -see: licenses/DWTFYWTPL - -reflections-0.10.2 \ No newline at end of file diff --git a/solr/licenses/profiler-NOTICE.txt b/solr/licenses/profiler-NOTICE.txt deleted file mode 100644 index e69de29bb2d1..000000000000 From 13a8a53ccf1205a51765daf8e4920d4884ef97d1 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Thu, 22 Jan 2026 18:47:26 +0100 Subject: [PATCH 08/15] Use OTEL API instead of Prometheus API. --- solr/cross-dc-manager/build.gradle | 17 +- solr/cross-dc-manager/gradle.lockfile | 56 +++-- .../crossdc/manager/consumer/Consumer.java | 17 +- .../manager/consumer/ConsumerMetrics.java | 50 +++++ .../consumer/KafkaCrossDcConsumer.java | 12 +- .../manager/consumer/MetricsServlet.java | 69 ++++++ .../crossdc/manager/consumer/OtelMetrics.java | 184 ++++++++++++++++ .../manager/consumer/PrometheusMetrics.java | 158 -------------- .../solr/crossdc/manager/consumer/Util.java | 35 +-- .../manager/DeleteByQueryToIdTest.java | 5 +- .../manager/SimpleSolrIntegrationTest.java | 4 +- .../manager/SolrAndKafkaIntegrationTest.java | 5 +- .../consumer/KafkaCrossDcConsumerTest.java | 8 +- .../SolrMessageProcessorTest.java | 5 +- .../TestMessageProcessor.java | 4 +- ...ive-boringssl-static-2.0.73.Final.jar.sha1 | 1 + ...tty-tcnative-classes-2.0.73.Final.jar.sha1 | 1 + .../prometheus-metrics-LICENSE-ASL.txt | 201 ------------------ solr/licenses/prometheus-metrics-NOTICE.txt | 11 - .../prometheus-metrics-config-1.1.0.jar.sha1 | 1 - .../prometheus-metrics-core-1.1.0.jar.sha1 | 1 - ...eus-metrics-exporter-common-1.1.0.jar.sha1 | 1 - ...cs-exporter-servlet-jakarta-1.1.0.jar.sha1 | 1 - ...eus-metrics-shaded-protobuf-1.1.0.jar.sha1 | 1 - ...theus-metrics-tracer-common-1.1.0.jar.sha1 | 1 - ...-metrics-tracer-initializer-1.1.0.jar.sha1 | 1 - ...metheus-metrics-tracer-otel-1.1.0.jar.sha1 | 1 - ...s-metrics-tracer-otel-agent-1.1.0.jar.sha1 | 1 - .../SolrKafkaTestsIgnoredThreadsFilter.java | 5 - 29 files changed, 411 insertions(+), 446 deletions(-) create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/MetricsServlet.java create mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java delete mode 100644 solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java create mode 100644 solr/licenses/netty-tcnative-boringssl-static-2.0.73.Final.jar.sha1 create mode 100644 solr/licenses/netty-tcnative-classes-2.0.73.Final.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-LICENSE-ASL.txt delete mode 100644 solr/licenses/prometheus-metrics-NOTICE.txt delete mode 100644 solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 delete mode 100644 solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 diff --git a/solr/cross-dc-manager/build.gradle b/solr/cross-dc-manager/build.gradle index 5aea2d3fe4cc..e101a10279d1 100644 --- a/solr/cross-dc-manager/build.gradle +++ b/solr/cross-dc-manager/build.gradle @@ -27,11 +27,22 @@ dependencies { implementation project(':solr:solrj') implementation project(':solr:solrj-zookeeper') implementation project(':solr:modules:cross-dc') + implementation project(':solr:modules:opentelemetry') - implementation libs.prometheus.metrics.core - implementation libs.prometheus.metrics.exporter.servlet.jakarta - implementation libs.prometheus.metrics.expositionformats + implementation platform(libs.opentelemetry.bom) + implementation libs.opentelemetry.api + implementation libs.opentelemetry.sdk + implementation libs.opentelemetry.sdk.metrics + implementation libs.opentelemetry.sdk.extension.autoconfigure + implementation libs.opentelemetry.exporter.otlp + implementation(libs.opentelemetry.exporter.prometheus) { + transitive = false + } implementation libs.prometheus.metrics.model + implementation(libs.prometheus.metrics.expositionformats, { + exclude group: "io.prometheus", module: "prometheus-metrics-shaded-protobuf" + exclude group: "io.prometheus", module: "prometheus-metrics-config" + }) implementation libs.eclipse.jetty.server implementation libs.eclipse.jetty.ee10.servlet implementation libs.slf4j.api diff --git a/solr/cross-dc-manager/gradle.lockfile b/solr/cross-dc-manager/gradle.lockfile index 6fbf5d69c42d..e46fd5214864 100644 --- a/solr/cross-dc-manager/gradle.lockfile +++ b/solr/cross-dc-manager/gradle.lockfile @@ -17,6 +17,8 @@ com.fasterxml.woodstox:woodstox-core:7.0.0=jarValidation,runtimeClasspath,runtim com.github.ben-manes.caffeine:caffeine:3.2.2=annotationProcessor,errorprone,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testAnnotationProcessor,testRuntimeClasspath com.github.kevinstern:software-and-algorithms:1.0=annotationProcessor,errorprone,testAnnotationProcessor com.github.luben:zstd-jni:1.5.6-4=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +com.google.android:annotations:4.1.1.4=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +com.google.api.grpc:proto-google-common-protos:2.61.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath com.google.auto.service:auto-service-annotations:1.0.1=annotationProcessor,errorprone,testAnnotationProcessor com.google.auto.value:auto-value-annotations:1.11.0=annotationProcessor,errorprone,testAnnotationProcessor com.google.auto:auto-common:1.2.2=annotationProcessor,errorprone,testAnnotationProcessor @@ -35,6 +37,10 @@ com.google.protobuf:protobuf-java:3.25.8=annotationProcessor,errorprone,jarValid com.j256.simplemagic:simplemagic:1.17=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.jayway.jsonpath:json-path:2.9.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.lmax:disruptor:3.4.4=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +com.squareup.okhttp3:okhttp-jvm:5.3.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +com.squareup.okhttp3:okhttp:5.3.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +com.squareup.okio:okio-jvm:3.16.2=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +com.squareup.okio:okio:3.16.2=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath com.tdunning:t-digest:3.3=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath com.thoughtworks.paranamer:paranamer:2.8.3=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath com.typesafe.scala-logging:scala-logging_2.13:3.9.5=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath @@ -52,13 +58,29 @@ io.dropwizard.metrics:metrics-jetty12-ee10:4.2.26=jarValidation,testRuntimeClass io.dropwizard.metrics:metrics-jetty12:4.2.26=jarValidation,testRuntimeClasspath io.github.eisop:dataflow-errorprone:3.41.0-eisop1=annotationProcessor,errorprone,testAnnotationProcessor io.github.java-diff-utils:java-diff-utils:4.12=annotationProcessor,errorprone,testAnnotationProcessor +io.grpc:grpc-api:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-context:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-core:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-netty:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-protobuf-lite:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-protobuf:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-stub:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.grpc:grpc-util:1.65.1=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.netty:netty-bom:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.netty:netty-buffer:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.netty:netty-codec-base:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.netty:netty-codec-compression:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.netty:netty-codec-http2:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.netty:netty-codec-http:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.netty:netty-codec-socks:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.netty:netty-common:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.netty:netty-handler-proxy:4.2.6.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.netty:netty-handler:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.netty:netty-resolver:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.netty:netty-tcnative-boringssl-static:2.0.70.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.netty:netty-tcnative-classes:2.0.70.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.netty:netty-tcnative-boringssl-static:2.0.70.Final=compileClasspath,solrPlatformLibs,testCompileClasspath +io.netty:netty-tcnative-boringssl-static:2.0.73.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.netty:netty-tcnative-classes:2.0.70.Final=compileClasspath,solrPlatformLibs,testCompileClasspath +io.netty:netty-tcnative-classes:2.0.73.Final=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.netty:netty-transport-classes-epoll:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.netty:netty-transport-native-epoll:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.netty:netty-transport-native-unix-common:4.2.6.Final=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath @@ -70,24 +92,24 @@ io.opentelemetry.instrumentation:opentelemetry-runtime-telemetry-java8:2.22.0-al io.opentelemetry.semconv:opentelemetry-semconv:1.37.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-api-incubator:1.56.0-alpha=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-api:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-bom:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath io.opentelemetry:opentelemetry-common:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.opentelemetry:opentelemetry-context:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-exporter-prometheus:1.56.0-alpha=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-common:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-metrics:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-trace:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath -io.prometheus:prometheus-metrics-config:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-core:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-exporter-common:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-exporter-servlet-jakarta:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-common:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-otlp-common:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-otlp:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-prometheus:1.56.0-alpha=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-sender-okhttp:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-common:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-extension-autoconfigure:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-logs:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-metrics:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-trace:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.perfmark:perfmark-api:0.27.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.prometheus:prometheus-metrics-exposition-formats:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.prometheus:prometheus-metrics-model:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-shaded-protobuf:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-tracer-common:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-tracer-initializer:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-tracer-otel-agent:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-tracer-otel:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath io.sgr:s2-geometry-library-java:1.0.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.swagger.core.v3:swagger-annotations-jakarta:2.2.22=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath jakarta.annotation:jakarta.annotation-api:2.1.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath @@ -195,6 +217,8 @@ org.glassfish.jersey.inject:jersey-hk2:3.1.11=jarValidation,runtimeClasspath,run org.glassfish.jersey.media:jersey-media-json-jackson:3.1.11=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath org.hamcrest:hamcrest:3.0=jarValidation,testCompileClasspath,testRuntimeClasspath org.javassist:javassist:3.30.2-GA=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +org.jetbrains.kotlin:kotlin-stdlib:2.2.21=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +org.jetbrains:annotations:26.0.2=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath org.jspecify:jspecify:1.0.0=annotationProcessor,compileClasspath,errorprone,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testAnnotationProcessor,testCompileClasspath,testRuntimeClasspath org.junit.jupiter:junit-jupiter-api:5.6.2=jarValidation,testRuntimeClasspath org.junit.platform:junit-platform-commons:1.6.2=jarValidation,testRuntimeClasspath diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java index af8b3e3d9d08..ca3d2a16532f 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Consumer.java @@ -17,10 +17,8 @@ package org.apache.solr.crossdc.manager.consumer; import static org.apache.solr.crossdc.common.KafkaCrossDcConf.PORT; -import static org.apache.solr.crossdc.common.KafkaCrossDcConf.TOPIC_NAME; import static org.apache.solr.crossdc.common.KafkaCrossDcConf.ZK_CONNECT_STRING; -import io.prometheus.metrics.exporter.servlet.jakarta.PrometheusMetricsServlet; import java.lang.invoke.MethodHandles; import java.util.HashMap; import java.util.Map; @@ -80,11 +78,10 @@ public void start(Map properties) { ConfUtil.verifyProperties(properties); - String bootstrapServers = (String) properties.get(KafkaCrossDcConf.BOOTSTRAP_SERVERS); - String topicName = (String) properties.get(TOPIC_NAME); + OtelMetrics metrics = new OtelMetrics(); KafkaCrossDcConf conf = new KafkaCrossDcConf(properties); - crossDcConsumer = getCrossDcConsumer(conf, startLatch); + crossDcConsumer = getCrossDcConsumer(conf, metrics, startLatch); // jetty endpoint for /metrics int port = conf.getInt(PORT); @@ -96,7 +93,10 @@ public void start(Map properties) { server.setHandler(context); context.addServlet(ThreadDumpServlet.class, "/threads/*"); - context.addServlet(PrometheusMetricsServlet.class, "/metrics/*"); + context.setAttribute( + MetricsServlet.SOLR_METRICS_MANAGER_ATTRIBUTE, metrics.getMetricManager()); + context.addServlet(MetricsServlet.class, "/metrics/*"); + for (ServletMapping mapping : context.getServletHandler().getServletMappings()) { if (log.isInfoEnabled()) { log.info(" - {}", mapping.getPathSpecs()[0]); @@ -147,8 +147,9 @@ public void run() { } } - protected CrossDcConsumer getCrossDcConsumer(KafkaCrossDcConf conf, CountDownLatch startLatch) { - return new KafkaCrossDcConsumer(conf, startLatch); + protected CrossDcConsumer getCrossDcConsumer( + KafkaCrossDcConf conf, ConsumerMetrics metrics, CountDownLatch startLatch) { + return new KafkaCrossDcConsumer(conf, metrics, startLatch); } public static void main(String[] args) { diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java index 010dee9a847f..32952f91987e 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.solr.crossdc.manager.consumer; import org.apache.solr.client.solrj.SolrRequest; @@ -5,6 +21,40 @@ public interface ConsumerMetrics { + ConsumerMetrics NOOP = + new ConsumerMetrics() { + @Override + public void incrementCollapsedCounter() {} + + @Override + public void incrementInputCounter(String type, String subType) {} + + @Override + public void incrementInputCounter(String type, String subType, int delta) {} + + @Override + public void incrementOutputCounter(String type, String result) {} + + @Override + public void incrementOutputCounter(String type, String result, int delta) {} + + @Override + public void recordOutputBatchSize( + MirroredSolrRequest.Type type, SolrRequest solrRequest) {} + + @Override + public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) {} + + @Override + public void recordOutputFirstAttemptSize( + MirroredSolrRequest.Type type, long firstAttemptTimeNs) {} + + @Override + public ConsumerTimer startOutputTimeTimer(String requestType) { + return () -> 0; + } + }; + interface ConsumerTimer { double observeDuration(); diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index 20419cf4bfab..61bd6201a512 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -81,7 +81,7 @@ public class KafkaCrossDcConsumer extends Consumer.CrossDcConsumer { private final CrossDcConf.CollapseUpdates collapseUpdates; private final int maxCollapseRecords; private final SolrMessageProcessor messageProcessor; - protected ConsumerMetrics metrics; + protected final ConsumerMetrics metrics; protected SolrClientSupplier solrClientSupplier; @@ -159,9 +159,9 @@ public CloudSolrClient get() { * @param conf The Kafka consumer configuration * @param startLatch To inform the caller when the Consumer has started */ - public KafkaCrossDcConsumer(KafkaCrossDcConf conf, CountDownLatch startLatch) { - this.metrics = new PrometheusMetrics(); - + public KafkaCrossDcConsumer( + KafkaCrossDcConf conf, ConsumerMetrics metrics, CountDownLatch startLatch) { + this.metrics = metrics; this.topicNames = conf.get(KafkaCrossDcConf.TOPIC_NAME).split(","); this.maxAttempts = conf.getInt(KafkaCrossDcConf.MAX_ATTEMPTS); this.collapseUpdates = @@ -586,8 +586,8 @@ public final void shutdown() { } catch (Exception e) { log.warn("Exception closing Solr client on shutdown", e); } finally { - if (metrics instanceof PrometheusMetrics) { - Util.logMetrics(((PrometheusMetrics) metrics).getRegistry()); + if (metrics instanceof OtelMetrics) { + Util.logMetrics(((OtelMetrics) metrics).getMetricManager()); } } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/MetricsServlet.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/MetricsServlet.java new file mode 100644 index 000000000000..9cb1ecae4829 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/MetricsServlet.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.solr.crossdc.manager.consumer; + +import jakarta.servlet.ServletException; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; +import java.io.IOException; +import org.apache.solr.handler.admin.MetricsHandler; +import org.apache.solr.metrics.SolrMetricManager; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.response.PrometheusResponseWriter; +import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.servlet.ServletUtils; +import org.apache.solr.servlet.SolrRequestParsers; + +/** + * Helper servlet that exports collected metrics in Prometheus format using {@link MetricsHandler}. + */ +public class MetricsServlet extends HttpServlet { + private static final long serialVersionUID = -2881083456665410780L; + + public static final String SOLR_METRICS_MANAGER_ATTRIBUTE = + MetricsServlet.class.getName() + ".solrMetricsManager"; + + private SolrMetricManager metricManager; + private MetricsHandler metricsHandler; + private static final PrometheusResponseWriter writer = new PrometheusResponseWriter(); + + @Override + public void init() throws ServletException { + metricManager = + (SolrMetricManager) getServletContext().getAttribute(SOLR_METRICS_MANAGER_ATTRIBUTE); + metricsHandler = new MetricsHandler(metricManager); + } + + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) + throws ServletException, IOException { + try { + final SolrQueryResponse solrQueryResponse = new SolrQueryResponse(); + final String path = ServletUtils.getPathAfterContext(req); + SolrQueryRequest solrQueryRequest = SolrRequestParsers.DEFAULT.parse(null, path, req); + metricsHandler.handleRequestBody(solrQueryRequest, solrQueryResponse); + resp.setStatus(HttpServletResponse.SC_OK); + final String contentType = writer.getContentType(solrQueryRequest, solrQueryResponse); + resp.setContentType(contentType); + resp.setHeader("Cache-Control", "must-revalidate,no-cache,no-store"); + writer.write(resp.getOutputStream(), solrQueryRequest, solrQueryResponse, contentType); + } catch (Exception e) { + throw new ServletException(e); + } + } +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java new file mode 100644 index 000000000000..000d025c9553 --- /dev/null +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.solr.crossdc.manager.consumer; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongHistogram; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.crossdc.common.MirroredSolrRequest; +import org.apache.solr.metrics.SolrMetricManager; +import org.apache.solr.metrics.SolrMetricsContext; +import org.apache.solr.metrics.otel.OtelUnit; +import org.apache.solr.opentelemetry.OtlpExporterFactory; +import org.apache.solr.util.RTimer; + +public class OtelMetrics implements ConsumerMetrics { + + public static final String REGISTRY = "crossdc.consumer.registry"; + public static final String NAME_PREFIX = "crossdc_consumer_"; + public static final String ATTR_TYPE = "type"; + public static final String ATTR_SUBTYPE = "subtype"; + public static final String ATTR_RESULT = "result"; + + protected final Map attributesCache = new ConcurrentHashMap<>(); + + protected SolrMetricManager metricManager; + + protected LongCounter input; + protected LongCounter collapsed; + protected LongCounter output; + protected LongHistogram outputBatchSizeHistogram; + protected LongHistogram outputTimeHistogram; + protected LongHistogram outputBackoffHistogram; + protected LongHistogram outputFirstAttemptHistogram; + + public OtelMetrics() { + register(REGISTRY); + } + + protected void register(String scope) { + this.metricManager = new SolrMetricManager(new OtlpExporterFactory().getExporter()); + SolrMetricsContext metricsContext = new SolrMetricsContext(metricManager, scope); + + input = + metricsContext.longCounter(NAME_PREFIX + "input_total", "Total number of input messages"); + + collapsed = + metricsContext.longCounter( + NAME_PREFIX + "collapsed_total", "Total number of collapsed messages"); + + output = + metricsContext.longCounter(NAME_PREFIX + "output_total", "Total number of output requests"); + + outputBatchSizeHistogram = + metricsContext.longHistogram( + NAME_PREFIX + "output_batch_size", "Histogram of output batch sizes"); + + outputBackoffHistogram = + metricsContext.longHistogram( + NAME_PREFIX + "output_backoff_size", "Histogram of output backoff sleep times"); + + outputTimeHistogram = + metricsContext.longHistogram( + NAME_PREFIX + "output_time", + "Histogram of output request times", + OtelUnit.MILLISECONDS); + + outputFirstAttemptHistogram = + metricsContext.longHistogram( + NAME_PREFIX + "output_first_attempt_time", + "Histogram of first attempt request times", + OtelUnit.MILLISECONDS); + } + + protected static final String KEY_SEPARATOR = "#"; + + protected Attributes attr(String key1, String value1) { + String key = key1 + KEY_SEPARATOR + value1; + return attributesCache.computeIfAbsent( + key, k -> Attributes.builder().put(key1, value1).build()); + } + + protected Attributes attr(String key1, String value1, String key2, String value2) { + String key = key1 + KEY_SEPARATOR + value1 + KEY_SEPARATOR + key2 + KEY_SEPARATOR + value2; + return attributesCache.computeIfAbsent( + key, k -> Attributes.builder().put(key1, value1).put(key2, value2).build()); + } + + public SolrMetricManager getMetricManager() { + return metricManager; + } + + @Override + public void incrementCollapsedCounter() { + collapsed.add(1L); + } + + @Override + public void incrementInputCounter(String type, String subType) { + incrementInputCounter(type, subType, 1); + } + + @Override + public void incrementInputCounter(String type, String subType, int delta) { + input.add(delta, attr("type", type, "subtype", subType)); + } + + @Override + public void incrementOutputCounter(String type, String result) { + incrementOutputCounter(type, result, 1); + } + + @Override + public void incrementOutputCounter(String type, String result, int delta) { + output.add(delta, attr("type", type, "result", result)); + } + + @Override + public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { + if (type != MirroredSolrRequest.Type.UPDATE) { + outputBatchSizeHistogram.record( + 1, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, solrRequest.getPath())); + return; + } + UpdateRequest req = (UpdateRequest) solrRequest; + int addCount = req.getDocuments() == null ? 0 : req.getDocuments().size(); + int dbiCount = req.getDeleteById() == null ? 0 : req.getDeleteById().size(); + int dbqCount = req.getDeleteQuery() == null ? 0 : req.getDeleteQuery().size(); + if (addCount > 0) { + outputBatchSizeHistogram.record(addCount, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, "add")); + } + if (dbiCount > 0) { + outputBatchSizeHistogram.record( + dbiCount, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, "delete_by_id")); + } + if (dbqCount > 0) { + outputBatchSizeHistogram.record( + dbiCount, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, "delete_by_query")); + } + } + + @Override + public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { + outputBackoffHistogram.record(backoffTimeMs, attr(ATTR_TYPE, type.name())); + } + + @Override + public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { + outputFirstAttemptHistogram.record(firstAttemptTimeNs, attr(ATTR_TYPE, type.name())); + } + + @Override + public ConsumerTimer startOutputTimeTimer(final String requestType) { + final RTimer timer = + new RTimer(TimeUnit.MILLISECONDS) { + @Override + public double stop() { + double elapsedTime = super.stop(); + outputTimeHistogram.record( + Double.valueOf(elapsedTime).longValue(), attr(ATTR_TYPE, requestType)); + return elapsedTime; + } + }; + return () -> timer.stop(); + } +} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java deleted file mode 100644 index ac8532c2fc7f..000000000000 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/PrometheusMetrics.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.solr.crossdc.manager.consumer; - -import io.prometheus.metrics.core.datapoints.Timer; -import io.prometheus.metrics.core.metrics.Counter; -import io.prometheus.metrics.core.metrics.Histogram; -import io.prometheus.metrics.model.registry.PrometheusRegistry; -import org.apache.solr.client.solrj.SolrRequest; -import org.apache.solr.client.solrj.request.UpdateRequest; -import org.apache.solr.crossdc.common.MirroredSolrRequest; - -public class PrometheusMetrics implements ConsumerMetrics { - - protected PrometheusRegistry registry; - protected Counter input; - protected Counter collapsed; - protected Counter output; - protected Histogram outputBatchSizeHistogram; - protected Histogram outputTimeHistogram; - protected Histogram outputBackoffHistogram; - protected Histogram outputFirstAttemptHistogram; - - public PrometheusMetrics() { - register(PrometheusRegistry.defaultRegistry); - } - - protected void register(PrometheusRegistry registry) { - this.registry = registry; - input = - Counter.builder() - .name("consumer_input_total") - .help("Total number of input messages") - .labelNames("type", "subtype") - .register(registry); - - collapsed = - Counter.builder() - .name("consumer_collapsed_total") - .help("Total number of collapsed messages") - .register(registry); - - output = - Counter.builder() - .name("consumer_output_total") - .help("Total number of output requests") - .labelNames("type", "result") - .register(registry); - - outputBatchSizeHistogram = - Histogram.builder() - .name("consumer_output_batch_size_histogram") - .help("Histogram of output batch sizes") - .labelNames("type", "subtype") - .register(registry); - - outputBackoffHistogram = - Histogram.builder() - .name("consumer_output_backoff_histogram") - .help("Histogram of output backoff sleep times") - .labelNames("type") - .register(registry); - - outputTimeHistogram = - Histogram.builder() - .name("consumer_output_time_histogram") - .help("Histogram of output request times") - .labelNames("type") - .register(registry); - - outputFirstAttemptHistogram = - Histogram.builder() - .name("consumer_output_first_attempt_histogram") - .help("Histogram of first attempt request times") - .labelNames("type") - .register(registry); - } - - public PrometheusRegistry getRegistry() { - return registry; - } - - @Override - public void incrementCollapsedCounter() { - collapsed.inc(); - } - - @Override - public void incrementInputCounter(String type, String subType) { - incrementInputCounter(type, subType, 1); - } - - @Override - public void incrementInputCounter(String type, String subType, int delta) { - input.labelValues(type, subType).inc(delta); - } - - @Override - public void incrementOutputCounter(String type, String result) { - incrementOutputCounter(type, result, 1); - } - - @Override - public void incrementOutputCounter(String type, String result, int delta) { - output.labelValues(type, result).inc(delta); - } - - @Override - public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { - if (type != MirroredSolrRequest.Type.UPDATE) { - outputBatchSizeHistogram.labelValues(type.name(), solrRequest.getPath()).observe(1); - return; - } - UpdateRequest req = (UpdateRequest) solrRequest; - int addCount = req.getDocuments() == null ? 0 : req.getDocuments().size(); - int dbiCount = req.getDeleteById() == null ? 0 : req.getDeleteById().size(); - int dbqCount = req.getDeleteQuery() == null ? 0 : req.getDeleteQuery().size(); - if (addCount > 0) { - outputBatchSizeHistogram.labelValues(type.name(), "add").observe(addCount); - } - if (dbiCount > 0) { - outputBatchSizeHistogram.labelValues(type.name(), "dbi").observe(dbiCount); - } - if (dbqCount > 0) { - outputBatchSizeHistogram.labelValues(type.name(), "dbq").observe(dbqCount); - } - } - - @Override - public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { - outputBackoffHistogram.labelValues(type.name()).observe(backoffTimeMs); - } - - @Override - public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { - outputFirstAttemptHistogram.labelValues(type.name()).observe(firstAttemptTimeNs); - } - - @Override - public ConsumerTimer startOutputTimeTimer(String requestType) { - final Timer timer = outputTimeHistogram.labelValues(requestType).startTimer(); - return () -> timer.observeDuration(); - } -} diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index 2c4695704f02..8d49d455a3d4 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -16,13 +16,15 @@ */ package org.apache.solr.crossdc.manager.consumer; -import io.prometheus.metrics.expositionformats.ExpositionFormats; -import io.prometheus.metrics.model.registry.PrometheusRegistry; +// import io.prometheus.metrics.expositionformats.ExpositionFormats; +// import io.prometheus.metrics.model.registry.PrometheusRegistry; +// import io.prometheus.metrics.model.snapshots.MetricSnapshots; +import io.prometheus.metrics.expositionformats.PrometheusTextFormatWriter; +import io.prometheus.metrics.model.snapshots.MetricSnapshot; import io.prometheus.metrics.model.snapshots.MetricSnapshots; import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.lang.invoke.MethodHandles; -import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Properties; import java.util.Set; import org.apache.kafka.clients.admin.AdminClient; @@ -34,24 +36,29 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.solr.crossdc.common.MirroredSolrRequestSerializer; +import org.apache.solr.metrics.SolrMetricManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class Util { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - @SuppressWarnings("rawtypes") - public static void logMetrics(PrometheusRegistry registry) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - MetricSnapshots snapshots = registry.scrape(); + public static void logMetrics(SolrMetricManager metricManager) { + List snapshotList = + metricManager.getPrometheusMetricReaders().values().stream() + .flatMap(r -> r.collect().stream()) + .toList(); + MetricSnapshots snapshots = MetricSnapshots.of(snapshotList.toArray(new MetricSnapshot[0])); + String output; try { - ExpositionFormats.init().getPrometheusTextFormatWriter().write(baos, snapshots); - if (log.isInfoEnabled()) { - log.info("### Final Consumer metrics:\n{}", baos.toString(StandardCharsets.UTF_8)); - } - } catch (IOException e) { - log.error("Error while logging metrics", e); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + new PrometheusTextFormatWriter(false).write(baos, snapshots); + output = baos.toString(); + } catch (Exception e) { + log.error("Error while writing final metrics", e); + output = snapshots.toString(); } + log.info("#### Consumer Metrics: ####\n{}", output); } public static void printKafkaInfo(String host, String groupId) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java index 752d5f643410..1898def635cd 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/DeleteByQueryToIdTest.java @@ -42,6 +42,7 @@ import org.apache.solr.crossdc.common.KafkaCrossDcConf; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.manager.consumer.Consumer; +import org.apache.solr.crossdc.manager.consumer.ConsumerMetrics; import org.apache.solr.crossdc.manager.consumer.KafkaCrossDcConsumer; import org.apache.solr.crossdc.manager.messageprocessor.SolrMessageProcessor; import org.apache.solr.util.SolrKafkaTestsIgnoredThreadsFilter; @@ -160,8 +161,8 @@ public String bootstrapServers() { new Consumer() { @Override protected CrossDcConsumer getCrossDcConsumer( - KafkaCrossDcConf conf, CountDownLatch startLatch) { - return new KafkaCrossDcConsumer(conf, startLatch) { + KafkaCrossDcConf conf, ConsumerMetrics metrics, CountDownLatch startLatch) { + return new KafkaCrossDcConsumer(conf, metrics, startLatch) { @Override protected SolrMessageProcessor createSolrMessageProcessor() { return new SolrMessageProcessor( diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java index a4837ab324f4..d6bf7fdcc602 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SimpleSolrIntegrationTest.java @@ -27,7 +27,7 @@ import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.crossdc.common.MirroredSolrRequest; -import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; +import org.apache.solr.crossdc.manager.consumer.OtelMetrics; import org.apache.solr.crossdc.manager.messageprocessor.SolrMessageProcessor; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -53,7 +53,7 @@ public static void beforeSimpleSolrIntegrationTest() throws Exception { CloudSolrClient cloudClient1 = cluster1.getSolrClient(); - processor = new SolrMessageProcessor(mock(PrometheusMetrics.class), () -> cloudClient1, null); + processor = new SolrMessageProcessor(mock(OtelMetrics.class), () -> cloudClient1, null); CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(COLLECTION, 1, 1); diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java index 152c9425e175..90c7913ed72c 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java @@ -108,6 +108,7 @@ public void beforeSolrAndKafkaIntegrationTest() throws Exception { uceh = Thread.getDefaultUncaughtExceptionHandler(); Thread.setDefaultUncaughtExceptionHandler( (t, e) -> log.error("Uncaught exception in thread {}", t, e)); + System.setProperty("otel.metrics.exporter", "prometheus"); consumer = new Consumer(); Properties config = new Properties(); @@ -359,12 +360,12 @@ public void testMetrics() throws Exception { new HttpJettySolrClient.Builder(baseUrl).useHttp1_1(true).build(); try { GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET, "/metrics"); - req.setResponseParser(new InputStreamResponseParser("test/plain")); + req.setResponseParser(new InputStreamResponseParser(null)); NamedList rsp = httpJettySolrClient.request(req); String content = IOUtils.toString( (InputStream) rsp.get(InputStreamResponseParser.STREAM_KEY), StandardCharsets.UTF_8); - assertTrue(content.contains("consumer_output_total{result=\"handled\",type=\"UPDATE\"} 1.0")); + assertTrue(content, content.contains("crossdc_consumer_output_total")); } finally { httpJettySolrClient.close(); client.close(); diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumerTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumerTest.java index 328a629a9fe0..34c37561d607 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumerTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumerTest.java @@ -111,7 +111,7 @@ protected CloudSolrClient createSolrClient() { // Set necessary configurations kafkaCrossDcConsumer = - new KafkaCrossDcConsumer(conf, new CountDownLatch(0)) { + new KafkaCrossDcConsumer(conf, ConsumerMetrics.NOOP, new CountDownLatch(0)) { @Override public KafkaConsumer> createKafkaConsumer( Properties properties) { @@ -182,7 +182,7 @@ public void kafkaCrossDcConsumerCreationWithConfigurationAndStartLatch() { KafkaConsumer> mockConsumer = mock(KafkaConsumer.class); KafkaCrossDcConsumer kafkaCrossDcConsumer = spy( - new KafkaCrossDcConsumer(conf, startLatch) { + new KafkaCrossDcConsumer(conf, ConsumerMetrics.NOOP, startLatch) { @Override public KafkaConsumer> createKafkaConsumer( Properties properties) { @@ -457,7 +457,7 @@ public void testHandleInvalidMirroredSolrRequest() { .handleItem(any()); KafkaCrossDcConsumer spyConsumer = spy( - new KafkaCrossDcConsumer(conf, new CountDownLatch(1)) { + new KafkaCrossDcConsumer(conf, ConsumerMetrics.NOOP, new CountDownLatch(1)) { @Override public KafkaConsumer> createKafkaConsumer( Properties properties) { @@ -551,7 +551,7 @@ public void testShutdown() { private KafkaCrossDcConsumer createCrossDcConsumerSpy( KafkaConsumer> mockConsumer) { return spy( - new KafkaCrossDcConsumer(conf, new CountDownLatch(1)) { + new KafkaCrossDcConsumer(conf, ConsumerMetrics.NOOP, new CountDownLatch(1)) { @Override public KafkaConsumer> createKafkaConsumer( Properties properties) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java index 20a72d2fa0fa..0d421b307c80 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessorTest.java @@ -36,7 +36,7 @@ import org.apache.solr.crossdc.common.IQueueHandler; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; -import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; +import org.apache.solr.crossdc.manager.consumer.OtelMetrics; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -57,8 +57,7 @@ public void setUp() { client = mock(CloudSolrClient.class); resubmitBackoffPolicy = mock(ResubmitBackoffPolicy.class); solrMessageProcessor = - new SolrMessageProcessor( - mock(PrometheusMetrics.class), () -> client, resubmitBackoffPolicy); + new SolrMessageProcessor(mock(OtelMetrics.class), () -> client, resubmitBackoffPolicy); } /** Should handle MirroredSolrRequest and return a failed result with no retry */ diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java index 0a686f650110..769fc58cb29c 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/messageprocessor/TestMessageProcessor.java @@ -39,7 +39,7 @@ import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.ResubmitBackoffPolicy; import org.apache.solr.crossdc.manager.consumer.ConsumerMetrics; -import org.apache.solr.crossdc.manager.consumer.PrometheusMetrics; +import org.apache.solr.crossdc.manager.consumer.OtelMetrics; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; @@ -72,7 +72,7 @@ public static void ensureWorkingMockito() { public void setUp() { MockitoAnnotations.initMocks(this); - ConsumerMetrics metrics = Mockito.mock(PrometheusMetrics.class); + ConsumerMetrics metrics = Mockito.mock(OtelMetrics.class); processor = Mockito.spy(new SolrMessageProcessor(metrics, () -> solrClient, backoffPolicy)); Mockito.doNothing().when(processor).uncheckedSleep(anyLong()); } diff --git a/solr/licenses/netty-tcnative-boringssl-static-2.0.73.Final.jar.sha1 b/solr/licenses/netty-tcnative-boringssl-static-2.0.73.Final.jar.sha1 new file mode 100644 index 000000000000..0bea976dc344 --- /dev/null +++ b/solr/licenses/netty-tcnative-boringssl-static-2.0.73.Final.jar.sha1 @@ -0,0 +1 @@ +de7380a74a7611e9937dd2106abfde5b405dbd15 diff --git a/solr/licenses/netty-tcnative-classes-2.0.73.Final.jar.sha1 b/solr/licenses/netty-tcnative-classes-2.0.73.Final.jar.sha1 new file mode 100644 index 000000000000..048e8b91eba6 --- /dev/null +++ b/solr/licenses/netty-tcnative-classes-2.0.73.Final.jar.sha1 @@ -0,0 +1 @@ +f50d875a46e4a7768f35dbc26fb796fd791b8b09 diff --git a/solr/licenses/prometheus-metrics-LICENSE-ASL.txt b/solr/licenses/prometheus-metrics-LICENSE-ASL.txt deleted file mode 100644 index f49a4e16e68b..000000000000 --- a/solr/licenses/prometheus-metrics-LICENSE-ASL.txt +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. \ No newline at end of file diff --git a/solr/licenses/prometheus-metrics-NOTICE.txt b/solr/licenses/prometheus-metrics-NOTICE.txt deleted file mode 100644 index cbd3cd95befd..000000000000 --- a/solr/licenses/prometheus-metrics-NOTICE.txt +++ /dev/null @@ -1,11 +0,0 @@ -Prometheus instrumentation library for JVM applications -Copyright 2012-2015 The Prometheus Authors - -This product includes software developed at -Boxever Ltd. (http://www.boxever.com/). - -This product includes software developed at -SoundCloud Ltd. (http://soundcloud.com/). - -This product includes software developed as part of the -Ocelli project by Netflix Inc. (https://github.com/Netflix/ocelli/). \ No newline at end of file diff --git a/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 deleted file mode 100644 index 954a48dcf646..000000000000 --- a/solr/licenses/prometheus-metrics-config-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4588ed1d1bbdcebb2663ccef1162f4cc4c5eb9fb diff --git a/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 deleted file mode 100644 index aefaedc39a60..000000000000 --- a/solr/licenses/prometheus-metrics-core-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b25424c069a44fce42b55626512718d49c9dcc5d diff --git a/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 deleted file mode 100644 index d087383b5789..000000000000 --- a/solr/licenses/prometheus-metrics-exporter-common-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2c719d8a7481f4dcc797c7450863bbb65a664e0d diff --git a/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 deleted file mode 100644 index e0b7056acce8..000000000000 --- a/solr/licenses/prometheus-metrics-exporter-servlet-jakarta-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3550df21bc7be5358bbe8b76e1430309dafb9c26 diff --git a/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 deleted file mode 100644 index c593438438de..000000000000 --- a/solr/licenses/prometheus-metrics-shaded-protobuf-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5a18603c6281609b0e9e1e7f21d647c6ce994adb diff --git a/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 deleted file mode 100644 index 942ecf85c06f..000000000000 --- a/solr/licenses/prometheus-metrics-tracer-common-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -daf7dbdb867221e418be0771663eae50a9199228 diff --git a/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 deleted file mode 100644 index f308f25736cd..000000000000 --- a/solr/licenses/prometheus-metrics-tracer-initializer-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -169f3dae0c95b4154696407de0e1213373f074e1 diff --git a/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 deleted file mode 100644 index c28f7197ac4f..000000000000 --- a/solr/licenses/prometheus-metrics-tracer-otel-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -25c2d488b084a9757a7b5e23cbce8dc812147e54 diff --git a/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 b/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 deleted file mode 100644 index 45a83f1dda19..000000000000 --- a/solr/licenses/prometheus-metrics-tracer-otel-agent-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -393255f85063e08684131ac75af5e65aef070a60 diff --git a/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java b/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java index 00ca4442d3da..0f45c8916bd4 100644 --- a/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java +++ b/solr/test-framework/src/java/org/apache/solr/util/SolrKafkaTestsIgnoredThreadsFilter.java @@ -45,11 +45,6 @@ public boolean reject(Thread t) { return true; } - // Prometheus Scheduler doesn't provide any method to shut down its worker threads - if (t.isDaemon()) { - return true; - } - return false; } } From b87ab4b27a8c048b6a80f63dba874a8a8dcb46c7 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Thu, 22 Jan 2026 19:10:47 +0100 Subject: [PATCH 09/15] Fix first time latency calculation. Clean up metric names. Properly record counts. --- .../solr/crossdc/manager/consumer/ConsumerMetrics.java | 8 ++++---- .../crossdc/manager/consumer/KafkaCrossDcConsumer.java | 6 +++--- .../solr/crossdc/manager/consumer/OtelMetrics.java | 6 +++--- .../manager/messageprocessor/SolrMessageProcessor.java | 10 ++++++---- .../crossdc/manager/SolrAndKafkaIntegrationTest.java | 4 ++-- .../update/processor/KafkaRequestMirroringHandler.java | 4 +++- 6 files changed, 21 insertions(+), 17 deletions(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java index 32952f91987e..a848c8a425f7 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -43,10 +43,10 @@ public void recordOutputBatchSize( MirroredSolrRequest.Type type, SolrRequest solrRequest) {} @Override - public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) {} + public void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffTimeMs) {} @Override - public void recordOutputFirstAttemptSize( + public void recordOutputFirstAttemptTime( MirroredSolrRequest.Type type, long firstAttemptTimeNs) {} @Override @@ -75,9 +75,9 @@ default void close() { void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest); - void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs); + void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffTimeMs); - void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs); + void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long firstAttemptTimeNs); ConsumerTimer startOutputTimeTimer(String requestType); } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index 61bd6201a512..6e3443083219 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -408,19 +408,19 @@ boolean pollAndProcessRequests() { List docs = update.getDocuments(); if (docs != null) { updateReqBatch.add(docs); - metrics.incrementInputCounter(type.name(), "add"); + metrics.incrementInputCounter(type.name(), "add", docs.size()); } List deletes = update.getDeleteById(); if (deletes != null) { updateReqBatch.deleteById(deletes); - metrics.incrementInputCounter(type.name(), "dbi"); + metrics.incrementInputCounter(type.name(), "delete_by_id", deletes.size()); } List deleteByQuery = update.getDeleteQuery(); if (deleteByQuery != null) { for (String delByQuery : deleteByQuery) { updateReqBatch.deleteByQuery(delByQuery); } - metrics.incrementInputCounter(type.name(), "dbq", deleteByQuery.size()); + metrics.incrementInputCounter(type.name(), "delete_by_query", deleteByQuery.size()); } } else { // non-update requests should be sent immediately diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java index 000d025c9553..d9ce0311cc5c 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java @@ -87,7 +87,7 @@ protected void register(String scope) { metricsContext.longHistogram( NAME_PREFIX + "output_first_attempt_time", "Histogram of first attempt request times", - OtelUnit.MILLISECONDS); + OtelUnit.NANOSECONDS); } protected static final String KEY_SEPARATOR = "#"; @@ -158,12 +158,12 @@ public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest } @Override - public void recordOutputBackoffSize(MirroredSolrRequest.Type type, long backoffTimeMs) { + public void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffTimeMs) { outputBackoffHistogram.record(backoffTimeMs, attr(ATTR_TYPE, type.name())); } @Override - public void recordOutputFirstAttemptSize(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { + public void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long firstAttemptTimeNs) { outputFirstAttemptHistogram.record(firstAttemptTimeNs, attr(ATTR_TYPE, type.name())); } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index 66262e48d6c7..fae42fd9b44c 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.TimeSource; import org.apache.solr.crossdc.common.CrossDcConstants; import org.apache.solr.crossdc.common.IQueueHandler; import org.apache.solr.crossdc.common.MirroredSolrRequest; @@ -142,7 +143,7 @@ private void maybeBackoff(MirroredSolrRequest request, SolrException solrExce sleepTimeMs = Math.max(1, Long.parseLong(backoffTimeSuggested)); } log.info("Consumer backoff. sleepTimeMs={}", sleepTimeMs); - metrics.recordOutputBackoffSize(request.getType(), sleepTimeMs); + metrics.recordOutputBackoffTime(request.getType(), sleepTimeMs); uncheckedSleep(sleepTimeMs); } @@ -304,9 +305,10 @@ private void logFirstAttemptLatency(MirroredSolrRequest mirroredSolrRequest) // submitting on the primary side until the request is eligible to be consumed on the buddy side // (or vice versa). if (mirroredSolrRequest.getAttempt() == 1) { - final long latency = System.nanoTime() - mirroredSolrRequest.getSubmitTimeNanos(); + final long latency = + TimeSource.CURRENT_TIME.getTimeNs() - mirroredSolrRequest.getSubmitTimeNanos(); log.debug("First attempt latency = {} ns", latency); - metrics.recordOutputFirstAttemptSize(mirroredSolrRequest.getType(), latency); + metrics.recordOutputFirstAttemptTime(mirroredSolrRequest.getType(), latency); } } @@ -375,7 +377,7 @@ private void backoffIfNeeded( if (result.status().equals(ResultStatus.FAILED_RESUBMIT)) { final long backoffMs = getResubmitBackoffPolicy().getBackoffTimeMs(result.getItem()); if (backoffMs > 0L) { - metrics.recordOutputBackoffSize(type, backoffMs); + metrics.recordOutputBackoffTime(type, backoffMs); try { Thread.sleep(backoffMs); } catch (final InterruptedException ex) { diff --git a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java index 90c7913ed72c..7d7941dc442e 100644 --- a/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java +++ b/solr/cross-dc-manager/src/test/org/apache/solr/crossdc/manager/SolrAndKafkaIntegrationTest.java @@ -192,7 +192,7 @@ public void testFullCloudToCloud() throws Exception { client.commit(COLLECTION); - System.out.println("Sent producer record"); + log.info("Sent producer record"); assertCluster2EventuallyHasDocs(COLLECTION, "*:*", 1); } @@ -351,7 +351,7 @@ public void testMetrics() throws Exception { client.commit(COLLECTION); - System.out.println("Sent producer record"); + log.info("Sent producer record"); assertCluster2EventuallyHasDocs(COLLECTION, "*:*", 1); diff --git a/solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/KafkaRequestMirroringHandler.java b/solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/KafkaRequestMirroringHandler.java index 384cd6ef9987..2201d1ca6337 100644 --- a/solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/KafkaRequestMirroringHandler.java +++ b/solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/KafkaRequestMirroringHandler.java @@ -18,6 +18,7 @@ import java.lang.invoke.MethodHandles; import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.common.util.TimeSource; import org.apache.solr.crossdc.common.KafkaMirroringSink; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.crossdc.common.MirroringException; @@ -51,7 +52,8 @@ public void mirror(UpdateRequest request) throws MirroringException { } // TODO: Enforce external version constraint for consistent update replication (cross-cluster) final MirroredSolrRequest mirroredRequest = - new MirroredSolrRequest<>(MirroredSolrRequest.Type.UPDATE, 1, request, System.nanoTime()); + new MirroredSolrRequest<>( + MirroredSolrRequest.Type.UPDATE, 1, request, TimeSource.CURRENT_TIME.getTimeNs()); try { sink.submit(mirroredRequest); } catch (MirroringException exception) { From 65fa041abd3e72250d453ea4dfb1b151a9854907 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 26 Jan 2026 17:46:30 +0100 Subject: [PATCH 10/15] Tidy. --- .../solr/handler/admin/MetricsHandler.java | 2 +- .../response/PrometheusResponseWriter.java | 4 +- solr/cross-dc-manager/build.gradle | 18 ++++----- solr/cross-dc-manager/gradle.lockfile | 18 ++++----- .../solr/crossdc/manager/consumer/Util.java | 40 ++++++++++++------- 5 files changed, 45 insertions(+), 37 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java index 13b4d044c7e2..a0ab70e07fdb 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHandler.java @@ -133,7 +133,7 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw } } - private void handleRequest(SolrParams params, BiConsumer consumer) { + public void handleRequest(SolrParams params, BiConsumer consumer) { if (!enabled) { consumer.accept("error", "metrics collection is disabled"); return; diff --git a/solr/core/src/java/org/apache/solr/response/PrometheusResponseWriter.java b/solr/core/src/java/org/apache/solr/response/PrometheusResponseWriter.java index 01a2af194210..610fbaa8df2f 100644 --- a/solr/core/src/java/org/apache/solr/response/PrometheusResponseWriter.java +++ b/solr/core/src/java/org/apache/solr/response/PrometheusResponseWriter.java @@ -34,8 +34,8 @@ public class PrometheusResponseWriter implements QueryResponseWriter { // not TextQueryResponseWriter because Prometheus libs work with an OutputStream - private static final String CONTENT_TYPE_PROMETHEUS = "text/plain; version=0.0.4"; - private static final String CONTENT_TYPE_OPEN_METRICS = + public static final String CONTENT_TYPE_PROMETHEUS = "text/plain; version=0.0.4"; + public static final String CONTENT_TYPE_OPEN_METRICS = "application/openmetrics-text; version=1.0.0; charset=utf-8"; @Override diff --git a/solr/cross-dc-manager/build.gradle b/solr/cross-dc-manager/build.gradle index e101a10279d1..9fae9d75e94f 100644 --- a/solr/cross-dc-manager/build.gradle +++ b/solr/cross-dc-manager/build.gradle @@ -24,6 +24,7 @@ description = 'Cross-DC Manager' dependencies { implementation platform(project(':platform')) + implementation project(':solr:core') implementation project(':solr:solrj') implementation project(':solr:solrj-zookeeper') implementation project(':solr:modules:cross-dc') @@ -31,20 +32,15 @@ dependencies { implementation platform(libs.opentelemetry.bom) implementation libs.opentelemetry.api - implementation libs.opentelemetry.sdk implementation libs.opentelemetry.sdk.metrics - implementation libs.opentelemetry.sdk.extension.autoconfigure - implementation libs.opentelemetry.exporter.otlp - implementation(libs.opentelemetry.exporter.prometheus) { - transitive = false - } - implementation libs.prometheus.metrics.model - implementation(libs.prometheus.metrics.expositionformats, { - exclude group: "io.prometheus", module: "prometheus-metrics-shaded-protobuf" - exclude group: "io.prometheus", module: "prometheus-metrics-config" - }) +// implementation libs.prometheus.metrics.model +// implementation(libs.prometheus.metrics.expositionformats, { +// exclude group: "io.prometheus", module: "prometheus-metrics-shaded-protobuf" +// exclude group: "io.prometheus", module: "prometheus-metrics-config" +// }) implementation libs.eclipse.jetty.server implementation libs.eclipse.jetty.ee10.servlet + implementation libs.jakarta.servlet.api implementation libs.slf4j.api runtimeOnly libs.google.protobuf.javautils runtimeOnly libs.commonscodec.commonscodec diff --git a/solr/cross-dc-manager/gradle.lockfile b/solr/cross-dc-manager/gradle.lockfile index e46fd5214864..ca8999d0c32c 100644 --- a/solr/cross-dc-manager/gradle.lockfile +++ b/solr/cross-dc-manager/gradle.lockfile @@ -97,19 +97,19 @@ io.opentelemetry:opentelemetry-common:1.56.0=compileClasspath,jarValidation,runt io.opentelemetry:opentelemetry-context:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath io.opentelemetry:opentelemetry-exporter-common:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-exporter-otlp-common:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath -io.opentelemetry:opentelemetry-exporter-otlp:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-exporter-prometheus:1.56.0-alpha=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-otlp:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-exporter-prometheus:1.56.0-alpha=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-exporter-sender-okhttp:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-sdk-common:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-extension-autoconfigure:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-logs:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-extension-autoconfigure:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-logs:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath io.opentelemetry:opentelemetry-sdk-metrics:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk-trace:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.opentelemetry:opentelemetry-sdk:1.56.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk-trace:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +io.opentelemetry:opentelemetry-sdk:1.56.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.perfmark:perfmark-api:0.27.0=jarValidation,runtimeClasspath,runtimeLibs,testRuntimeClasspath -io.prometheus:prometheus-metrics-exposition-formats:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath -io.prometheus:prometheus-metrics-model:1.1.0=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath +io.prometheus:prometheus-metrics-exposition-formats:1.1.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath +io.prometheus:prometheus-metrics-model:1.1.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.sgr:s2-geometry-library-java:1.0.0=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath io.swagger.core.v3:swagger-annotations-jakarta:2.2.22=compileClasspath,jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testCompileClasspath,testRuntimeClasspath jakarta.annotation:jakarta.annotation-api:2.1.1=jarValidation,runtimeClasspath,runtimeLibs,solrPlatformLibs,testRuntimeClasspath diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index 8d49d455a3d4..14ba5316a639 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -16,15 +16,8 @@ */ package org.apache.solr.crossdc.manager.consumer; -// import io.prometheus.metrics.expositionformats.ExpositionFormats; -// import io.prometheus.metrics.model.registry.PrometheusRegistry; -// import io.prometheus.metrics.model.snapshots.MetricSnapshots; -import io.prometheus.metrics.expositionformats.PrometheusTextFormatWriter; -import io.prometheus.metrics.model.snapshots.MetricSnapshot; -import io.prometheus.metrics.model.snapshots.MetricSnapshots; import java.io.ByteArrayOutputStream; import java.lang.invoke.MethodHandles; -import java.util.List; import java.util.Properties; import java.util.Set; import org.apache.kafka.clients.admin.AdminClient; @@ -35,8 +28,12 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.solr.common.params.SolrParams; import org.apache.solr.crossdc.common.MirroredSolrRequestSerializer; +import org.apache.solr.handler.admin.MetricsHandler; import org.apache.solr.metrics.SolrMetricManager; +import org.apache.solr.response.PrometheusResponseWriter; +import org.apache.solr.response.SolrQueryResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,21 +41,36 @@ public class Util { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static void logMetrics(SolrMetricManager metricManager) { - List snapshotList = - metricManager.getPrometheusMetricReaders().values().stream() - .flatMap(r -> r.collect().stream()) - .toList(); - MetricSnapshots snapshots = MetricSnapshots.of(snapshotList.toArray(new MetricSnapshot[0])); + SolrQueryResponse rsp = new SolrQueryResponse(); + new MetricsHandler(metricManager) + .handleRequest(SolrParams.of(), (key, value) -> rsp.add(key, value)); String output; try { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - new PrometheusTextFormatWriter(false).write(baos, snapshots); + new PrometheusResponseWriter() + .write(baos, null, rsp, PrometheusResponseWriter.CONTENT_TYPE_PROMETHEUS); output = baos.toString(); } catch (Exception e) { log.error("Error while writing final metrics", e); - output = snapshots.toString(); + output = rsp.toString(); } log.info("#### Consumer Metrics: ####\n{}", output); + // List snapshotList = + // metricManager.getPrometheusMetricReaders().values().stream() + // .flatMap(r -> r.collect().stream()) + // .toList(); + // MetricSnapshots snapshots = MetricSnapshots.of(snapshotList.toArray(new + // MetricSnapshot[0])); + // String output; + // try { + // ByteArrayOutputStream baos = new ByteArrayOutputStream(); + // new PrometheusTextFormatWriter(false).write(baos, snapshots); + // output = baos.toString(); + // } catch (Exception e) { + // log.error("Error while writing final metrics", e); + // output = snapshots.toString(); + // } + // log.info("#### Consumer Metrics: ####\n{}", output); } public static void printKafkaInfo(String host, String groupId) { From dc5622209326079909fb2bde40c8566091171d1d Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 27 Jan 2026 16:16:35 +0100 Subject: [PATCH 11/15] Fix issues from review. Add javadocs. Cleanups. --- .../manager/consumer/ConsumerMetrics.java | 116 ++++++++++++++++-- .../consumer/KafkaCrossDcConsumer.java | 18 +-- .../crossdc/manager/consumer/OtelMetrics.java | 60 +++++---- .../solr/crossdc/manager/consumer/Util.java | 16 --- .../SolrMessageProcessor.java | 10 +- 5 files changed, 150 insertions(+), 70 deletions(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java index a848c8a425f7..371245ce66b9 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -19,21 +19,24 @@ import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.crossdc.common.MirroredSolrRequest; +/** + * Interface for tracking and recording metrics related to the processing of messages and requests + * in the {@link Consumer}. Provides methods to increment counters, record timing metrics, and + * capture other performance-related data points. + */ public interface ConsumerMetrics { + /** No-op implementation of {@link ConsumerMetrics}. */ ConsumerMetrics NOOP = new ConsumerMetrics() { @Override public void incrementCollapsedCounter() {} @Override - public void incrementInputCounter(String type, String subType) {} - - @Override - public void incrementInputCounter(String type, String subType, int delta) {} + public void incrementInputMsgCounter(long delta) {} @Override - public void incrementOutputCounter(String type, String result) {} + public void incrementInputReqCounter(String type, String subType, int delta) {} @Override public void incrementOutputCounter(String type, String result, int delta) {} @@ -47,7 +50,7 @@ public void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffT @Override public void recordOutputFirstAttemptTime( - MirroredSolrRequest.Type type, long firstAttemptTimeNs) {} + MirroredSolrRequest.Type type, long firstAttemptTimeMs) {} @Override public ConsumerTimer startOutputTimeTimer(String requestType) { @@ -55,29 +58,118 @@ public ConsumerTimer startOutputTimeTimer(String requestType) { } }; + /** + * Represents a timer interface used for measuring and observing the duration of tasks. Start + * measuring elapsed time when created. + */ interface ConsumerTimer { + /** + * Return the elapsed time in milliseconds. + * + * @return + */ double observeDuration(); + /** */ default void close() { observeDuration(); } } - void incrementCollapsedCounter(); + /** Increments the counter for input messages. */ + default void incrementInputMsgCounter() { + incrementInputMsgCounter(1L); + } - void incrementInputCounter(String type, String subType); + /** + * Increments the counter for input messages. + * + * @param delta increase the counter by this value + */ + void incrementInputMsgCounter(long delta); - void incrementInputCounter(String type, String subType, int delta); + /** Increments the counter for collapsed "add" requests. */ + void incrementCollapsedCounter(); - void incrementOutputCounter(String type, String result); + /** + * Increments the counter for input requests by type and subtype. + * + * @param type request type, one of {@link + * org.apache.solr.crossdc.common.MirroredSolrRequest.Type} values. + * @param subType additional subtype: add, delete_by_id, delete_by_query, or action for other + * request types. + */ + default void incrementInputReqCounter(String type, String subType) { + incrementInputReqCounter(type, subType, 1); + } + /** + * Increments the counter for input requests by type, subtype, and delta. + * + * @param type request type, one of {@link + * org.apache.solr.crossdc.common.MirroredSolrRequest.Type} values. + * @param subType additional subtype: add, delete_by_id, delete_by_query, or action for other + * request types. + * @param delta increase the counter by this value + */ + void incrementInputReqCounter(String type, String subType, int delta); + + /** + * Increments the counter for output requests by type and result. + * + * @param type the type of the request + * @param result the result of the request, such as success or failure + */ + default void incrementOutputCounter(String type, String result) { + incrementOutputCounter(type, result, 1); + } + + /** + * Increments the counter for output requests by type, result, and delta. + * + * @param type the type of the request + * @param result the result of the request, such as success or failure + * @param delta the value by which the counter should be increased + */ void incrementOutputCounter(String type, String result, int delta); + /** + * Records the batch size of the output request. Batch size is defined as the number of operations + * in an output {@link SolrRequest} (which may be different than the input size due to + * collapsing). + * + * @param type the type of the request, corresponding to one of the {@link + * MirroredSolrRequest.Type} values + * @param solrRequest SolrRequest object for which the batch size is being recorded + */ void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest); + /** + * Records the backoff time for output requests. Backoff time represents the delay before the next + * retry for the specified request type. + * + * @param type the type of the request, corresponding to one of the {@link + * MirroredSolrRequest.Type} values. + * @param backoffTimeMs the backoff time in milliseconds. + */ void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffTimeMs); - void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long firstAttemptTimeNs); - + /** + * Records the latency between the time when the message was sent at source and the time of the + * first attempt at processing. + * + * @param type the type of the request, corresponding to one of the {@link + * MirroredSolrRequest.Type} values + * @param firstAttemptTimeMs the latency of the first attempt in milliseconds. + */ + void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long firstAttemptTimeMs); + + /** + * Starts a timer to measure the duration of an output request processing by the given request + * type. + * + * @param requestType the type of the request for which the timer is started + * @return a {@link ConsumerTimer} that allows to measure the elapsed time + */ ConsumerTimer startOutputTimeTimer(String requestType); } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index 6e3443083219..c904deb02e5a 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -338,12 +338,15 @@ boolean pollAndProcessRequests() { requestRecord.value()); } + metrics.incrementInputMsgCounter(); lastRecord = requestRecord; MirroredSolrRequest req = requestRecord.value(); SolrRequest solrReq = req.getSolrRequest(); MirroredSolrRequest.Type type = req.getType(); + if (type != MirroredSolrRequest.Type.UPDATE) { - metrics.incrementInputCounter(type.name(), solrReq.getPath()); + String action = solrReq.getParams().get("action", "unknown"); + metrics.incrementInputReqCounter(type.name(), action); } ModifiableSolrParams params = new ModifiableSolrParams(solrReq.getParams()); @@ -408,19 +411,20 @@ boolean pollAndProcessRequests() { List docs = update.getDocuments(); if (docs != null) { updateReqBatch.add(docs); - metrics.incrementInputCounter(type.name(), "add", docs.size()); + metrics.incrementInputReqCounter(type.name(), "add", docs.size()); } List deletes = update.getDeleteById(); if (deletes != null) { updateReqBatch.deleteById(deletes); - metrics.incrementInputCounter(type.name(), "delete_by_id", deletes.size()); + metrics.incrementInputReqCounter(type.name(), "delete_by_id", deletes.size()); } List deleteByQuery = update.getDeleteQuery(); if (deleteByQuery != null) { for (String delByQuery : deleteByQuery) { updateReqBatch.deleteByQuery(delByQuery); } - metrics.incrementInputCounter(type.name(), "delete_by_query", deleteByQuery.size()); + metrics.incrementInputReqCounter( + type.name(), "delete_by_query", deleteByQuery.size()); } } else { // non-update requests should be sent immediately @@ -533,10 +537,10 @@ protected void processResult( "Sending message to dead letter queue because of max attempts limit with current value = {}", attempt); kafkaMirroringSink.submitToDlq(item); - metrics.incrementOutputCounter(type.name(), "failed-dlq"); + metrics.incrementOutputCounter(type.name(), "failed_dlq"); } else { kafkaMirroringSink.submit(item); - metrics.incrementOutputCounter(type.name(), "failed-resubmit"); + metrics.incrementOutputCounter(type.name(), "failed_resubmit"); } break; case HANDLED: @@ -555,7 +559,7 @@ protected void processResult( case FAILED_RETRY: log.error( "Unexpected response while processing request. We never expect {}.", result.status()); - metrics.incrementOutputCounter(type.name(), "failed-retry"); + metrics.incrementOutputCounter(type.name(), "failed_retry"); break; default: if (log.isTraceEnabled()) { diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java index d9ce0311cc5c..1c0fcd5bc775 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/OtelMetrics.java @@ -21,15 +21,14 @@ import io.opentelemetry.api.metrics.LongHistogram; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.crossdc.common.MirroredSolrRequest; import org.apache.solr.metrics.SolrMetricManager; import org.apache.solr.metrics.SolrMetricsContext; import org.apache.solr.metrics.otel.OtelUnit; +import org.apache.solr.metrics.otel.instruments.AttributedLongTimer; import org.apache.solr.opentelemetry.OtlpExporterFactory; -import org.apache.solr.util.RTimer; public class OtelMetrics implements ConsumerMetrics { @@ -43,7 +42,8 @@ public class OtelMetrics implements ConsumerMetrics { protected SolrMetricManager metricManager; - protected LongCounter input; + protected LongCounter inputMsg; + protected LongCounter inputReq; protected LongCounter collapsed; protected LongCounter output; protected LongHistogram outputBatchSizeHistogram; @@ -59,12 +59,17 @@ protected void register(String scope) { this.metricManager = new SolrMetricManager(new OtlpExporterFactory().getExporter()); SolrMetricsContext metricsContext = new SolrMetricsContext(metricManager, scope); - input = - metricsContext.longCounter(NAME_PREFIX + "input_total", "Total number of input messages"); + inputMsg = + metricsContext.longCounter( + NAME_PREFIX + "input_msg_total", "Total number of input Kafka messages"); + + inputReq = + metricsContext.longCounter( + NAME_PREFIX + "input_req_total", "Total number of input Solr requests"); collapsed = metricsContext.longCounter( - NAME_PREFIX + "collapsed_total", "Total number of collapsed messages"); + NAME_PREFIX + "collapsed_total", "Total number of collapsed update requests"); output = metricsContext.longCounter(NAME_PREFIX + "output_total", "Total number of output requests"); @@ -75,7 +80,7 @@ protected void register(String scope) { outputBackoffHistogram = metricsContext.longHistogram( - NAME_PREFIX + "output_backoff_size", "Histogram of output backoff sleep times"); + NAME_PREFIX + "output_backoff_time", "Histogram of output backoff sleep times"); outputTimeHistogram = metricsContext.longHistogram( @@ -87,7 +92,7 @@ protected void register(String scope) { metricsContext.longHistogram( NAME_PREFIX + "output_first_attempt_time", "Histogram of first attempt request times", - OtelUnit.NANOSECONDS); + OtelUnit.MILLISECONDS); } protected static final String KEY_SEPARATOR = "#"; @@ -114,30 +119,30 @@ public void incrementCollapsedCounter() { } @Override - public void incrementInputCounter(String type, String subType) { - incrementInputCounter(type, subType, 1); - } - - @Override - public void incrementInputCounter(String type, String subType, int delta) { - input.add(delta, attr("type", type, "subtype", subType)); + public void incrementInputMsgCounter(long delta) { + inputMsg.add(delta); } @Override - public void incrementOutputCounter(String type, String result) { - incrementOutputCounter(type, result, 1); + public void incrementInputReqCounter(String type, String subType, int delta) { + inputReq.add(delta, attr(ATTR_TYPE, type, ATTR_SUBTYPE, subType)); } @Override public void incrementOutputCounter(String type, String result, int delta) { - output.add(delta, attr("type", type, "result", result)); + output.add(delta, attr(ATTR_TYPE, type, ATTR_RESULT, result)); } @Override public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest) { if (type != MirroredSolrRequest.Type.UPDATE) { outputBatchSizeHistogram.record( - 1, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, solrRequest.getPath())); + 1, + attr( + ATTR_TYPE, + type.name(), + ATTR_SUBTYPE, + solrRequest.getParams().get("action", "unknown"))); return; } UpdateRequest req = (UpdateRequest) solrRequest; @@ -153,7 +158,7 @@ public void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest } if (dbqCount > 0) { outputBatchSizeHistogram.record( - dbiCount, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, "delete_by_query")); + dbqCount, attr(ATTR_TYPE, type.name(), ATTR_SUBTYPE, "delete_by_query")); } } @@ -169,16 +174,9 @@ public void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long fir @Override public ConsumerTimer startOutputTimeTimer(final String requestType) { - final RTimer timer = - new RTimer(TimeUnit.MILLISECONDS) { - @Override - public double stop() { - double elapsedTime = super.stop(); - outputTimeHistogram.record( - Double.valueOf(elapsedTime).longValue(), attr(ATTR_TYPE, requestType)); - return elapsedTime; - } - }; - return () -> timer.stop(); + final AttributedLongTimer timer = + new AttributedLongTimer(outputTimeHistogram, attr(ATTR_TYPE, requestType)); + final AttributedLongTimer.MetricTimer metricTimer = timer.start(); + return () -> metricTimer.stop(); } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index 14ba5316a639..ce3dddd044d1 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -55,22 +55,6 @@ public static void logMetrics(SolrMetricManager metricManager) { output = rsp.toString(); } log.info("#### Consumer Metrics: ####\n{}", output); - // List snapshotList = - // metricManager.getPrometheusMetricReaders().values().stream() - // .flatMap(r -> r.collect().stream()) - // .toList(); - // MetricSnapshots snapshots = MetricSnapshots.of(snapshotList.toArray(new - // MetricSnapshot[0])); - // String output; - // try { - // ByteArrayOutputStream baos = new ByteArrayOutputStream(); - // new PrometheusTextFormatWriter(false).write(baos, snapshots); - // output = baos.toString(); - // } catch (Exception e) { - // log.error("Error while writing final metrics", e); - // output = snapshots.toString(); - // } - // log.info("#### Consumer Metrics: ####\n{}", output); } public static void printKafkaInfo(String host, String groupId) { diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index fae42fd9b44c..80314b8f9617 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -19,6 +19,7 @@ import java.lang.invoke.MethodHandles; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.impl.CloudSolrClient; @@ -305,10 +306,11 @@ private void logFirstAttemptLatency(MirroredSolrRequest mirroredSolrRequest) // submitting on the primary side until the request is eligible to be consumed on the buddy side // (or vice versa). if (mirroredSolrRequest.getAttempt() == 1) { - final long latency = - TimeSource.CURRENT_TIME.getTimeNs() - mirroredSolrRequest.getSubmitTimeNanos(); - log.debug("First attempt latency = {} ns", latency); - metrics.recordOutputFirstAttemptTime(mirroredSolrRequest.getType(), latency); + final long latencyMs = + TimeUnit.NANOSECONDS.toMillis( + TimeSource.CURRENT_TIME.getTimeNs() - mirroredSolrRequest.getSubmitTimeNanos()); + log.debug("First attempt latency = {} ms", latencyMs); + metrics.recordOutputFirstAttemptTime(mirroredSolrRequest.getType(), latencyMs); } } From ed8f582dcb36be11296c4574500beed2857b1958 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 27 Jan 2026 17:15:50 +0100 Subject: [PATCH 12/15] Resolve api drift. --- .../crossdc/manager/messageprocessor/SolrMessageProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index 256c6545c5a5..14bb2a89e507 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -210,7 +210,7 @@ private Result> processMirroredSolrRequest( "Skipping update request to nonexistent / not updatable collection {}", request.getCollection()); } - metrics.counter(MetricRegistry.name(type.name(), "invalid-collection")).inc(); + metrics.incrementOutputCounter(type.name(), "failed_collection_not_found"); return new Result<>(ResultStatus.FAILED_NO_RETRY, mirroredSolrRequest); } } From 97fdd65748673e8e43477bca827ab192403dbdc4 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 27 Jan 2026 18:31:34 +0100 Subject: [PATCH 13/15] Javadoc tidy. --- .../crossdc/manager/consumer/ConsumerMetrics.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java index 371245ce66b9..92d96fae866e 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ConsumerMetrics.java @@ -63,14 +63,10 @@ public ConsumerTimer startOutputTimeTimer(String requestType) { * measuring elapsed time when created. */ interface ConsumerTimer { - /** - * Return the elapsed time in milliseconds. - * - * @return - */ + /** Return the elapsed time in milliseconds. */ double observeDuration(); - /** */ + /** Close this timer. */ default void close() { observeDuration(); } @@ -139,7 +135,7 @@ default void incrementOutputCounter(String type, String result) { * collapsing). * * @param type the type of the request, corresponding to one of the {@link - * MirroredSolrRequest.Type} values + * MirroredSolrRequest#getType()} values * @param solrRequest SolrRequest object for which the batch size is being recorded */ void recordOutputBatchSize(MirroredSolrRequest.Type type, SolrRequest solrRequest); @@ -149,7 +145,7 @@ default void incrementOutputCounter(String type, String result) { * retry for the specified request type. * * @param type the type of the request, corresponding to one of the {@link - * MirroredSolrRequest.Type} values. + * MirroredSolrRequest#getType()} values. * @param backoffTimeMs the backoff time in milliseconds. */ void recordOutputBackoffTime(MirroredSolrRequest.Type type, long backoffTimeMs); @@ -159,7 +155,7 @@ default void incrementOutputCounter(String type, String result) { * first attempt at processing. * * @param type the type of the request, corresponding to one of the {@link - * MirroredSolrRequest.Type} values + * MirroredSolrRequest#getType()} values * @param firstAttemptTimeMs the latency of the first attempt in milliseconds. */ void recordOutputFirstAttemptTime(MirroredSolrRequest.Type type, long firstAttemptTimeMs); From c068dc179055806bc6fe34aabf2d22a94458c25b Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Wed, 28 Jan 2026 10:49:17 +0100 Subject: [PATCH 14/15] Forbidden APIs. --- .../crossdc/manager/consumer/ThreadDump.java | 37 +++++++++++++------ .../manager/consumer/ThreadDumpServlet.java | 17 +++++---- .../solr/crossdc/manager/consumer/Util.java | 3 +- 3 files changed, 38 insertions(+), 19 deletions(-) diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java index b93b9d441f61..2f0fb116bf9d 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDump.java @@ -16,6 +16,8 @@ */ package org.apache.solr.crossdc.manager.consumer; +import static java.nio.charset.StandardCharsets.UTF_8; + import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.PrintWriter; @@ -23,7 +25,7 @@ import java.lang.management.MonitorInfo; import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; -import java.nio.charset.StandardCharsets; +import java.util.Locale; /** * A convenience class for getting a thread dump. @@ -65,22 +67,34 @@ public void dump(OutputStream out) { public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStream out) { final ThreadInfo[] threads = this.threadMXBean.dumpAllThreads(lockedMonitors, lockedSynchronizers); - final PrintWriter writer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); + final PrintWriter writer = new PrintWriter(new OutputStreamWriter(out, UTF_8)); for (int ti = threads.length - 1; ti >= 0; ti--) { final ThreadInfo t = threads[ti]; writer.printf( - "\"%s\" id=%d state=%s", t.getThreadName(), t.getThreadId(), t.getThreadState()); + Locale.ROOT, + "\"%s\" id=%d state=%s", + t.getThreadName(), + t.getThreadId(), + t.getThreadState()); final LockInfo lock = t.getLockInfo(); if (lock != null && t.getThreadState() != Thread.State.BLOCKED) { writer.printf( - "%n - waiting on <0x%08x> (a %s)", lock.getIdentityHashCode(), lock.getClassName()); + Locale.ROOT, + "%n - waiting on <0x%08x> (a %s)", + lock.getIdentityHashCode(), + lock.getClassName()); writer.printf( - "%n - locked <0x%08x> (a %s)", lock.getIdentityHashCode(), lock.getClassName()); + Locale.ROOT, + "%n - locked <0x%08x> (a %s)", + lock.getIdentityHashCode(), + lock.getClassName()); } else if (lock != null && t.getThreadState() == Thread.State.BLOCKED) { writer.printf( + Locale.ROOT, "%n - waiting to lock <0x%08x> (a %s)", - lock.getIdentityHashCode(), lock.getClassName()); + lock.getIdentityHashCode(), + lock.getClassName()); } if (t.isSuspended()) { @@ -93,7 +107,8 @@ public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStre writer.println(); if (t.getLockOwnerName() != null) { - writer.printf(" owned by %s id=%d%n", t.getLockOwnerName(), t.getLockOwnerId()); + writer.printf( + Locale.ROOT, " owned by %s id=%d%n", t.getLockOwnerName(), t.getLockOwnerId()); } final StackTraceElement[] elements = t.getStackTrace(); @@ -101,11 +116,11 @@ public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStre for (int i = 0; i < elements.length; i++) { final StackTraceElement element = elements[i]; - writer.printf(" at %s%n", element); + writer.printf(Locale.ROOT, " at %s%n", element); for (int j = 1; j < monitors.length; j++) { final MonitorInfo monitor = monitors[j]; if (monitor.getLockedStackDepth() == i) { - writer.printf(" - locked %s%n", monitor); + writer.printf(Locale.ROOT, " - locked %s%n", monitor); } } } @@ -113,9 +128,9 @@ public void dump(boolean lockedMonitors, boolean lockedSynchronizers, OutputStre final LockInfo[] locks = t.getLockedSynchronizers(); if (locks.length > 0) { - writer.printf(" Locked synchronizers: count = %d%n", locks.length); + writer.printf(Locale.ROOT, " Locked synchronizers: count = %d%n", locks.length); for (LockInfo l : locks) { - writer.printf(" - %s%n", l); + writer.printf(Locale.ROOT, " - %s%n", l); } writer.println(); } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java index 7967365d0f61..cea2bb339884 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/ThreadDumpServlet.java @@ -23,6 +23,9 @@ import java.io.IOException; import java.io.OutputStream; import java.lang.management.ManagementFactory; +import java.nio.charset.StandardCharsets; +import org.apache.solr.common.params.SolrParams; +import org.apache.solr.servlet.SolrRequestParsers; /** * An HTTP servlets which outputs a {@code text/plain} dump of all threads in the VM. Only responds @@ -52,22 +55,22 @@ public void init() throws ServletException { @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - final boolean includeMonitors = getParam(req.getParameter("monitors"), true); - final boolean includeSynchronizers = getParam(req.getParameter("synchronizers"), true); + final SolrParams queryParams = SolrRequestParsers.parseQueryString(req.getQueryString()); + final boolean includeMonitors = queryParams.getBool("monitors", true); + final boolean includeSynchronizers = queryParams.getBool("synchronizers", true); resp.setStatus(HttpServletResponse.SC_OK); resp.setContentType(CONTENT_TYPE); resp.setHeader("Cache-Control", "must-revalidate,no-cache,no-store"); if (threadDump == null) { - resp.getWriter().println("Sorry your runtime environment does not allow to dump threads."); + resp.getOutputStream() + .write( + "Sorry your runtime environment does not allow to dump threads.\r\n" + .getBytes(StandardCharsets.UTF_8)); return; } try (OutputStream output = resp.getOutputStream()) { threadDump.dump(includeMonitors, includeSynchronizers, output); } } - - private static Boolean getParam(String initParam, boolean defaultValue) { - return initParam == null ? defaultValue : Boolean.parseBoolean(initParam); - } } diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java index ce3dddd044d1..f0d4dcca4da5 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/Util.java @@ -18,6 +18,7 @@ import java.io.ByteArrayOutputStream; import java.lang.invoke.MethodHandles; +import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.Set; import org.apache.kafka.clients.admin.AdminClient; @@ -49,7 +50,7 @@ public static void logMetrics(SolrMetricManager metricManager) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); new PrometheusResponseWriter() .write(baos, null, rsp, PrometheusResponseWriter.CONTENT_TYPE_PROMETHEUS); - output = baos.toString(); + output = baos.toString(StandardCharsets.UTF_8); } catch (Exception e) { log.error("Error while writing final metrics", e); output = rsp.toString(); From 2f893660a3650bb1f256186506b27377d730baa7 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 2 Feb 2026 13:07:24 +0100 Subject: [PATCH 15/15] Fix issues from review. --- gradle/libs.versions.toml | 2 -- solr/cross-dc-manager/build.gradle | 5 ----- .../solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java | 4 ++-- .../manager/messageprocessor/SolrMessageProcessor.java | 2 +- 4 files changed, 3 insertions(+), 10 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 6d1255659ecb..c9616523957d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -498,8 +498,6 @@ ow2-asm-commons = { module = "org.ow2.asm:asm-commons", version.ref = "ow2-asm" ow2-asm-tree = { module = "org.ow2.asm:asm-tree", version.ref = "ow2-asm" } # @keep transitive dependency for version alignment perfmark-api = { module = "io.perfmark:perfmark-api", version.ref = "perfmark" } -prometheus-metrics-core = { module = "io.prometheus:prometheus-metrics-core", version.ref = "prometheus-metrics" } -prometheus-metrics-exporter-servlet-jakarta = { module = "io.prometheus:prometheus-metrics-exporter-servlet-jakarta", version.ref = "prometheus-metrics" } prometheus-metrics-expositionformats = { module = "io.prometheus:prometheus-metrics-exposition-formats", version.ref = "prometheus-metrics" } prometheus-metrics-model = { module = "io.prometheus:prometheus-metrics-model", version.ref = "prometheus-metrics" } prometheus-simpleclient = { module = "io.prometheus:simpleclient", version.ref = "prometheus-simpleclient" } diff --git a/solr/cross-dc-manager/build.gradle b/solr/cross-dc-manager/build.gradle index 7d2637c44a7d..4ce538f67f7f 100644 --- a/solr/cross-dc-manager/build.gradle +++ b/solr/cross-dc-manager/build.gradle @@ -33,11 +33,6 @@ dependencies { implementation platform(libs.opentelemetry.bom) implementation libs.opentelemetry.api implementation libs.opentelemetry.sdk.metrics -// implementation libs.prometheus.metrics.model -// implementation(libs.prometheus.metrics.expositionformats, { -// exclude group: "io.prometheus", module: "prometheus-metrics-shaded-protobuf" -// exclude group: "io.prometheus", module: "prometheus-metrics-config" -// }) implementation libs.eclipse.jetty.server implementation libs.eclipse.jetty.ee10.servlet implementation libs.jakarta.servlet.api diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java index b59d335a0d37..19a8a4d115c1 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/consumer/KafkaCrossDcConsumer.java @@ -561,9 +561,9 @@ protected void processResult( break; case NOT_HANDLED_SHUTDOWN: if (log.isTraceEnabled()) { - log.trace("result=nothandled_shutdown"); + log.trace("result=unhandled_shutdown"); } - metrics.incrementOutputCounter(type.name(), "nothandled_shutdown"); + metrics.incrementOutputCounter(type.name(), "unhandled_shutdown"); break; case FAILED_RETRY: log.error( diff --git a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java index 14bb2a89e507..de3484ca45a6 100644 --- a/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java +++ b/solr/cross-dc-manager/src/java/org/apache/solr/crossdc/manager/messageprocessor/SolrMessageProcessor.java @@ -234,7 +234,7 @@ private Result> processMirroredSolrRequest( } if (status != 0) { - metrics.incrementOutputCounter(type.name(), "solrError"); + metrics.incrementOutputCounter(type.name(), "solr_error"); throw new SolrException(SolrException.ErrorCode.getErrorCode(status), "response=" + response); }