This is an automated email from the ASF dual-hosted git repository.
chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new c39e2af92c7 KAFKA-20408 Remove ConsumerMetrics wrapper (#21974)
c39e2af92c7 is described below
commit c39e2af92c7a3176669770903f28a984192cfaae
Author: Murali Basani <[email protected]>
AuthorDate: Tue Apr 7 19:51:47 2026 +0200
KAFKA-20408 Remove ConsumerMetrics wrapper (#21974)
https://issues.apache.org/jira/browse/KAFKA-20408
ConsumerMetrics which is redundant was referenced in
ClassicKafkaConsumer and AsyncKafkaConsumer
Updated those referenced classes, tests and build.gradle.
Tried running the script `./gradlew genConsumerMetricsDocs` and
generated consumer_metrics.html looks good.
Reviewers: Ken Huang <[email protected]>, Chia-Ping Tsai
<[email protected]>
---
build.gradle | 2 +-
.../consumer/internals/AsyncKafkaConsumer.java | 5 ++-
.../consumer/internals/ClassicKafkaConsumer.java | 4 +-
.../consumer/internals/ConsumerMetrics.java | 51 ----------------------
.../consumer/internals/FetchMetricsRegistry.java | 8 ++++
5 files changed, 14 insertions(+), 56 deletions(-)
diff --git a/build.gradle b/build.gradle
index 5c9d580f8be..7abd1b7c600 100644
--- a/build.gradle
+++ b/build.gradle
@@ -1232,7 +1232,7 @@ project(':core') {
task genConsumerMetricsDocs(type: JavaExec) {
classpath = sourceSets.test.runtimeClasspath
- mainClass = 'org.apache.kafka.clients.consumer.internals.ConsumerMetrics'
+ mainClass =
'org.apache.kafka.clients.consumer.internals.FetchMetricsRegistry'
if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() }
standardOutput = new File(generatedDocsDir,
"consumer_metrics.html").newOutputStream()
}
diff --git
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
index ec27794e6d9..17330650e29 100644
---
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
+++
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
@@ -148,6 +148,7 @@ import static
org.apache.kafka.clients.consumer.internals.AbstractMembershipMana
import static
org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED;
import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_JMX_PREFIX;
import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP;
+import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS;
import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED;
import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors;
@@ -680,8 +681,8 @@ public class AsyncKafkaConsumer<K, V> implements
ConsumerDelegate<K, V> {
this.deserializers = new Deserializers<>(keyDeserializer,
valueDeserializer, metrics);
this.clientTelemetryReporter = Optional.empty();
- ConsumerMetrics metricsRegistry = new ConsumerMetrics();
- this.fetchMetricsManager = new FetchMetricsManager(metrics,
metricsRegistry.fetcherMetrics);
+ FetchMetricsRegistry fetchMetricsRegistry = new
FetchMetricsRegistry(CONSUMER_METRIC_GROUP_PREFIX);
+ this.fetchMetricsManager = new FetchMetricsManager(metrics,
fetchMetricsRegistry);
this.fetchCollector = new FetchCollector<>(logContext,
metadata,
subscriptions,
diff --git
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java
index b013bc21f66..4851e54d8ac 100644
---
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java
+++
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java
@@ -378,8 +378,8 @@ public class ClassicKafkaConsumer<K, V> implements
ConsumerDelegate<K, V> {
int maxPollRecords =
config.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG);
boolean checkCrcs =
config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG);
- ConsumerMetrics metricsRegistry = new ConsumerMetrics();
- this.fetchMetricsManager = new FetchMetricsManager(metrics,
metricsRegistry.fetcherMetrics);
+ FetchMetricsRegistry fetchMetricsRegistry = new
FetchMetricsRegistry(CONSUMER_METRIC_GROUP_PREFIX);
+ this.fetchMetricsManager = new FetchMetricsManager(metrics,
fetchMetricsRegistry);
ApiVersions apiVersions = new ApiVersions();
FetchConfig fetchConfig = new FetchConfig(
minBytes,
diff --git
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetrics.java
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetrics.java
deleted file mode 100644
index 3aa0bbcfbcf..00000000000
---
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerMetrics.java
+++ /dev/null
@@ -1,51 +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.kafka.clients.consumer.internals;
-
-import org.apache.kafka.common.MetricNameTemplate;
-import org.apache.kafka.common.metrics.Metrics;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static
org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
-
-public class ConsumerMetrics {
-
- public FetchMetricsRegistry fetcherMetrics;
-
- public ConsumerMetrics(Set<String> metricsTags, String metricGrpPrefix) {
- this.fetcherMetrics = new FetchMetricsRegistry(metricsTags,
metricGrpPrefix);
- }
-
- public ConsumerMetrics() {
- this(new HashSet<>(), CONSUMER_METRIC_GROUP_PREFIX);
- }
-
- private List<MetricNameTemplate> getAllTemplates() {
- return new ArrayList<>(this.fetcherMetrics.getAllTemplates());
- }
-
- public static void main(String[] args) {
- Set<String> tags = new HashSet<>();
- tags.add("client-id");
- ConsumerMetrics metrics = new ConsumerMetrics(tags, "consumer");
- System.out.println(Metrics.toHtmlTable("kafka.consumer",
metrics.getAllTemplates()));
- }
-}
diff --git
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchMetricsRegistry.java
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchMetricsRegistry.java
index 589cb6736b3..e006cf3f391 100644
---
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchMetricsRegistry.java
+++
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchMetricsRegistry.java
@@ -17,6 +17,7 @@
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.common.MetricNameTemplate;
+import org.apache.kafka.common.metrics.Metrics;
import java.util.Arrays;
import java.util.HashSet;
@@ -148,6 +149,13 @@ public class FetchMetricsRegistry {
"The current read replica for the partition, or -1 if reading
from leader. " + DEPRECATED_TOPIC_METRICS_MESSAGE, partitionTags);
}
+ public static void main(String[] args) {
+ Set<String> tags = new HashSet<>();
+ tags.add("client-id");
+ FetchMetricsRegistry metrics = new FetchMetricsRegistry(tags,
"consumer");
+ System.out.println(Metrics.toHtmlTable("kafka.consumer",
metrics.getAllTemplates()));
+ }
+
public List<MetricNameTemplate> getAllTemplates() {
return Arrays.asList(
fetchSizeAvg,