From 2ba2522694afd1fcd1cb0b258147395720fd6f46 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 27 Aug 2024 13:48:13 -0400 Subject: [PATCH 01/54] Fix merge conflicts from breaking into smaller PR --- .../kafka/clients/admin/KafkaAdminClient.java | 32 +++++-- .../kafka/clients/consumer/Consumer.java | 2 + .../internals/AsyncKafkaConsumer.java | 14 +++- .../internals/ClassicKafkaConsumer.java | 14 +++- .../StreamsDelegatingMetricsReporter.java | 84 +++++++++++++++++++ .../processor/internals/StreamThread.java | 24 ++++++ 6 files changed, 155 insertions(+), 15 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 30fdac4687d96..56e81750f1691 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -254,6 +254,7 @@ import org.apache.kafka.common.security.scram.internals.ScramFormatter; import org.apache.kafka.common.security.token.delegation.DelegationToken; import org.apache.kafka.common.security.token.delegation.TokenInformation; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.KafkaThread; @@ -409,6 +410,7 @@ public class KafkaAdminClient extends AdminClient { private final boolean clientTelemetryEnabled; private final MetadataRecoveryStrategy metadataRecoveryStrategy; private final AdminFetchMetricsManager adminFetchMetricsManager; + private final Optional clientTelemetryReporter; /** * The telemetry requests client instance id. @@ -529,6 +531,7 @@ static KafkaAdminClient createInternal( String clientId = generateClientId(config); ApiVersions apiVersions = new ApiVersions(); LogContext logContext = createLogContext(clientId); + Optional clientTelemetryReporter; try { // Since we only request node information, it's safe to pass true for allowAutoTopicCreation (and it @@ -548,6 +551,8 @@ static KafkaAdminClient createInternal( MetricsContext metricsContext = new KafkaMetricsContext(JMX_PREFIX, config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); + clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); + clientTelemetryReporter.ifPresent(telemetryReporter -> telemetryReporter.contextChange(metricsContext)); networkClient = ClientUtils.createNetworkClient(config, clientId, metrics, @@ -557,10 +562,13 @@ static KafkaAdminClient createInternal( time, 1, (int) TimeUnit.HOURS.toMillis(1), + null, metadataManager.updater(), - (hostResolver == null) ? new DefaultHostResolver() : hostResolver); + (hostResolver == null) ? new DefaultHostResolver() : hostResolver, + null, + clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, - timeoutProcessorFactory, logContext); + timeoutProcessorFactory, logContext, clientTelemetryReporter); } catch (Throwable exc) { closeQuietly(metrics, "Metrics"); closeQuietly(networkClient, "NetworkClient"); @@ -580,7 +588,7 @@ static KafkaAdminClient createInternal(AdminClientConfig config, metrics = new Metrics(new MetricConfig(), new LinkedList<>(), time); LogContext logContext = createLogContext(clientId); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, - client, null, logContext); + client, null, logContext, null); } catch (Throwable exc) { closeQuietly(metrics, "Metrics"); throw new KafkaException("Failed to create new KafkaAdminClient", exc); @@ -598,7 +606,8 @@ private KafkaAdminClient(AdminClientConfig config, Metrics metrics, KafkaClient client, TimeoutProcessorFactory timeoutProcessorFactory, - LogContext logContext) { + LogContext logContext, + Optional clientTelemetryReporter) { this.clientId = clientId; this.log = logContext.logger(KafkaAdminClient.class); this.logContext = logContext; @@ -622,6 +631,9 @@ private KafkaAdminClient(AdminClientConfig config, retryBackoffMaxMs, CommonClientConfigs.RETRY_BACKOFF_JITTER); this.clientTelemetryEnabled = config.getBoolean(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG); + List reporters = CommonClientConfigs.metricsReporters(this.clientId, config); + this.clientTelemetryReporter = clientTelemetryReporter; + this.clientTelemetryReporter.ifPresent(reporters::add); this.metadataRecoveryStrategy = MetadataRecoveryStrategy.forName(config.getString(AdminClientConfig.METADATA_RECOVERY_STRATEGY_CONFIG)); this.adminFetchMetricsManager = new AdminFetchMetricsManager(metrics); config.logUnused(); @@ -4272,12 +4284,18 @@ private KafkaFutureImpl> getMembersFromGroup(String groupId @Override public void registerMetricForSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricChange(metric); + } } @Override - public void unregisterMetricFromSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + public void unregisterMetricForSubscription(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricRemoval(metric); + } } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 4201395578390..5815f8dbff336 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -117,6 +117,8 @@ public interface Consumer extends Closeable { */ void commitAsync(Map offsets, OffsetCommitCallback callback); + void registerMetric(KafkaMetric metric); + void unregisterMetric(KafkaMetric metric); /** * @see KafkaConsumer#registerMetricForSubscription(KafkaMetric) */ 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 fccd69c86b851..3f2aba3e86111 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 @@ -646,13 +646,19 @@ private void updateGroupMetadata(final Optional memberEpoch, final Stri } @Override - public void registerMetricForSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + public void registerMetric(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricChange(metric); + } } @Override - public void unregisterMetricFromSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + public void unregisterMetric(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricRemoval(metric); + } } /** 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 4a732f0aeb04a..c94a8e76d7c81 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 @@ -429,13 +429,19 @@ public void subscribe(Collection topics, ConsumerRebalanceListener liste @Override - public void registerMetricForSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + public void registerMetric(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricChange(metric); + } } @Override - public void unregisterMetricFromSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + public void unregisterMetric(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricRemoval(metric); + } } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java new file mode 100644 index 0000000000000..f112eed882aea --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java @@ -0,0 +1,84 @@ +/* + * 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.streams.internals.metrics; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsReporter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class StreamsDelegatingMetricsReporter implements MetricsReporter { + + + private static final Logger LOG = LoggerFactory.getLogger(org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter.class); + private Consumer consumer; + final String threadId; + private static final String THREAD_ID_TAG = "thread-id"; + + + public StreamsDelegatingMetricsReporter(final Consumer consumer, final String threadId) { + this.consumer = consumer; + this.threadId = threadId; + } + + @Override + public void init(final List metrics) { + + } + + @Override + public void metricChange(final KafkaMetric metric) { + if (filteredMetric(metric).isPresent()) { + LOG.info("Registering metric {} for thread={}", metric.metricName().name(), threadId); + consumer.registerMetric(metric); + } + } + + Optional filteredMetric(final KafkaMetric kafkaMetric) { + final Map tags = kafkaMetric.metricName().tags(); + KafkaMetric maybeKafkaMetric = null; + if (!tags.containsKey(THREAD_ID_TAG) || tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { + maybeKafkaMetric = kafkaMetric; + } + return Optional.ofNullable(maybeKafkaMetric); + } + + @Override + public void metricRemoval(final KafkaMetric metric) { + if (filteredMetric(metric).isPresent()) { + LOG.info("Unregistering metric {} for thread={}", metric.metricName().name(), threadId); + consumer.unregisterMetric(metric); + } + } + + @Override + public void close() { + this.consumer = null; + } + + @Override + public void configure(final Map configs) { + + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index e45021f25c374..5588fb8ba28b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -33,6 +33,8 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; @@ -48,6 +50,7 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.internals.metrics.ClientMetrics; +import org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter; import org.apache.kafka.streams.processor.StandbyUpdateListener; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; @@ -63,11 +66,14 @@ import org.slf4j.Logger; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Queue; @@ -644,6 +650,24 @@ public StreamThread(final Time time, this.stateUpdaterEnabled = InternalConfig.stateUpdaterEnabled(config.originals()); this.processingThreadsEnabled = InternalConfig.processingThreadsEnabled(config.originals()); this.logSummaryIntervalMs = config.getLong(StreamsConfig.LOG_SUMMARY_INTERVAL_MS_CONFIG); + + + final StreamsDelegatingMetricsReporter reporter = new StreamsDelegatingMetricsReporter(mainConsumer, threadId); + final Metrics metricsRegistry = streamsMetrics.metricsRegistry(); + metricsRegistry.reporters().add(reporter); + metricsRegistry.metrics().values().forEach(reporter::metricChange); + } + + private Collection filterMetricsForCurrentThread(final String threadId, + final Map metrics) { + final List filteredMetrics = new ArrayList<>(); + for (final Map.Entry entry : metrics.entrySet()) { + final Map tags = entry.getKey().tags(); + if (!tags.containsKey("thread-id") || tags.containsKey("thread-id") && tags.get("thread-id").contains(threadId)) { + filteredMetrics.add(entry.getValue()); + } + } + return filteredMetrics; } private static final class InternalConsumerConfig extends ConsumerConfig { From 2701efe0c24baa63f96f2f247dc788c44f0ba330 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 27 Aug 2024 15:11:25 -0400 Subject: [PATCH 02/54] Resolving merge conflics --- .../kafka/clients/admin/ForwardingAdmin.java | 2 +- .../kafka/clients/consumer/Consumer.java | 7 ++ .../kafka/clients/consumer/KafkaConsumer.java | 25 ++++++ .../StreamsDelegatingMetricsReporter.java | 76 +++++++++---------- 4 files changed, 71 insertions(+), 39 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java b/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java index 87e350c5e7ac5..6c6ff6d4c26d6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java @@ -320,7 +320,7 @@ public void registerMetricForSubscription(KafkaMetric metric) { } @Override - public void unregisterMetricFromSubscription(KafkaMetric metric) { + public void unregisterMetricForSubscription(KafkaMetric metric) { throw new UnsupportedOperationException(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 5815f8dbff336..20d08302b0171 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -117,7 +117,14 @@ public interface Consumer extends Closeable { */ void commitAsync(Map offsets, OffsetCommitCallback callback); + /** + * @see KafkaConsumer#registerMetric(KafkaMetric) + */ void registerMetric(KafkaMetric metric); + + /** + * @see KafkaConsumer#unregisterMetric(KafkaMetric) + */ void unregisterMetric(KafkaMetric metric); /** * @see KafkaConsumer#registerMetricForSubscription(KafkaMetric) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 1b12ae3d4b47b..d59c96a37db85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1427,6 +1427,31 @@ public void resume(Collection partitions) { delegate.resume(partitions); } + /** + * An application metric provided for subscription. + * This metric will be added to this client's metrics + * that are available for subscription and sent as + * telemetry data to the broker. + * + * @param metric, the application metric to register + */ + @Override + public void registerMetric(KafkaMetric metric) { + delegate.registerMetric(metric); + } + + /** + * An application to be removed from subscription. + * This metric is removed from this client's metrics + * and will not be available for subscription. + * + * @param metric, the application metric to remove + */ + @Override + public void unregisterMetric(KafkaMetric metric) { + delegate.unregisterMetric(metric); + } + /** * Get the set of partitions that were previously paused by a call to {@link #pause(Collection)}. * diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java index f112eed882aea..6e1a8dc6a9163 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java @@ -31,54 +31,54 @@ public class StreamsDelegatingMetricsReporter implements MetricsReporter { - private static final Logger LOG = LoggerFactory.getLogger(org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter.class); - private Consumer consumer; - final String threadId; - private static final String THREAD_ID_TAG = "thread-id"; + private static final Logger LOG = LoggerFactory.getLogger(org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter.class); + private Consumer consumer; + final String threadId; + private static final String THREAD_ID_TAG = "thread-id"; - public StreamsDelegatingMetricsReporter(final Consumer consumer, final String threadId) { - this.consumer = consumer; - this.threadId = threadId; - } + public StreamsDelegatingMetricsReporter(final Consumer consumer, final String threadId) { + this.consumer = consumer; + this.threadId = threadId; + } - @Override - public void init(final List metrics) { + @Override + public void init(final List metrics) { - } + } - @Override - public void metricChange(final KafkaMetric metric) { - if (filteredMetric(metric).isPresent()) { - LOG.info("Registering metric {} for thread={}", metric.metricName().name(), threadId); - consumer.registerMetric(metric); - } + @Override + public void metricChange(final KafkaMetric metric) { + if (filteredMetric(metric).isPresent()) { + LOG.info("Registering metric {} for thread={}", metric.metricName().name(), threadId); + consumer.registerMetric(metric); } + } - Optional filteredMetric(final KafkaMetric kafkaMetric) { - final Map tags = kafkaMetric.metricName().tags(); - KafkaMetric maybeKafkaMetric = null; - if (!tags.containsKey(THREAD_ID_TAG) || tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { - maybeKafkaMetric = kafkaMetric; - } - return Optional.ofNullable(maybeKafkaMetric); + Optional filteredMetric(final KafkaMetric kafkaMetric) { + final Map tags = kafkaMetric.metricName().tags(); + KafkaMetric maybeKafkaMetric = null; + if (!tags.containsKey(THREAD_ID_TAG) || tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { + maybeKafkaMetric = kafkaMetric; } - - @Override - public void metricRemoval(final KafkaMetric metric) { - if (filteredMetric(metric).isPresent()) { - LOG.info("Unregistering metric {} for thread={}", metric.metricName().name(), threadId); - consumer.unregisterMetric(metric); - } + return Optional.ofNullable(maybeKafkaMetric); + } + + @Override + public void metricRemoval(final KafkaMetric metric) { + if (filteredMetric(metric).isPresent()) { + LOG.info("Unregistering metric {} for thread={}", metric.metricName().name(), threadId); + consumer.unregisterMetric(metric); } + } - @Override - public void close() { - this.consumer = null; - } + @Override + public void close() { + this.consumer = null; + } - @Override - public void configure(final Map configs) { + @Override + public void configure(final Map configs) { - } + } } From 56b912d79e28dd272b092dedea80aa31970d2df0 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 27 Aug 2024 15:54:53 -0400 Subject: [PATCH 03/54] Fixing rebase --- .../kafka/clients/producer/KafkaProducer.java | 33 +++++++++++++++++++ .../kafka/clients/producer/MockProducer.java | 11 +++++++ .../kafka/clients/producer/Producer.java | 11 +++++++ 3 files changed, 55 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 0b290318b0bd0..76438323d07f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -57,6 +57,7 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.internals.ClusterResourceListeners; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.KafkaMetricsContext; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -1300,6 +1301,38 @@ public List partitionsFor(String topic) { return Collections.unmodifiableMap(this.metrics.metrics()); } + + /** + * An application metric provided for subscription. + * This metric will be added to this client's metrics + * that are available for subscription and sent as + * telemetry data to the broker. + * + * @param metric, the application metric to register + */ + @Override + public void registerMetricForSubscription(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricChange(metric); + } + } + + /** + * An application to be removed from subscription. + * This metric is removed from this client's metrics + * and will not be available for subscription. + * + * @param metric, the application metric to remove + */ + @Override + public void unregisterMetricFromSubscription(KafkaMetric metric) { + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricRemoval(metric); + } + } + /** * Determines the client's unique client instance ID used for telemetry. This ID is unique to * this specific client instance and will not change after it is initially generated. diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 971ea0194c397..b6fab9bc4377f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Time; @@ -71,6 +72,7 @@ public class MockProducer implements Producer { private boolean producerFenced; private boolean sentOffsets; private long commitCount = 0L; + final List addedMetrics = new ArrayList<>(); public RuntimeException initTransactionException = null; public RuntimeException beginTransactionException = null; @@ -607,4 +609,13 @@ public void complete(RuntimeException e) { } } + @Override + public void registerMetricForSubscription(KafkaMetric metric) { + addedMetrics.add(metric); + } + + @Override + public void unregisterMetricFromSubscription(KafkaMetric metric) { + addedMetrics.remove(metric); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 2579fa0eeb9f2..87e9d6042eeb8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.metrics.KafkaMetric; import java.io.Closeable; import java.time.Duration; @@ -71,6 +72,16 @@ void sendOffsetsToTransaction(Map offsets, */ void abortTransaction() throws ProducerFencedException; + /** + * @see KafkaProducer#registerMetricForSubscription(KafkaMetric) + */ + void registerMetricForSubscription(KafkaMetric metric); + + /** + * @see KafkaProducer#unregisterMetricFromSubscription(KafkaMetric) + */ + void unregisterMetricFromSubscription(KafkaMetric metric); + /** * See {@link KafkaProducer#send(ProducerRecord)} */ From 3eca4eb8a00d3dd47ad291c69129ae24cc0d70d7 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 27 Aug 2024 16:27:59 -0400 Subject: [PATCH 04/54] Fix producer javadoc comment Correct metric filtering to only include metrics with thread-id Remove unused method --- .../apache/kafka/clients/producer/KafkaProducer.java | 2 +- .../metrics/StreamsDelegatingMetricsReporter.java | 2 +- .../streams/processor/internals/StreamThread.java | 12 ------------ 3 files changed, 2 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 76438323d07f2..9e40441b03645 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1319,7 +1319,7 @@ public void registerMetricForSubscription(KafkaMetric metric) { } /** - * An application to be removed from subscription. + * An application metric to be removed from subscription. * This metric is removed from this client's metrics * and will not be available for subscription. * diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java index 6e1a8dc6a9163..0cd42d3cfb4ce 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java @@ -58,7 +58,7 @@ public void metricChange(final KafkaMetric metric) { Optional filteredMetric(final KafkaMetric kafkaMetric) { final Map tags = kafkaMetric.metricName().tags(); KafkaMetric maybeKafkaMetric = null; - if (!tags.containsKey(THREAD_ID_TAG) || tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { + if (tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { maybeKafkaMetric = kafkaMetric; } return Optional.ofNullable(maybeKafkaMetric); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 5588fb8ba28b3..b56fe3d528757 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -658,18 +658,6 @@ public StreamThread(final Time time, metricsRegistry.metrics().values().forEach(reporter::metricChange); } - private Collection filterMetricsForCurrentThread(final String threadId, - final Map metrics) { - final List filteredMetrics = new ArrayList<>(); - for (final Map.Entry entry : metrics.entrySet()) { - final Map tags = entry.getKey().tags(); - if (!tags.containsKey("thread-id") || tags.containsKey("thread-id") && tags.get("thread-id").contains(threadId)) { - filteredMetrics.add(entry.getValue()); - } - } - return filteredMetrics; - } - private static final class InternalConsumerConfig extends ConsumerConfig { private InternalConsumerConfig(final Map props) { super(ConsumerConfig.appendDeserializerToConfig(props, new ByteArrayDeserializer(), From d1ed133f248d060d03ef39e4054541c85d10d11d Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 28 Aug 2024 11:25:28 -0400 Subject: [PATCH 05/54] Fix spotbugs and checkstyle errors --- .../kafka/clients/admin/KafkaAdminClient.java | 2 +- .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../kafka/clients/producer/KafkaProducer.java | 8 ++--- .../kafka/clients/producer/MockProducer.java | 31 +++++++++---------- .../processor/internals/StreamThread.java | 4 --- 5 files changed, 21 insertions(+), 26 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 56e81750f1691..a5a55a6e94877 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -588,7 +588,7 @@ static KafkaAdminClient createInternal(AdminClientConfig config, metrics = new Metrics(new MetricConfig(), new LinkedList<>(), time); LogContext logContext = createLogContext(clientId); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, - client, null, logContext, null); + client, null, logContext, Optional.empty()); } catch (Throwable exc) { closeQuietly(metrics, "Metrics"); throw new KafkaException("Failed to create new KafkaAdminClient", exc); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d59c96a37db85..aef22230c6a9a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1449,7 +1449,7 @@ public void registerMetric(KafkaMetric metric) { */ @Override public void unregisterMetric(KafkaMetric metric) { - delegate.unregisterMetric(metric); + delegate.unregisterMetric(metric); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 9e40441b03645..8859fda406927 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1327,10 +1327,10 @@ public void registerMetricForSubscription(KafkaMetric metric) { */ @Override public void unregisterMetricFromSubscription(KafkaMetric metric) { - if (clientTelemetryReporter.isPresent()) { - ClientTelemetryReporter reporter = clientTelemetryReporter.get(); - reporter.metricRemoval(metric); - } + if (clientTelemetryReporter.isPresent()) { + ClientTelemetryReporter reporter = clientTelemetryReporter.get(); + reporter.metricRemoval(metric); + } } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index b6fab9bc4377f..f4b034a4bc88b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -90,13 +90,13 @@ public class MockProducer implements Producer { /** * Create a mock producer * - * @param cluster The cluster holding metadata for this producer - * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise - * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after - * {@link #send(ProducerRecord) send()} to complete the call and unblock the {@link - * java.util.concurrent.Future Future<RecordMetadata>} that is returned. - * @param partitioner The partition strategy - * @param keySerializer The serializer for key that implements {@link Serializer}. + * @param cluster The cluster holding metadata for this producer + * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise + * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after + * {@link #send(ProducerRecord) send()} to complete the call and unblock the {@link + * java.util.concurrent.Future Future<RecordMetadata>} that is returned. + * @param partitioner The partition strategy + * @param keySerializer The serializer for key that implements {@link Serializer}. * @param valueSerializer The serializer for value that implements {@link Serializer}. */ public MockProducer(final Cluster cluster, @@ -120,7 +120,7 @@ public MockProducer(final Cluster cluster, /** * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers. - * + *

* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ @SuppressWarnings("deprecation") @@ -132,7 +132,7 @@ public MockProducer(final boolean autoComplete, /** * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers. - * + *

* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(cluster, autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ @SuppressWarnings("deprecation") @@ -145,7 +145,7 @@ public MockProducer(final Cluster cluster, /** * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers. - * + *

* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)} */ public MockProducer(final boolean autoComplete, @@ -157,7 +157,7 @@ public MockProducer(final boolean autoComplete, /** * Create a new mock producer with invented metadata. - * + *

* Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), false, null, null, null)} */ public MockProducer() { @@ -226,7 +226,7 @@ public void sendOffsetsToTransaction(Map offs return; } Map uncommittedOffsets = - this.uncommittedConsumerGroupOffsets.computeIfAbsent(groupMetadata.groupId(), k -> new HashMap<>()); + this.uncommittedConsumerGroupOffsets.computeIfAbsent(groupMetadata.groupId(), k -> new HashMap<>()); uncommittedOffsets.putAll(offsets); this.sentOffsets = true; } @@ -336,7 +336,7 @@ public synchronized Future send(ProducerRecord record, Cal keySerializer.serialize(record.topic(), record.key()); valueSerializer.serialize(record.topic(), record.value()); } - + TopicPartition topicPartition = new TopicPartition(record.topic(), partition); ProduceRequestResult result = new ProduceRequestResult(topicPartition); FutureRecordMetadata future = new FutureRecordMetadata(result, 0, RecordBatch.NO_TIMESTAMP, @@ -504,7 +504,6 @@ public synchronized List> uncommittedRecords() { } /** - * * Get the list of committed consumer group offsets since the last call to {@link #clear()} */ public synchronized List>> consumerGroupOffsetsHistory() { @@ -611,11 +610,11 @@ public void complete(RuntimeException e) { @Override public void registerMetricForSubscription(KafkaMetric metric) { - addedMetrics.add(metric); + addedMetrics.add(metric); } @Override public void unregisterMetricFromSubscription(KafkaMetric metric) { - addedMetrics.remove(metric); + addedMetrics.remove(metric); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index b56fe3d528757..dc5896b2e8959 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -33,7 +33,6 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -66,14 +65,11 @@ import org.slf4j.Logger; import java.time.Duration; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Queue; From 17f78082410a9651c5685f543dbebd516f7db7af Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 28 Aug 2024 18:11:30 -0400 Subject: [PATCH 06/54] Simplify filtering for adding metrics --- .../metrics/StreamsDelegatingMetricsReporter.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java index 0cd42d3cfb4ce..7dd3438f8d51e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Map; -import java.util.Optional; public class StreamsDelegatingMetricsReporter implements MetricsReporter { @@ -49,24 +48,20 @@ public void init(final List metrics) { @Override public void metricChange(final KafkaMetric metric) { - if (filteredMetric(metric).isPresent()) { + if (tagMatchesCurrentThread(metric)) { LOG.info("Registering metric {} for thread={}", metric.metricName().name(), threadId); consumer.registerMetric(metric); } } - Optional filteredMetric(final KafkaMetric kafkaMetric) { + boolean tagMatchesCurrentThread(final KafkaMetric kafkaMetric) { final Map tags = kafkaMetric.metricName().tags(); - KafkaMetric maybeKafkaMetric = null; - if (tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId)) { - maybeKafkaMetric = kafkaMetric; - } - return Optional.ofNullable(maybeKafkaMetric); + return tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId); } @Override public void metricRemoval(final KafkaMetric metric) { - if (filteredMetric(metric).isPresent()) { + if (tagMatchesCurrentThread(metric)) { LOG.info("Unregistering metric {} for thread={}", metric.metricName().name(), threadId); consumer.unregisterMetric(metric); } From 852fbfa7d7cee3af765ab559f0cd50e47ef4d9cd Mon Sep 17 00:00:00 2001 From: bbejeck Date: Thu, 29 Aug 2024 11:08:06 -0400 Subject: [PATCH 07/54] Clean up the creation of the reporter such that it relys on the metrics infra totally for adding removing metrics. --- ...a => StreamsThreadDelegatingMetricsReporter.java} | 6 +++--- .../streams/processor/internals/StreamThread.java | 12 ++++-------- 2 files changed, 7 insertions(+), 11 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/internals/metrics/{StreamsDelegatingMetricsReporter.java => StreamsThreadDelegatingMetricsReporter.java} (88%) diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java similarity index 88% rename from streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java rename to streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 7dd3438f8d51e..8eb7ce76b413f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -27,16 +27,16 @@ import java.util.List; import java.util.Map; -public class StreamsDelegatingMetricsReporter implements MetricsReporter { +public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { - private static final Logger LOG = LoggerFactory.getLogger(org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter.class); + private static final Logger LOG = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); private Consumer consumer; final String threadId; private static final String THREAD_ID_TAG = "thread-id"; - public StreamsDelegatingMetricsReporter(final Consumer consumer, final String threadId) { + public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId) { this.consumer = consumer; this.threadId = threadId; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index dc5896b2e8959..a30c5f14c59c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -33,7 +33,6 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; @@ -49,7 +48,7 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.internals.metrics.ClientMetrics; -import org.apache.kafka.streams.internals.metrics.StreamsDelegatingMetricsReporter; +import org.apache.kafka.streams.internals.metrics.StreamsThreadDelegatingMetricsReporter; import org.apache.kafka.streams.processor.StandbyUpdateListener; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; @@ -474,6 +473,9 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, final Consumer mainConsumer = clientSupplier.getConsumer(consumerConfigs); taskManager.setMainConsumer(mainConsumer); referenceContainer.mainConsumer = mainConsumer; + + final StreamsThreadDelegatingMetricsReporter reporter = new StreamsThreadDelegatingMetricsReporter(mainConsumer, threadId); + streamsMetrics.metricsRegistry().addReporter(reporter); final StreamThread streamThread = new StreamThread( time, @@ -646,12 +648,6 @@ public StreamThread(final Time time, this.stateUpdaterEnabled = InternalConfig.stateUpdaterEnabled(config.originals()); this.processingThreadsEnabled = InternalConfig.processingThreadsEnabled(config.originals()); this.logSummaryIntervalMs = config.getLong(StreamsConfig.LOG_SUMMARY_INTERVAL_MS_CONFIG); - - - final StreamsDelegatingMetricsReporter reporter = new StreamsDelegatingMetricsReporter(mainConsumer, threadId); - final Metrics metricsRegistry = streamsMetrics.metricsRegistry(); - metricsRegistry.reporters().add(reporter); - metricsRegistry.metrics().values().forEach(reporter::metricChange); } private static final class InternalConsumerConfig extends ConsumerConfig { From 84b201e0ec363e445197f648985b5395414d8664 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Fri, 30 Aug 2024 19:01:04 -0400 Subject: [PATCH 08/54] Resolving merge conflict from this previous commit Added parameterized integration test --- .../KafkaStreamsTelemetryIntegrationTest.java | 268 ++++++++++++++++++ ...treamsThreadDelegatingMetricsReporter.java | 32 ++- .../processor/internals/StreamThread.java | 5 +- 3 files changed, 291 insertions(+), 14 deletions(-) create mode 100644 streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java new file mode 100644 index 0000000000000..2da84acdda220 --- /dev/null +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -0,0 +1,268 @@ +/* + * 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.streams.integration; + + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KafkaClientSupplier; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Timeout(600) +@Tag("integration") +public class KafkaStreamsTelemetryIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); + private static final int NUM_BROKERS = 1; + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + + private String appId; + private String inputTopicTwoPartitions; + private String outputTopicTwoPartitions; + private String inputTopicOnePartition; + private String outputTopicOnePartition; + private final List streamsConfigurations = new ArrayList<>(); + private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); + + @BeforeAll + public static void startCluster() throws IOException { + CLUSTER.start(); + } + + @BeforeEach + public void setUp(final TestInfo testInfo) throws InterruptedException { + appId = safeUniqueTestName(testInfo); + inputTopicTwoPartitions = appId + "-input-two"; + outputTopicTwoPartitions = appId + "-output-two"; + inputTopicOnePartition = appId + "-input-one"; + outputTopicOnePartition = appId + "-output-one"; + CLUSTER.createTopic(inputTopicTwoPartitions, 2, 1); + CLUSTER.createTopic(outputTopicTwoPartitions, 2, 1); + CLUSTER.createTopic(inputTopicOnePartition, 1, 1); + CLUSTER.createTopic(outputTopicOnePartition, 1, 1); + } + + @AfterAll + public static void closeCluster() { + CLUSTER.stop(); + } + + @AfterEach + public void tearDown() throws Exception { + INTERCEPTING_CONSUMERS.clear(); + IntegrationTestUtils.purgeLocalStreamsState(streamsConfigurations); + streamsConfigurations.clear(); + } + + + @ParameterizedTest + @MethodSource("provideStreamParameters") + @DisplayName("Streams metrics should get passed to Consumer") + void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { + final Properties properties = props(stateUpdaterEnabled); + final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); + + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + streams.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams never transitioned to a RUNNING state."); + + final List streamsThreadMetrics = streams.metrics().values().stream().filter(m -> m.metricName().tags().containsKey("thread-id")) + .map(m -> m.metricName().name()).sorted().collect(Collectors.toList()); + + final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(0).addedMetrics.stream().map(m -> m.metricName().name()).sorted().collect(Collectors.toList()); + if (consumerPassedStreamMetricNames.size() > streamsThreadMetrics.size()) { + log.info("Streams metrics size={} passed consumer size={}", streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); + log.info("Unfiltered streams metrics size={}", streams.metrics().values().size()); + final List consumerPassedCopy = new ArrayList<>(consumerPassedStreamMetricNames); + log.info("Consumer passed copy size {}", consumerPassedCopy.size()); + consumerPassedCopy.removeAll(streamsThreadMetrics); + log.info("EXTRA metric names {}", consumerPassedCopy); + } + assertEquals(streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); + assertEquals(streamsThreadMetrics, consumerPassedStreamMetricNames); + } + } + + private static Stream provideStreamParameters() { + return Stream.of(Arguments.of("simple", true), + Arguments.of("simple", false), + Arguments.of("complex", true), + Arguments.of("complex", false)); + } + + + private Properties props() { + return props(new Properties()); + } + private Properties props(final boolean stateUpdaterEnabled) { + return props(mkObjectProperties(mkMap(mkEntry(StreamsConfig.InternalConfig.STATE_UPDATER_ENABLED, stateUpdaterEnabled)))); + } + private Properties props(final Properties extraProperties) { + final Properties streamsConfiguration = new Properties(); + + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); + streamsConfiguration.put(StreamsConfig.DEFAULT_CLIENT_SUPPLIER_CONFIG, TestClientSupplier.class); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.putAll(extraProperties); + + streamsConfigurations.add(streamsConfiguration); + + return streamsConfiguration; + } + + private Topology complexTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream(inputTopicTwoPartitions, Consumed.with(Serdes.String(), Serdes.String())) + .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .groupBy((key, value) -> value) + .count() + .toStream().to(outputTopicTwoPartitions, Produced.with(Serdes.String(), Serdes.Long())); + return builder.build(); + } + + private Topology simpleTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + builder.stream(inputTopicOnePartition, Consumed.with(Serdes.String(), Serdes.String())) + .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .to(outputTopicOnePartition, Produced.with(Serdes.String(), Serdes.String())); + return builder.build(); + } + + + public static class TestClientSupplier implements KafkaClientSupplier { + + @Override + public Producer getProducer(final Map config) { + return new KafkaProducer<>(config, new ByteArraySerializer(), new ByteArraySerializer()); + } + + @Override + public Consumer getConsumer(final Map config) { + final MetricsInterceptingConsumer consumer = new MetricsInterceptingConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + INTERCEPTING_CONSUMERS.add(consumer); + return consumer; + } + + @Override + public Consumer getRestoreConsumer(final Map config) { + return new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + + @Override + public Consumer getGlobalConsumer(final Map config) { + return new KafkaConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + + @Override + public Admin getAdmin(final Map config) { + return AdminClient.create(config); + } + } + + public static class MetricsInterceptingConsumer extends KafkaConsumer { + + public List addedMetrics = new ArrayList<>(); + public List removedMetrics = new ArrayList<>(); + + public MetricsInterceptingConsumer(final Map configs) { + super(configs); + } + + public MetricsInterceptingConsumer(final Properties properties) { + super(properties); + } + + public MetricsInterceptingConsumer(final Properties properties, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { + super(properties, keyDeserializer, valueDeserializer); + } + + public MetricsInterceptingConsumer(final Map configs, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { + super(configs, keyDeserializer, valueDeserializer); + } + + @Override + public void registerMetric(final KafkaMetric metric) { + addedMetrics.add(metric); + super.registerMetric(metric); + } + + @Override + public void unregisterMetric(final KafkaMetric metric) { + removedMetrics.add(metric); + super.unregisterMetric(metric); + } + } + + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 8eb7ce76b413f..2e9e72d5a9241 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { @@ -33,36 +34,43 @@ public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { private static final Logger LOG = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); private Consumer consumer; final String threadId; + final String stateUpdaterThreadId; private static final String THREAD_ID_TAG = "thread-id"; - public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId) { - this.consumer = consumer; - this.threadId = threadId; + public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId, final String stateUpdaterThreadId) { + this.consumer = Objects.requireNonNull(consumer); + this.threadId = Objects.requireNonNull(threadId); + this.stateUpdaterThreadId = Objects.requireNonNull(stateUpdaterThreadId); + LOG.info("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); } @Override public void init(final List metrics) { - + metrics.forEach(this::metricChange); } @Override public void metricChange(final KafkaMetric metric) { - if (tagMatchesCurrentThread(metric)) { - LOG.info("Registering metric {} for thread={}", metric.metricName().name(), threadId); + if (tagMatchStreamOrStateUpdaterThreadId(metric)) { + LOG.info("Registering metric {}", metric.metricName()); consumer.registerMetric(metric); } } - boolean tagMatchesCurrentThread(final KafkaMetric kafkaMetric) { - final Map tags = kafkaMetric.metricName().tags(); - return tags.containsKey(THREAD_ID_TAG) && tags.get(THREAD_ID_TAG).equals(threadId); + boolean tagMatchStreamOrStateUpdaterThreadId(final KafkaMetric metric) { + final Map tags = metric.metricName().tags(); + final boolean shouldInclude = tags.containsKey(THREAD_ID_TAG) && (tags.get(THREAD_ID_TAG).equals(threadId) || tags.get(THREAD_ID_TAG).equals(stateUpdaterThreadId)); + if (!shouldInclude) { + LOG.warn("Rejecting metric {}", metric.metricName()); + } + return shouldInclude; } @Override public void metricRemoval(final KafkaMetric metric) { - if (tagMatchesCurrentThread(metric)) { - LOG.info("Unregistering metric {} for thread={}", metric.metricName().name(), threadId); + if (tagMatchStreamOrStateUpdaterThreadId(metric)) { + LOG.info("Unregistering metric {}", metric.metricName()); consumer.unregisterMetric(metric); } } @@ -74,6 +82,6 @@ public void close() { @Override public void configure(final Map configs) { - + // No op } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index a30c5f14c59c1..64137850a0999 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -473,8 +473,9 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, final Consumer mainConsumer = clientSupplier.getConsumer(consumerConfigs); taskManager.setMainConsumer(mainConsumer); referenceContainer.mainConsumer = mainConsumer; - - final StreamsThreadDelegatingMetricsReporter reporter = new StreamsThreadDelegatingMetricsReporter(mainConsumer, threadId); + + final String stateUpdaterId = stateUpdaterEnabled ? threadId.replace("-StreamThread-", "-StateUpdater-") : "NA"; + final StreamsThreadDelegatingMetricsReporter reporter = new StreamsThreadDelegatingMetricsReporter(mainConsumer, threadId, stateUpdaterId); streamsMetrics.metricsRegistry().addReporter(reporter); final StreamThread streamThread = new StreamThread( From 2c9d54852cb4c4dfb628647d44481897956b5327 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 2 Sep 2024 17:31:42 -0400 Subject: [PATCH 09/54] Added test for passing streams metrics --- .../KafkaStreamsTelemetryIntegrationTest.java | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2da84acdda220..ae36918992239 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -25,6 +25,8 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -71,6 +73,7 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(600) @Tag("integration") @@ -132,20 +135,13 @@ void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnab IntegrationTestUtils.DEFAULT_TIMEOUT, () -> "Kafka Streams never transitioned to a RUNNING state."); - final List streamsThreadMetrics = streams.metrics().values().stream().filter(m -> m.metricName().tags().containsKey("thread-id")) - .map(m -> m.metricName().name()).sorted().collect(Collectors.toList()); - - final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(0).addedMetrics.stream().map(m -> m.metricName().name()).sorted().collect(Collectors.toList()); - if (consumerPassedStreamMetricNames.size() > streamsThreadMetrics.size()) { - log.info("Streams metrics size={} passed consumer size={}", streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); - log.info("Unfiltered streams metrics size={}", streams.metrics().values().size()); - final List consumerPassedCopy = new ArrayList<>(consumerPassedStreamMetricNames); - log.info("Consumer passed copy size {}", consumerPassedCopy.size()); - consumerPassedCopy.removeAll(streamsThreadMetrics); - log.info("EXTRA metric names {}", consumerPassedCopy); - } + final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); + + final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(0).addedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + assertEquals(streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); - assertEquals(streamsThreadMetrics, consumerPassedStreamMetricNames); + consumerPassedStreamMetricNames.forEach(metricName -> assertTrue(streamsThreadMetrics.contains(metricName))); } } @@ -259,10 +255,9 @@ public void registerMetric(final KafkaMetric metric) { @Override public void unregisterMetric(final KafkaMetric metric) { + addedMetrics.remove(metric); removedMetrics.add(metric); super.unregisterMetric(metric); } } - - } From ef5286b5b71017231e9f514f774c2cfd0a213a4f Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 3 Sep 2024 16:27:34 -0400 Subject: [PATCH 10/54] Added integration tests, set logging level in reporter to debug --- .../KafkaStreamsTelemetryIntegrationTest.java | 109 +++++++++++++++--- ...treamsThreadDelegatingMetricsReporter.java | 4 +- 2 files changed, 95 insertions(+), 18 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index ae36918992239..89751ec679390 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -90,6 +90,8 @@ public class KafkaStreamsTelemetryIntegrationTest { private String outputTopicOnePartition; private final List streamsConfigurations = new ArrayList<>(); private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); + private static final int FIRST_INSTANCE_CONSUMER = 0; + private static final int SECOND_INSTANCE_CONSUMER = 1; @BeforeAll public static void startCluster() throws IOException { @@ -123,7 +125,7 @@ public void tearDown() throws Exception { @ParameterizedTest - @MethodSource("provideStreamParameters") + @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Consumer") void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties = props(stateUpdaterEnabled); @@ -134,31 +136,108 @@ void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnab waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), IntegrationTestUtils.DEFAULT_TIMEOUT, () -> "Kafka Streams never transitioned to a RUNNING state."); - + final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); - - final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(0).addedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + + final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); assertEquals(streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); - consumerPassedStreamMetricNames.forEach(metricName -> assertTrue(streamsThreadMetrics.contains(metricName))); + consumerPassedStreamMetricNames.forEach(metricName -> assertTrue(streamsThreadMetrics.contains(metricName), "Streams metrics doesn't contain " + metricName)); } } - private static Stream provideStreamParameters() { + @ParameterizedTest + @MethodSource("multiTaskParameters") + @DisplayName("Correct treams metrics should get passed with dynamic membership") + void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws InterruptedException { + final Properties properties1 = props(stateUpdaterEnabled); + properties1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); + properties1.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); + + + final Properties properties2 = props(stateUpdaterEnabled); + properties2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); + properties2.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); + + final Topology topology = complexTopology(); + try (final KafkaStreams streamsOne = new KafkaStreams(topology, properties1)) { + streamsOne.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streamsOne.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams never transitioned to a RUNNING state."); + + final List streamsTaskMetricNames = streamsOne.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final List consumerPassedStreamTaskMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName) + .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final List streamTaskIds = getTaskIdsAsStrings(streamsOne); + final long consumerPassedTaskMetricCount = consumerPassedStreamTaskMetricNames.stream().filter(metricName -> streamTaskIds.contains(metricName.tags().get("task-id"))).count(); + assertEquals(streamsTaskMetricNames.size(), consumerPassedStreamTaskMetricNames.size()); + assertEquals(consumerPassedTaskMetricCount, streamsTaskMetricNames.size()); + + + try (final KafkaStreams streamsTwo = new KafkaStreams(topology, properties2)) { + streamsTwo.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streamsTwo.state() && KafkaStreams.State.RUNNING == streamsOne.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams one or two never transitioned to a RUNNING state."); + + final List streamOneTaskIds = getTaskIdsAsStrings(streamsOne); + final List streamTwoTasksIds = getTaskIdsAsStrings(streamsTwo); + + final List streamsOneTaskMetrics = streamsOne.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final List consumerOnePassedTaskMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER) + .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final List streamsTwoTaskMetrics = streamsTwo.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final List consumerTwoPassedTaskMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) + .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + + final long consumerOneStreamOneTaskCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerOneTaskTwoMetricCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); + + final long consumerTwoStreamTwoTaskCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerTwoTaskOneMetricCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); + + assertEquals(streamsOneTaskMetrics.size(), consumerOneStreamOneTaskCount); + assertEquals(0, consumerOneTaskTwoMetricCount); + + assertEquals(streamsTwoTaskMetrics.size(), consumerTwoStreamTwoTaskCount); + assertEquals(0, consumerTwoTaskOneMetricCount); + } + } + } + + private List getTaskIdsAsStrings(final KafkaStreams streams) { + return streams.metadataForLocalThreads().stream() + .flatMap(threadMeta -> threadMeta.activeTasks().stream() + .map(taskMeta -> taskMeta.taskId().toString())) + .collect(Collectors.toList()); + } + + private static Stream singleAndMultiTaskParameters() { return Stream.of(Arguments.of("simple", true), - Arguments.of("simple", false), + Arguments.of("simple", false), Arguments.of("complex", true), - Arguments.of("complex", false)); + Arguments.of("complex", false)); } - - private Properties props() { - return props(new Properties()); + private static Stream multiTaskParameters() { + return Stream.of(Arguments.of(true), + Arguments.of(false)); } + private Properties props(final boolean stateUpdaterEnabled) { return props(mkObjectProperties(mkMap(mkEntry(StreamsConfig.InternalConfig.STATE_UPDATER_ENABLED, stateUpdaterEnabled)))); } + private Properties props(final Properties extraProperties) { final Properties streamsConfiguration = new Properties(); @@ -228,8 +307,7 @@ public Admin getAdmin(final Map config) { public static class MetricsInterceptingConsumer extends KafkaConsumer { - public List addedMetrics = new ArrayList<>(); - public List removedMetrics = new ArrayList<>(); + public List passedMetrics = new ArrayList<>(); public MetricsInterceptingConsumer(final Map configs) { super(configs); @@ -249,14 +327,13 @@ public MetricsInterceptingConsumer(final Map configs, final Dese @Override public void registerMetric(final KafkaMetric metric) { - addedMetrics.add(metric); + passedMetrics.add(metric); super.registerMetric(metric); } @Override public void unregisterMetric(final KafkaMetric metric) { - addedMetrics.remove(metric); - removedMetrics.add(metric); + passedMetrics.remove(metric); super.unregisterMetric(metric); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 2e9e72d5a9241..30359106a319f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -42,7 +42,7 @@ public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, fin this.consumer = Objects.requireNonNull(consumer); this.threadId = Objects.requireNonNull(threadId); this.stateUpdaterThreadId = Objects.requireNonNull(stateUpdaterThreadId); - LOG.info("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); + LOG.debug("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); } @Override @@ -53,7 +53,7 @@ public void init(final List metrics) { @Override public void metricChange(final KafkaMetric metric) { if (tagMatchStreamOrStateUpdaterThreadId(metric)) { - LOG.info("Registering metric {}", metric.metricName()); + LOG.debug("Registering metric {}", metric.metricName()); consumer.registerMetric(metric); } } From 87f58f157b005202b58ef357df3c067dc6508e8f Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 3 Sep 2024 18:28:14 -0400 Subject: [PATCH 11/54] Added test to confirm stream metrics don't bleed into consumer metrics --- .../KafkaStreamsTelemetryIntegrationTest.java | 26 ++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 89751ec679390..fd51f042e18f0 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -49,13 +49,12 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; @@ -73,13 +72,13 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(600) @Tag("integration") public class KafkaStreamsTelemetryIntegrationTest { - private static final Logger log = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); private static final int NUM_BROKERS = 1; public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); @@ -215,6 +214,27 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) } } + @Test + @DisplayName("Streams metrics should not be visible in consumer metrics") + void passedMetricsShouldNotLeakIntoConsumerMetrics() throws InterruptedException { + final Properties properties = props(true); + final Topology topology = complexTopology(); + + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + streams.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams never transitioned to a RUNNING state."); + + final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); + + final Map embeddedConsumerMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).metrics(); + + streamsThreadMetrics.forEach(metricName -> assertFalse(embeddedConsumerMetrics.containsKey(metricName), "Stream thread metric found in client metrics" + metricName)); + } + } + private List getTaskIdsAsStrings(final KafkaStreams streams) { return streams.metadataForLocalThreads().stream() .flatMap(threadMeta -> threadMeta.activeTasks().stream() From abcc6a7cc8333b99cd3f44772ca5fcd381687c4f Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 4 Sep 2024 12:28:17 -0400 Subject: [PATCH 12/54] Rebasing work --- ...msThreadDelegatingMetricsReporterTest.java | 108 ++++++++++++++++++ 1 file changed, 108 insertions(+) create mode 100644 streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java new file mode 100644 index 0000000000000..835c9b61614c1 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -0,0 +1,108 @@ +package org.apache.kafka.streams.internals.metrics; + +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class StreamsThreadDelegatingMetricsReporterTest { + + private MockConsumer mockConsumer; + private StreamsThreadDelegatingMetricsReporter streamsThreadDelegatingMetricsReporter; + + private KafkaMetric kafkaMetricOneHasThreadIdTag; + private KafkaMetric kafkaMetricTwoHasThreadIdTag; + private KafkaMetric kafkaMetricThreeHasThreadIdTag; + private KafkaMetric kafkaMetricWithoutThreadIdTag; + private final Object lock = new Object(); + private final MetricConfig metricConfig = new MetricConfig(); + + + @BeforeEach + void setUp() { + Map threadIdTagMap = new HashMap<>(); + String threadId = "abcxyz-StreamThread-1"; + threadIdTagMap.put("thread-id", threadId); + + Map threadIdWithStateUpdaterTagMap = new HashMap<>(); + String stateUpdaterId = "deftuv-StateUpdater-1"; + threadIdWithStateUpdaterTagMap.put("thread-id", stateUpdaterId); + + Map noThreadIdTagMap = new HashMap<>(); + noThreadIdTagMap.put("client-id", "foo"); + + mockConsumer = new MockConsumer<>(OffsetResetStrategy.NONE); + streamsThreadDelegatingMetricsReporter = new StreamsThreadDelegatingMetricsReporter(mockConsumer, threadId, stateUpdaterId); + + MetricName metricNameOne = new MetricName("metric-one", "test-group-one", "foo bar baz", threadIdTagMap); + MetricName metricNameTwo = new MetricName("metric-two", "test-group-two", "description two", threadIdWithStateUpdaterTagMap); + MetricName metricNameThree = new MetricName("metric-three", "test-group-three", "description three", threadIdTagMap); + MetricName metricNameFour = new MetricName("metric-four", "test-group-three", "description three", noThreadIdTagMap); + + kafkaMetricOneHasThreadIdTag = new KafkaMetric(lock, metricNameOne, (Measurable)(m, now) -> 1.0, metricConfig, Time.SYSTEM); + kafkaMetricTwoHasThreadIdTag = new KafkaMetric(lock, metricNameTwo, (Measurable)(m, now) -> 2.0, metricConfig, Time.SYSTEM); + kafkaMetricThreeHasThreadIdTag = new KafkaMetric(lock, metricNameThree, (Measurable)(m, now) -> 3.0, metricConfig, Time.SYSTEM); + kafkaMetricWithoutThreadIdTag = new KafkaMetric(lock, metricNameFour, (Measurable)(m, now) -> 4.0, metricConfig, Time.SYSTEM); + } + @AfterEach + void tearDown() { + mockConsumer.close(); + } + + + @Test + @DisplayName("Init method should register metrics it receives as parameters") + void shouldInitMetrics() { + List allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + List expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + streamsThreadDelegatingMetricsReporter.init(allMetrics); + assertEquals(expectedMetrics, mockConsumer.addedMetrics()); + } + + @Test + @DisplayName("Should register metrics with thread-id in tag map") + void shouldRegisterMetrics() { + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); + assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); + } + + @Test + @DisplayName("Should remove metrics") + void shouldRemoveMetrics() { + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricTwoHasThreadIdTag); + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricThreeHasThreadIdTag); + List expected = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + assertEquals(expected, mockConsumer.addedMetrics()); + streamsThreadDelegatingMetricsReporter.metricRemoval(kafkaMetricOneHasThreadIdTag); + expected = Arrays.asList(kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + assertEquals(expected, mockConsumer.addedMetrics()); + } + + @Test + @DisplayName("Should not register metrics without thread-id tag") + void shouldNotRegisterMetricsWithoutThreadIdTag() { + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricWithoutThreadIdTag); + assertEquals(0, mockConsumer.addedMetrics().size()); + } + + @Test + @DisplayName("Should set its reference to the consumer to null on closing") + void shouldSetConsumerToNullOnClose() { + streamsThreadDelegatingMetricsReporter.close(); + assertThrows(NullPointerException.class, + () -> streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag)); + } +} \ No newline at end of file From 6057978528cbe086483c4f88e91907a2e852ab31 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 4 Sep 2024 12:55:48 -0400 Subject: [PATCH 13/54] Add some description to large test method --- .../KafkaStreamsTelemetryIntegrationTest.java | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index fd51f042e18f0..a98f85e959a58 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -129,7 +129,9 @@ public void tearDown() throws Exception { void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties = props(stateUpdaterEnabled); final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); - + /* + This test verifies that all Kafka Streams metrics with a thread-id tag get passed to the consumer + */ try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { streams.start(); waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), @@ -159,6 +161,10 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) properties2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); properties2.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); + /* + This test ensures metrics are registered and removed correctly with Kafka Steams dynamic membership changes + */ + final Topology topology = complexTopology(); try (final KafkaStreams streamsOne = new KafkaStreams(topology, properties1)) { streamsOne.start(); @@ -172,6 +178,9 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) final List consumerPassedStreamTaskMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + /* + With only one instance, Kafka Streams should register task metrics for all tasks 0_0, 0_1, 1_0, 1_1 + */ final List streamTaskIds = getTaskIdsAsStrings(streamsOne); final long consumerPassedTaskMetricCount = consumerPassedStreamTaskMetricNames.stream().filter(metricName -> streamTaskIds.contains(metricName.tags().get("task-id"))).count(); assertEquals(streamsTaskMetricNames.size(), consumerPassedStreamTaskMetricNames.size()); @@ -184,6 +193,9 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) IntegrationTestUtils.DEFAULT_TIMEOUT, () -> "Kafka Streams one or two never transitioned to a RUNNING state."); + /* + Now with 2 instances, the tasks will get split amongst both Kafka Streams applications + */ final List streamOneTaskIds = getTaskIdsAsStrings(streamsOne); final List streamTwoTasksIds = getTaskIdsAsStrings(streamsTwo); @@ -198,10 +210,15 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) final List consumerTwoPassedTaskMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); - + /* + Confirm pre-existing KafkaStreams instance one only passes metrics for its tasks and has no metrics for previous tasks + */ final long consumerOneStreamOneTaskCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); final long consumerOneTaskTwoMetricCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); + /* + Confirm new KafkaStreams instance only passes metrics for the newly assigned tasks + */ final long consumerTwoStreamTwoTaskCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); final long consumerTwoTaskOneMetricCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); From bfcb3dd589f53f3fad488f2df123d535555edbc5 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 4 Sep 2024 15:56:32 -0400 Subject: [PATCH 14/54] checkstyle fixes --- ...msThreadDelegatingMetricsReporterTest.java | 68 ++++++++++++------- 1 file changed, 45 insertions(+), 23 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java index 835c9b61614c1..a363f39822b31 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -1,3 +1,20 @@ +/* + * 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.streams.internals.metrics; import org.apache.kafka.clients.consumer.MockConsumer; @@ -7,19 +24,23 @@ import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.utils.Time; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.DisplayName; import org.junit.jupiter.api.Test; -import java.util.*; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; class StreamsThreadDelegatingMetricsReporterTest { - private MockConsumer mockConsumer; + private MockConsumer mockConsumer; private StreamsThreadDelegatingMetricsReporter streamsThreadDelegatingMetricsReporter; private KafkaMetric kafkaMetricOneHasThreadIdTag; @@ -32,50 +53,51 @@ class StreamsThreadDelegatingMetricsReporterTest { @BeforeEach void setUp() { - Map threadIdTagMap = new HashMap<>(); - String threadId = "abcxyz-StreamThread-1"; + final Map threadIdTagMap = new HashMap<>(); + final String threadId = "abcxyz-StreamThread-1"; threadIdTagMap.put("thread-id", threadId); - Map threadIdWithStateUpdaterTagMap = new HashMap<>(); - String stateUpdaterId = "deftuv-StateUpdater-1"; + final Map threadIdWithStateUpdaterTagMap = new HashMap<>(); + final String stateUpdaterId = "deftuv-StateUpdater-1"; threadIdWithStateUpdaterTagMap.put("thread-id", stateUpdaterId); - Map noThreadIdTagMap = new HashMap<>(); + final Map noThreadIdTagMap = new HashMap<>(); noThreadIdTagMap.put("client-id", "foo"); mockConsumer = new MockConsumer<>(OffsetResetStrategy.NONE); streamsThreadDelegatingMetricsReporter = new StreamsThreadDelegatingMetricsReporter(mockConsumer, threadId, stateUpdaterId); - MetricName metricNameOne = new MetricName("metric-one", "test-group-one", "foo bar baz", threadIdTagMap); - MetricName metricNameTwo = new MetricName("metric-two", "test-group-two", "description two", threadIdWithStateUpdaterTagMap); - MetricName metricNameThree = new MetricName("metric-three", "test-group-three", "description three", threadIdTagMap); - MetricName metricNameFour = new MetricName("metric-four", "test-group-three", "description three", noThreadIdTagMap); - - kafkaMetricOneHasThreadIdTag = new KafkaMetric(lock, metricNameOne, (Measurable)(m, now) -> 1.0, metricConfig, Time.SYSTEM); - kafkaMetricTwoHasThreadIdTag = new KafkaMetric(lock, metricNameTwo, (Measurable)(m, now) -> 2.0, metricConfig, Time.SYSTEM); - kafkaMetricThreeHasThreadIdTag = new KafkaMetric(lock, metricNameThree, (Measurable)(m, now) -> 3.0, metricConfig, Time.SYSTEM); - kafkaMetricWithoutThreadIdTag = new KafkaMetric(lock, metricNameFour, (Measurable)(m, now) -> 4.0, metricConfig, Time.SYSTEM); + final MetricName metricNameOne = new MetricName("metric-one", "test-group-one", "foo bar baz", threadIdTagMap); + final MetricName metricNameTwo = new MetricName("metric-two", "test-group-two", "description two", threadIdWithStateUpdaterTagMap); + final MetricName metricNameThree = new MetricName("metric-three", "test-group-three", "description three", threadIdTagMap); + final MetricName metricNameFour = new MetricName("metric-four", "test-group-three", "description three", noThreadIdTagMap); + + kafkaMetricOneHasThreadIdTag = new KafkaMetric(lock, metricNameOne, (Measurable) (m, now) -> 1.0, metricConfig, Time.SYSTEM); + kafkaMetricTwoHasThreadIdTag = new KafkaMetric(lock, metricNameTwo, (Measurable) (m, now) -> 2.0, metricConfig, Time.SYSTEM); + kafkaMetricThreeHasThreadIdTag = new KafkaMetric(lock, metricNameThree, (Measurable) (m, now) -> 3.0, metricConfig, Time.SYSTEM); + kafkaMetricWithoutThreadIdTag = new KafkaMetric(lock, metricNameFour, (Measurable) (m, now) -> 4.0, metricConfig, Time.SYSTEM); } + @AfterEach void tearDown() { mockConsumer.close(); } - + @Test @DisplayName("Init method should register metrics it receives as parameters") void shouldInitMetrics() { - List allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - List expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - streamsThreadDelegatingMetricsReporter.init(allMetrics); - assertEquals(expectedMetrics, mockConsumer.addedMetrics()); + final List allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + final List expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); + streamsThreadDelegatingMetricsReporter.init(allMetrics); + assertEquals(expectedMetrics, mockConsumer.addedMetrics()); } @Test @DisplayName("Should register metrics with thread-id in tag map") void shouldRegisterMetrics() { - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); - assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); + streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); + assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); } @Test From eed082725821183943596ec1665807dbd8842ee2 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 4 Sep 2024 16:22:23 -0400 Subject: [PATCH 15/54] Remove unintentional formatting --- .../kafka/clients/producer/MockProducer.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index f4b034a4bc88b..3ecc132befa2d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -90,13 +90,13 @@ public class MockProducer implements Producer { /** * Create a mock producer * - * @param cluster The cluster holding metadata for this producer - * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise - * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after - * {@link #send(ProducerRecord) send()} to complete the call and unblock the {@link - * java.util.concurrent.Future Future<RecordMetadata>} that is returned. - * @param partitioner The partition strategy - * @param keySerializer The serializer for key that implements {@link Serializer}. + * @param cluster The cluster holding metadata for this producer + * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise + * the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after + * {@link #send(ProducerRecord) send()} to complete the call and unblock the {@link + * java.util.concurrent.Future Future<RecordMetadata>} that is returned. + * @param partitioner The partition strategy + * @param keySerializer The serializer for key that implements {@link Serializer}. * @param valueSerializer The serializer for value that implements {@link Serializer}. */ public MockProducer(final Cluster cluster, @@ -120,7 +120,7 @@ public MockProducer(final Cluster cluster, /** * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers. - *

+ * * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ @SuppressWarnings("deprecation") @@ -132,7 +132,7 @@ public MockProducer(final boolean autoComplete, /** * Create a new mock producer with invented metadata the given autoComplete setting and key\value serializers. - *

+ * * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(cluster, autoComplete, new DefaultPartitioner(), keySerializer, valueSerializer)} */ @SuppressWarnings("deprecation") @@ -145,7 +145,7 @@ public MockProducer(final Cluster cluster, /** * Create a new mock producer with invented metadata the given autoComplete setting, partitioner and key\value serializers. - *

+ * * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), autoComplete, partitioner, keySerializer, valueSerializer)} */ public MockProducer(final boolean autoComplete, @@ -157,7 +157,7 @@ public MockProducer(final boolean autoComplete, /** * Create a new mock producer with invented metadata. - *

+ * * Equivalent to {@link #MockProducer(Cluster, boolean, Partitioner, Serializer, Serializer) new MockProducer(Cluster.empty(), false, null, null, null)} */ public MockProducer() { @@ -226,7 +226,7 @@ public void sendOffsetsToTransaction(Map offs return; } Map uncommittedOffsets = - this.uncommittedConsumerGroupOffsets.computeIfAbsent(groupMetadata.groupId(), k -> new HashMap<>()); + this.uncommittedConsumerGroupOffsets.computeIfAbsent(groupMetadata.groupId(), k -> new HashMap<>()); uncommittedOffsets.putAll(offsets); this.sentOffsets = true; } @@ -504,6 +504,7 @@ public synchronized List> uncommittedRecords() { } /** + * * Get the list of committed consumer group offsets since the last call to {@link #clear()} */ public synchronized List>> consumerGroupOffsetsHistory() { From 5a380a410e6bcb41c1cc205063fbb3b6d8dddab3 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Thu, 5 Sep 2024 17:26:37 -0400 Subject: [PATCH 16/54] Changes per comments, resolving merge conflicts from PR breakdown --- .../kafka/clients/producer/MockProducer.java | 7 ++++++- .../KafkaStreamsTelemetryIntegrationTest.java | 1 - ...treamsThreadDelegatingMetricsReporter.java | 19 +++++++++---------- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 3ecc132befa2d..3d278c40cb067 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -37,6 +37,7 @@ import java.time.Duration; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collections; import java.util.Deque; import java.util.HashMap; import java.util.List; @@ -72,7 +73,7 @@ public class MockProducer implements Producer { private boolean producerFenced; private boolean sentOffsets; private long commitCount = 0L; - final List addedMetrics = new ArrayList<>(); + private final List addedMetrics = new ArrayList<>(); public RuntimeException initTransactionException = null; public RuntimeException beginTransactionException = null; @@ -609,6 +610,10 @@ public void complete(RuntimeException e) { } } + public List addedMetrics() { + return Collections.unmodifiableList(addedMetrics); + } + @Override public void registerMetricForSubscription(KafkaMetric metric) { addedMetrics.add(metric); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index a98f85e959a58..dde2c9a100666 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.integration; - import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.Consumer; diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 30359106a319f..0e1afabdf85d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -29,20 +29,19 @@ import java.util.Objects; public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { - - - private static final Logger LOG = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); - private Consumer consumer; - final String threadId; - final String stateUpdaterThreadId; + + private static final Logger log = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); private static final String THREAD_ID_TAG = "thread-id"; + private Consumer consumer; + private final String threadId; + private final String stateUpdaterThreadId; public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId, final String stateUpdaterThreadId) { this.consumer = Objects.requireNonNull(consumer); this.threadId = Objects.requireNonNull(threadId); this.stateUpdaterThreadId = Objects.requireNonNull(stateUpdaterThreadId); - LOG.debug("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); + log.debug("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); } @Override @@ -53,7 +52,7 @@ public void init(final List metrics) { @Override public void metricChange(final KafkaMetric metric) { if (tagMatchStreamOrStateUpdaterThreadId(metric)) { - LOG.debug("Registering metric {}", metric.metricName()); + log.debug("Registering metric {}", metric.metricName()); consumer.registerMetric(metric); } } @@ -62,7 +61,7 @@ boolean tagMatchStreamOrStateUpdaterThreadId(final KafkaMetric metric) { final Map tags = metric.metricName().tags(); final boolean shouldInclude = tags.containsKey(THREAD_ID_TAG) && (tags.get(THREAD_ID_TAG).equals(threadId) || tags.get(THREAD_ID_TAG).equals(stateUpdaterThreadId)); if (!shouldInclude) { - LOG.warn("Rejecting metric {}", metric.metricName()); + log.warn("Rejecting metric {}", metric.metricName()); } return shouldInclude; } @@ -70,7 +69,7 @@ boolean tagMatchStreamOrStateUpdaterThreadId(final KafkaMetric metric) { @Override public void metricRemoval(final KafkaMetric metric) { if (tagMatchStreamOrStateUpdaterThreadId(metric)) { - LOG.info("Unregistering metric {}", metric.metricName()); + log.debug("Unregistering metric {}", metric.metricName()); consumer.unregisterMetric(metric); } } From 71f4ed58225a4843b3ba5fef87bb0b91fe05fc84 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Thu, 5 Sep 2024 17:35:41 -0400 Subject: [PATCH 17/54] Close telemetry reporter on admin client close. --- .../java/org/apache/kafka/clients/admin/KafkaAdminClient.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index a5a55a6e94877..f2cd3e64195e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -676,6 +676,7 @@ public void close(Duration timeout) { long now = time.milliseconds(); long newHardShutdownTimeMs = now + waitTimeMs; long prev = INVALID_SHUTDOWN_TIME; + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::close); while (true) { if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { if (prev == INVALID_SHUTDOWN_TIME) { From ef83d32d990e0e81830476cccdbace7505ce5a2a Mon Sep 17 00:00:00 2001 From: bbejeck Date: Fri, 6 Sep 2024 17:58:17 -0400 Subject: [PATCH 18/54] Cleanup, revert closing telemetry reporter pending investigation, remove setting consumer to null in reporter close --- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 1 - .../integration/KafkaStreamsTelemetryIntegrationTest.java | 2 +- .../metrics/StreamsThreadDelegatingMetricsReporter.java | 2 +- .../StreamsThreadDelegatingMetricsReporterTest.java | 8 -------- 4 files changed, 2 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index f2cd3e64195e5..a5a55a6e94877 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -676,7 +676,6 @@ public void close(Duration timeout) { long now = time.milliseconds(); long newHardShutdownTimeMs = now + waitTimeMs; long prev = INVALID_SHUTDOWN_TIME; - clientTelemetryReporter.ifPresent(ClientTelemetryReporter::close); while (true) { if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { if (prev == INVALID_SHUTDOWN_TIME) { diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index dde2c9a100666..2f2345febcbaf 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -149,7 +149,7 @@ void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnab @ParameterizedTest @MethodSource("multiTaskParameters") - @DisplayName("Correct treams metrics should get passed with dynamic membership") + @DisplayName("Correct streams metrics should get passed with dynamic membership") void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties1 = props(stateUpdaterEnabled); properties1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 0e1afabdf85d0..670987bb5b015 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -76,7 +76,7 @@ public void metricRemoval(final KafkaMetric metric) { @Override public void close() { - this.consumer = null; + // No op } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java index a363f39822b31..94eab118da3ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -119,12 +119,4 @@ void shouldNotRegisterMetricsWithoutThreadIdTag() { streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricWithoutThreadIdTag); assertEquals(0, mockConsumer.addedMetrics().size()); } - - @Test - @DisplayName("Should set its reference to the consumer to null on closing") - void shouldSetConsumerToNullOnClose() { - streamsThreadDelegatingMetricsReporter.close(); - assertThrows(NullPointerException.class, - () -> streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag)); - } } \ No newline at end of file From 57961caddd9bba3892e6a5034c7063e2f6729424 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Fri, 6 Sep 2024 18:07:52 -0400 Subject: [PATCH 19/54] Checkstyle errors --- .../metrics/StreamsThreadDelegatingMetricsReporterTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java index 94eab118da3ad..70e3a87244f1f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -36,7 +36,6 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; class StreamsThreadDelegatingMetricsReporterTest { From c1c7bd3f3dd2811517eb369edcc76986231d3e6e Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 9 Sep 2024 14:29:42 -0400 Subject: [PATCH 20/54] Address review comments: More descriptive javadoc --- .../org/apache/kafka/clients/admin/Admin.java | 2 +- .../kafka/clients/consumer/KafkaConsumer.java | 17 ++++++++++---- .../kafka/clients/producer/KafkaProducer.java | 17 ++++++++++---- .../KafkaStreamsTelemetryIntegrationTest.java | 23 +++++++++++++++++++ 4 files changed, 50 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index d6695566bc2fa..e2c7c37b7a3bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1864,7 +1864,7 @@ default ListShareGroupsResult listShareGroups() { * Metrics not matching these types are silently ignored. * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. * - * @param metric The application metric to register + * @param metric, the application metric to register */ void registerMetricForSubscription(KafkaMetric metric); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index aef22230c6a9a..34104f9ec5fe0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1432,8 +1432,15 @@ public void resume(Collection partitions) { * This metric will be added to this client's metrics * that are available for subscription and sent as * telemetry data to the broker. - * - * @param metric, the application metric to register + * The provided metric must map to an OTLP metric data point + * type in the OpenTelemetry v1 metrics protobuf message types. + * Specifically, the metric should be one of the following: + * - `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. + * - `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + * Metrics not matching these types are silently ignored. + * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. + * + * @param metric The application metric to register */ @Override public void registerMetric(KafkaMetric metric) { @@ -1443,9 +1450,11 @@ public void registerMetric(KafkaMetric metric) { /** * An application to be removed from subscription. * This metric is removed from this client's metrics - * and will not be available for subscription. + * and will not be available for subscription any longer. + * Executing this method with a metric that has not been registered is a + * benign operation and does not result in any action taken (no-op) * - * @param metric, the application metric to remove + * @param metric The application metric to remove */ @Override public void unregisterMetric(KafkaMetric metric) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 8859fda406927..ee8ac221d8554 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1307,8 +1307,15 @@ public List partitionsFor(String topic) { * This metric will be added to this client's metrics * that are available for subscription and sent as * telemetry data to the broker. + * The provided metric must map to an OTLP metric data point + * type in the OpenTelemetry v1 metrics protobuf message types. + * Specifically, the metric should be one of the following: + * - `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. + * - `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + * Metrics not matching these types are silently ignored. + * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. * - * @param metric, the application metric to register + * @param metric The application metric to register */ @Override public void registerMetricForSubscription(KafkaMetric metric) { @@ -1319,11 +1326,13 @@ public void registerMetricForSubscription(KafkaMetric metric) { } /** - * An application metric to be removed from subscription. + * An application to be removed from subscription. * This metric is removed from this client's metrics - * and will not be available for subscription. + * and will not be available for subscription any longer. + * Executing this method with a metric that has not been registered is a + * benign operation and does not result in any action taken (no-op) * - * @param metric, the application metric to remove + * @param metric The application metric to remove */ @Override public void unregisterMetricFromSubscription(KafkaMetric metric) { diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2f2345febcbaf..d81958a93741c 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -27,10 +27,13 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Measurable; +import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; @@ -58,6 +61,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -70,6 +74,7 @@ import static org.apache.kafka.common.utils.Utils.mkObjectProperties; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -121,6 +126,24 @@ public void tearDown() throws Exception { streamsConfigurations.clear(); } + @Test + @DisplayName("Calling unregisterMetric on metrics not registered should not cause an error") + void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedException { + final Properties properties = props(true); + final Topology topology = complexTopology(); + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + streams.start(); + waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), + IntegrationTestUtils.DEFAULT_TIMEOUT, + () -> "Kafka Streams never transitioned to a RUNNING state."); + + final Consumer embeddedConsumer = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER); + final MetricName metricName = new MetricName("fakeMetric", "fakeGroup", "It's a fake metric", new HashMap<>()); + final KafkaMetric nonExitingMetric = new KafkaMetric(new Object(), metricName, (Measurable) (m, now) -> 1.0, new MetricConfig(), Time.SYSTEM); + assertDoesNotThrow(() -> embeddedConsumer.unregisterMetric(nonExitingMetric)); + } + } + @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") From 67bd91b76d498d2f1c051860f74a1877f17e544b Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 9 Sep 2024 15:00:52 -0400 Subject: [PATCH 21/54] Address review comments: indentation, consumer generic types --- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 6 +++--- .../metrics/StreamsThreadDelegatingMetricsReporter.java | 4 ++-- .../metrics/StreamsThreadDelegatingMetricsReporterTest.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index a5a55a6e94877..1018213e64e16 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -562,11 +562,11 @@ static KafkaAdminClient createInternal( time, 1, (int) TimeUnit.HOURS.toMillis(1), - null, + null, metadataManager.updater(), (hostResolver == null) ? new DefaultHostResolver() : hostResolver, - null, - clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); + null, + clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); } catch (Throwable exc) { diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index 670987bb5b015..ac52d625fa23b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -32,12 +32,12 @@ public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { private static final Logger log = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); private static final String THREAD_ID_TAG = "thread-id"; - private Consumer consumer; + private Consumer consumer; private final String threadId; private final String stateUpdaterThreadId; - public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId, final String stateUpdaterThreadId) { + public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId, final String stateUpdaterThreadId) { this.consumer = Objects.requireNonNull(consumer); this.threadId = Objects.requireNonNull(threadId); this.stateUpdaterThreadId = Objects.requireNonNull(stateUpdaterThreadId); diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java index 70e3a87244f1f..0f4d25c362371 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -39,7 +39,7 @@ class StreamsThreadDelegatingMetricsReporterTest { - private MockConsumer mockConsumer; + private MockConsumer mockConsumer; private StreamsThreadDelegatingMetricsReporter streamsThreadDelegatingMetricsReporter; private KafkaMetric kafkaMetricOneHasThreadIdTag; From 1a772de7cc4d9e4f1649e55e2c73c92c5b549016 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 9 Sep 2024 16:46:31 -0400 Subject: [PATCH 22/54] Address review comments: update test to account for correct state metrics, change visibility on test methods to public --- .../KafkaStreamsTelemetryIntegrationTest.java | 22 ++++++++++++++++--- ...treamsThreadDelegatingMetricsReporter.java | 2 +- ...msThreadDelegatingMetricsReporterTest.java | 12 +++++----- 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index d81958a93741c..815a0fa4b4e9e 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -148,7 +148,7 @@ void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedE @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Consumer") - void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { + public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties = props(stateUpdaterEnabled); final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); /* @@ -173,7 +173,7 @@ void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnab @ParameterizedTest @MethodSource("multiTaskParameters") @DisplayName("Correct streams metrics should get passed with dynamic membership") - void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws InterruptedException { + public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties1 = props(stateUpdaterEnabled); properties1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); properties1.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); @@ -223,39 +223,55 @@ void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) final List streamsOneTaskMetrics = streamsOne.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + final List streamsOneStateMetrics = streamsOne.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); final List consumerOnePassedTaskMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + final List consumerOnePassedStateMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER) + .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); final List streamsTwoTaskMetrics = streamsTwo.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + final List streamsTwoStateMetrics = streamsTwo.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); final List consumerTwoPassedTaskMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); + final List consumerTwoPassedStateMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) + .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); /* Confirm pre-existing KafkaStreams instance one only passes metrics for its tasks and has no metrics for previous tasks */ final long consumerOneStreamOneTaskCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerOneStateMetricCount = consumerOnePassedStateMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); final long consumerOneTaskTwoMetricCount = consumerOnePassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerOneStateTwoMetricCount = consumerOnePassedStateMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); /* Confirm new KafkaStreams instance only passes metrics for the newly assigned tasks */ final long consumerTwoStreamTwoTaskCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerTwoStateMetricCount = consumerTwoPassedStateMetrics.stream().filter(metricName -> streamTwoTasksIds.contains(metricName.tags().get("task-id"))).count(); final long consumerTwoTaskOneMetricCount = consumerTwoPassedTaskMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); + final long consumerTwoStateMetricOneCount = consumerTwoPassedStateMetrics.stream().filter(metricName -> streamOneTaskIds.contains(metricName.tags().get("task-id"))).count(); assertEquals(streamsOneTaskMetrics.size(), consumerOneStreamOneTaskCount); + assertEquals(streamsOneStateMetrics.size(), consumerOneStateMetricCount); assertEquals(0, consumerOneTaskTwoMetricCount); + assertEquals(0, consumerOneStateTwoMetricCount); assertEquals(streamsTwoTaskMetrics.size(), consumerTwoStreamTwoTaskCount); + assertEquals(streamsTwoStateMetrics.size(), consumerTwoStateMetricCount); assertEquals(0, consumerTwoTaskOneMetricCount); + assertEquals(0, consumerTwoStateMetricOneCount); } } } @Test @DisplayName("Streams metrics should not be visible in consumer metrics") - void passedMetricsShouldNotLeakIntoConsumerMetrics() throws InterruptedException { + public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws InterruptedException { final Properties properties = props(true); final Topology topology = complexTopology(); diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java index ac52d625fa23b..472a4edbd9de1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java @@ -32,7 +32,7 @@ public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { private static final Logger log = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); private static final String THREAD_ID_TAG = "thread-id"; - private Consumer consumer; + private final Consumer consumer; private final String threadId; private final String stateUpdaterThreadId; diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java index 0f4d25c362371..e1c18366e3fa7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java @@ -51,7 +51,7 @@ class StreamsThreadDelegatingMetricsReporterTest { @BeforeEach - void setUp() { + public void setUp() { final Map threadIdTagMap = new HashMap<>(); final String threadId = "abcxyz-StreamThread-1"; threadIdTagMap.put("thread-id", threadId); @@ -78,14 +78,14 @@ void setUp() { } @AfterEach - void tearDown() { + public void tearDown() { mockConsumer.close(); } @Test @DisplayName("Init method should register metrics it receives as parameters") - void shouldInitMetrics() { + public void shouldInitMetrics() { final List allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); final List expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); streamsThreadDelegatingMetricsReporter.init(allMetrics); @@ -94,14 +94,14 @@ void shouldInitMetrics() { @Test @DisplayName("Should register metrics with thread-id in tag map") - void shouldRegisterMetrics() { + public void shouldRegisterMetrics() { streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); } @Test @DisplayName("Should remove metrics") - void shouldRemoveMetrics() { + public void shouldRemoveMetrics() { streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricTwoHasThreadIdTag); streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricThreeHasThreadIdTag); @@ -114,7 +114,7 @@ void shouldRemoveMetrics() { @Test @DisplayName("Should not register metrics without thread-id tag") - void shouldNotRegisterMetricsWithoutThreadIdTag() { + public void shouldNotRegisterMetricsWithoutThreadIdTag() { streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricWithoutThreadIdTag); assertEquals(0, mockConsumer.addedMetrics().size()); } From caaa53c2fd1cc870c956a002d178ee262167ad62 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 9 Sep 2024 18:32:32 -0400 Subject: [PATCH 23/54] Merge Admin delegate for testing sending metrics via admin client --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 815a0fa4b4e9e..2efc996438f09 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -92,7 +92,8 @@ public class KafkaStreamsTelemetryIntegrationTest { private String inputTopicOnePartition; private String outputTopicOnePartition; private final List streamsConfigurations = new ArrayList<>(); - private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); + private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); + private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; @@ -376,7 +377,9 @@ public Consumer getGlobalConsumer(final Map conf @Override public Admin getAdmin(final Map config) { - return AdminClient.create(config); + final TestingMetricsInterceptingAdminClient adminClient = new TestingMetricsInterceptingAdminClient(config); + INTERCEPTING_ADMIN_CLIENTS.add(adminClient); + return adminClient; } } @@ -412,4 +415,5 @@ public void unregisterMetric(final KafkaMetric metric) { super.unregisterMetric(metric); } } + } From 98b28e93bec8993c48793006d3fee0f97c87654e Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 10 Sep 2024 18:08:42 -0400 Subject: [PATCH 24/54] Rebasing work Added a new class `StreamsThreadMetricsDelegatingReporter` to handle metrics reporting for Kafka Streams threads. Updated method names across multiple files to improve clarity and consistency by changing `registerMetric` to `registerMetricForSubscription` and `unregisterMetric` to `unregisterMetricFromSubscription`. Additionally, included new tests to verify the changes. --- .../kafka/clients/admin/ForwardingAdmin.java | 2 +- .../kafka/clients/admin/KafkaAdminClient.java | 2 +- .../kafka/clients/consumer/Consumer.java | 8 +- .../kafka/clients/consumer/KafkaConsumer.java | 8 +- .../internals/AsyncKafkaConsumer.java | 4 +- .../internals/ClassicKafkaConsumer.java | 4 +- .../KafkaStreamsTelemetryIntegrationTest.java | 32 +++-- ...TestingMetricsInterceptingAdminClient.java | 6 +- .../apache/kafka/streams/KafkaStreams.java | 4 + ...treamsThreadDelegatingMetricsReporter.java | 86 ------------- .../processor/internals/StreamThread.java | 4 +- ...msThreadDelegatingMetricsReporterTest.java | 121 ------------------ 12 files changed, 45 insertions(+), 236 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java b/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java index 6c6ff6d4c26d6..87e350c5e7ac5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java @@ -320,7 +320,7 @@ public void registerMetricForSubscription(KafkaMetric metric) { } @Override - public void unregisterMetricForSubscription(KafkaMetric metric) { + public void unregisterMetricFromSubscription(KafkaMetric metric) { throw new UnsupportedOperationException(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 1018213e64e16..0dc48f5f33780 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -4291,7 +4291,7 @@ public void registerMetricForSubscription(KafkaMetric metric) { } @Override - public void unregisterMetricForSubscription(KafkaMetric metric) { + public void unregisterMetricFromSubscription(KafkaMetric metric) { if (clientTelemetryReporter.isPresent()) { ClientTelemetryReporter reporter = clientTelemetryReporter.get(); reporter.metricRemoval(metric); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 20d08302b0171..86a00ff6d9e6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -118,14 +118,14 @@ public interface Consumer extends Closeable { void commitAsync(Map offsets, OffsetCommitCallback callback); /** - * @see KafkaConsumer#registerMetric(KafkaMetric) + * @see KafkaConsumer#registerMetricForSubscription(KafkaMetric) */ - void registerMetric(KafkaMetric metric); + void registerMetricForSubscription(KafkaMetric metric); /** - * @see KafkaConsumer#unregisterMetric(KafkaMetric) + * @see KafkaConsumer#unregisterMetricFromSubscription(KafkaMetric) */ - void unregisterMetric(KafkaMetric metric); + void unregisterMetricFromSubscription(KafkaMetric metric); /** * @see KafkaConsumer#registerMetricForSubscription(KafkaMetric) */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 34104f9ec5fe0..bb45a4769c39b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1443,8 +1443,8 @@ public void resume(Collection partitions) { * @param metric The application metric to register */ @Override - public void registerMetric(KafkaMetric metric) { - delegate.registerMetric(metric); + public void registerMetricForSubscription(KafkaMetric metric) { + delegate.registerMetricForSubscription(metric); } /** @@ -1457,8 +1457,8 @@ public void registerMetric(KafkaMetric metric) { * @param metric The application metric to remove */ @Override - public void unregisterMetric(KafkaMetric metric) { - delegate.unregisterMetric(metric); + public void unregisterMetricFromSubscription(KafkaMetric metric) { + delegate.unregisterMetricFromSubscription(metric); } /** 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 3f2aba3e86111..972eba8098906 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 @@ -646,7 +646,7 @@ private void updateGroupMetadata(final Optional memberEpoch, final Stri } @Override - public void registerMetric(KafkaMetric metric) { + public void registerMetricForSubscription(KafkaMetric metric) { if (clientTelemetryReporter.isPresent()) { ClientTelemetryReporter reporter = clientTelemetryReporter.get(); reporter.metricChange(metric); @@ -654,7 +654,7 @@ public void registerMetric(KafkaMetric metric) { } @Override - public void unregisterMetric(KafkaMetric metric) { + public void unregisterMetricFromSubscription(KafkaMetric metric) { if (clientTelemetryReporter.isPresent()) { ClientTelemetryReporter reporter = clientTelemetryReporter.get(); reporter.metricRemoval(metric); 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 c94a8e76d7c81..82a9bd2a53bfc 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 @@ -429,7 +429,7 @@ public void subscribe(Collection topics, ConsumerRebalanceListener liste @Override - public void registerMetric(KafkaMetric metric) { + public void registerMetricForSubscription(KafkaMetric metric) { if (clientTelemetryReporter.isPresent()) { ClientTelemetryReporter reporter = clientTelemetryReporter.get(); reporter.metricChange(metric); @@ -437,7 +437,7 @@ public void registerMetric(KafkaMetric metric) { } @Override - public void unregisterMetric(KafkaMetric metric) { + public void unregisterMetricFromSubscription(KafkaMetric metric) { if (clientTelemetryReporter.isPresent()) { ClientTelemetryReporter reporter = clientTelemetryReporter.get(); reporter.metricRemoval(metric); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2efc996438f09..fbf2c7dcbbfa3 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -18,7 +18,6 @@ package org.apache.kafka.streams.integration; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -93,7 +92,7 @@ public class KafkaStreamsTelemetryIntegrationTest { private String outputTopicOnePartition; private final List streamsConfigurations = new ArrayList<>(); private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); - private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); + private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; @@ -123,13 +122,14 @@ public static void closeCluster() { @AfterEach public void tearDown() throws Exception { INTERCEPTING_CONSUMERS.clear(); + INTERCEPTING_ADMIN_CLIENTS.clear(); IntegrationTestUtils.purgeLocalStreamsState(streamsConfigurations); streamsConfigurations.clear(); } @Test @DisplayName("Calling unregisterMetric on metrics not registered should not cause an error") - void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedException { + public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedException { final Properties properties = props(true); final Topology topology = complexTopology(); try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { @@ -141,7 +141,7 @@ void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedE final Consumer embeddedConsumer = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER); final MetricName metricName = new MetricName("fakeMetric", "fakeGroup", "It's a fake metric", new HashMap<>()); final KafkaMetric nonExitingMetric = new KafkaMetric(new Object(), metricName, (Measurable) (m, now) -> 1.0, new MetricConfig(), Time.SYSTEM); - assertDoesNotThrow(() -> embeddedConsumer.unregisterMetric(nonExitingMetric)); + assertDoesNotThrow(() -> embeddedConsumer.unregisterMetricFromSubscription(nonExitingMetric)); } } @@ -164,10 +164,20 @@ public void shouldPassMetrics(final String topologyType, final boolean stateUpda final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); - final List consumerPassedStreamMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + final List streamsClientMetrics = streams.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.group().equals("stream-metrics")).collect(Collectors.toList()); - assertEquals(streamsThreadMetrics.size(), consumerPassedStreamMetricNames.size()); - consumerPassedStreamMetricNames.forEach(metricName -> assertTrue(streamsThreadMetrics.contains(metricName), "Streams metrics doesn't contain " + metricName)); + + + final List consumerPassedStreamThreadMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + final List adminPassedStreamClientMetricNames = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + + + assertEquals(streamsThreadMetrics.size(), consumerPassedStreamThreadMetricNames.size()); + consumerPassedStreamThreadMetricNames.forEach(metricName -> assertTrue(streamsThreadMetrics.contains(metricName), "Streams metrics doesn't contain " + metricName)); + + assertEquals(streamsClientMetrics.size(), adminPassedStreamClientMetricNames.size()); + adminPassedStreamClientMetricNames.forEach(metricName -> assertTrue(streamsClientMetrics.contains(metricName), "Client metrics doesn't contain " + metricName)); } } @@ -404,15 +414,15 @@ public MetricsInterceptingConsumer(final Map configs, final Dese } @Override - public void registerMetric(final KafkaMetric metric) { + public void registerMetricForSubscription(final KafkaMetric metric) { passedMetrics.add(metric); - super.registerMetric(metric); + super.registerMetricForSubscription(metric); } @Override - public void unregisterMetric(final KafkaMetric metric) { + public void unregisterMetricFromSubscription(final KafkaMetric metric) { passedMetrics.remove(metric); - super.unregisterMetric(metric); + super.unregisterMetricFromSubscription(metric); } } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TestingMetricsInterceptingAdminClient.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TestingMetricsInterceptingAdminClient.java index e1cb7411f8082..a01324ef7fd85 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TestingMetricsInterceptingAdminClient.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TestingMetricsInterceptingAdminClient.java @@ -427,12 +427,14 @@ public ListShareGroupsResult listShareGroups(final ListShareGroupsOptions option @Override public void registerMetricForSubscription(final KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + passedMetrics.add(metric); + adminDelegate.registerMetricForSubscription(metric); } @Override public void unregisterMetricFromSubscription(final KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); + passedMetrics.remove(metric); + adminDelegate.unregisterMetricFromSubscription(metric); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index cb6cbbfb1f404..caf8bf23af467 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -53,6 +53,7 @@ import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.internals.ClientInstanceIdsImpl; import org.apache.kafka.streams.internals.metrics.ClientMetrics; +import org.apache.kafka.streams.internals.metrics.StreamsClientMetricsDelegatingReporter; import org.apache.kafka.streams.processor.StandbyUpdateListener; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateStore; @@ -1016,6 +1017,9 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); metrics = createMetrics(applicationConfigs, time, clientId); + final StreamsClientMetricsDelegatingReporter reporter = new StreamsClientMetricsDelegatingReporter(adminClient); + metrics.addReporter(reporter); + streamsMetrics = new StreamsMetricsImpl( metrics, clientId, diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java deleted file mode 100644 index 472a4edbd9de1..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporter.java +++ /dev/null @@ -1,86 +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.streams.internals.metrics; - -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.common.metrics.KafkaMetric; -import org.apache.kafka.common.metrics.MetricsReporter; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public class StreamsThreadDelegatingMetricsReporter implements MetricsReporter { - - private static final Logger log = LoggerFactory.getLogger(StreamsThreadDelegatingMetricsReporter.class); - private static final String THREAD_ID_TAG = "thread-id"; - private final Consumer consumer; - private final String threadId; - private final String stateUpdaterThreadId; - - - public StreamsThreadDelegatingMetricsReporter(final Consumer consumer, final String threadId, final String stateUpdaterThreadId) { - this.consumer = Objects.requireNonNull(consumer); - this.threadId = Objects.requireNonNull(threadId); - this.stateUpdaterThreadId = Objects.requireNonNull(stateUpdaterThreadId); - log.debug("Creating MetricsReporter for threadId {} and stateUpdaterId {}", threadId, stateUpdaterThreadId); - } - - @Override - public void init(final List metrics) { - metrics.forEach(this::metricChange); - } - - @Override - public void metricChange(final KafkaMetric metric) { - if (tagMatchStreamOrStateUpdaterThreadId(metric)) { - log.debug("Registering metric {}", metric.metricName()); - consumer.registerMetric(metric); - } - } - - boolean tagMatchStreamOrStateUpdaterThreadId(final KafkaMetric metric) { - final Map tags = metric.metricName().tags(); - final boolean shouldInclude = tags.containsKey(THREAD_ID_TAG) && (tags.get(THREAD_ID_TAG).equals(threadId) || tags.get(THREAD_ID_TAG).equals(stateUpdaterThreadId)); - if (!shouldInclude) { - log.warn("Rejecting metric {}", metric.metricName()); - } - return shouldInclude; - } - - @Override - public void metricRemoval(final KafkaMetric metric) { - if (tagMatchStreamOrStateUpdaterThreadId(metric)) { - log.debug("Unregistering metric {}", metric.metricName()); - consumer.unregisterMetric(metric); - } - } - - @Override - public void close() { - // No op - } - - @Override - public void configure(final Map configs) { - // No op - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 64137850a0999..c6eeb5098de66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -48,7 +48,7 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.internals.metrics.ClientMetrics; -import org.apache.kafka.streams.internals.metrics.StreamsThreadDelegatingMetricsReporter; +import org.apache.kafka.streams.internals.metrics.StreamsThreadMetricsDelegatingReporter; import org.apache.kafka.streams.processor.StandbyUpdateListener; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.TaskId; @@ -475,7 +475,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, referenceContainer.mainConsumer = mainConsumer; final String stateUpdaterId = stateUpdaterEnabled ? threadId.replace("-StreamThread-", "-StateUpdater-") : "NA"; - final StreamsThreadDelegatingMetricsReporter reporter = new StreamsThreadDelegatingMetricsReporter(mainConsumer, threadId, stateUpdaterId); + final StreamsThreadMetricsDelegatingReporter reporter = new StreamsThreadMetricsDelegatingReporter(mainConsumer, threadId, stateUpdaterId); streamsMetrics.metricsRegistry().addReporter(reporter); final StreamThread streamThread = new StreamThread( diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java deleted file mode 100644 index e1c18366e3fa7..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/StreamsThreadDelegatingMetricsReporterTest.java +++ /dev/null @@ -1,121 +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.streams.internals.metrics; - -import org.apache.kafka.clients.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.metrics.KafkaMetric; -import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; -import org.apache.kafka.common.utils.Time; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -class StreamsThreadDelegatingMetricsReporterTest { - - private MockConsumer mockConsumer; - private StreamsThreadDelegatingMetricsReporter streamsThreadDelegatingMetricsReporter; - - private KafkaMetric kafkaMetricOneHasThreadIdTag; - private KafkaMetric kafkaMetricTwoHasThreadIdTag; - private KafkaMetric kafkaMetricThreeHasThreadIdTag; - private KafkaMetric kafkaMetricWithoutThreadIdTag; - private final Object lock = new Object(); - private final MetricConfig metricConfig = new MetricConfig(); - - - @BeforeEach - public void setUp() { - final Map threadIdTagMap = new HashMap<>(); - final String threadId = "abcxyz-StreamThread-1"; - threadIdTagMap.put("thread-id", threadId); - - final Map threadIdWithStateUpdaterTagMap = new HashMap<>(); - final String stateUpdaterId = "deftuv-StateUpdater-1"; - threadIdWithStateUpdaterTagMap.put("thread-id", stateUpdaterId); - - final Map noThreadIdTagMap = new HashMap<>(); - noThreadIdTagMap.put("client-id", "foo"); - - mockConsumer = new MockConsumer<>(OffsetResetStrategy.NONE); - streamsThreadDelegatingMetricsReporter = new StreamsThreadDelegatingMetricsReporter(mockConsumer, threadId, stateUpdaterId); - - final MetricName metricNameOne = new MetricName("metric-one", "test-group-one", "foo bar baz", threadIdTagMap); - final MetricName metricNameTwo = new MetricName("metric-two", "test-group-two", "description two", threadIdWithStateUpdaterTagMap); - final MetricName metricNameThree = new MetricName("metric-three", "test-group-three", "description three", threadIdTagMap); - final MetricName metricNameFour = new MetricName("metric-four", "test-group-three", "description three", noThreadIdTagMap); - - kafkaMetricOneHasThreadIdTag = new KafkaMetric(lock, metricNameOne, (Measurable) (m, now) -> 1.0, metricConfig, Time.SYSTEM); - kafkaMetricTwoHasThreadIdTag = new KafkaMetric(lock, metricNameTwo, (Measurable) (m, now) -> 2.0, metricConfig, Time.SYSTEM); - kafkaMetricThreeHasThreadIdTag = new KafkaMetric(lock, metricNameThree, (Measurable) (m, now) -> 3.0, metricConfig, Time.SYSTEM); - kafkaMetricWithoutThreadIdTag = new KafkaMetric(lock, metricNameFour, (Measurable) (m, now) -> 4.0, metricConfig, Time.SYSTEM); - } - - @AfterEach - public void tearDown() { - mockConsumer.close(); - } - - - @Test - @DisplayName("Init method should register metrics it receives as parameters") - public void shouldInitMetrics() { - final List allMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - final List expectedMetrics = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - streamsThreadDelegatingMetricsReporter.init(allMetrics); - assertEquals(expectedMetrics, mockConsumer.addedMetrics()); - } - - @Test - @DisplayName("Should register metrics with thread-id in tag map") - public void shouldRegisterMetrics() { - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); - assertEquals(kafkaMetricOneHasThreadIdTag, mockConsumer.addedMetrics().get(0)); - } - - @Test - @DisplayName("Should remove metrics") - public void shouldRemoveMetrics() { - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricOneHasThreadIdTag); - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricTwoHasThreadIdTag); - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricThreeHasThreadIdTag); - List expected = Arrays.asList(kafkaMetricOneHasThreadIdTag, kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - assertEquals(expected, mockConsumer.addedMetrics()); - streamsThreadDelegatingMetricsReporter.metricRemoval(kafkaMetricOneHasThreadIdTag); - expected = Arrays.asList(kafkaMetricTwoHasThreadIdTag, kafkaMetricThreeHasThreadIdTag); - assertEquals(expected, mockConsumer.addedMetrics()); - } - - @Test - @DisplayName("Should not register metrics without thread-id tag") - public void shouldNotRegisterMetricsWithoutThreadIdTag() { - streamsThreadDelegatingMetricsReporter.metricChange(kafkaMetricWithoutThreadIdTag); - assertEquals(0, mockConsumer.addedMetrics().size()); - } -} \ No newline at end of file From 732a203d23c690d2ab6d1527aec37575815d3963 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 11 Sep 2024 18:05:38 -0400 Subject: [PATCH 25/54] Refactor and add telemetry support in integration test Refactored `KafkaStreamsTelemetryIntegrationTest` to use an added a custom telemetry reporter `TestingClientTelemetry` to integrate telemetry with the test environment. Updated checkstyle to permit new telemetry-related imports. --- checkstyle/import-control.xml | 4 ++ .../KafkaStreamsTelemetryIntegrationTest.java | 67 ++++++++++++++++--- 2 files changed, 60 insertions(+), 11 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 77d668c699465..1f47117b5aa27 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -418,6 +418,10 @@ + + + + diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index fbf2c7dcbbfa3..5d66366530c92 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -28,11 +28,16 @@ import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; +import org.apache.kafka.server.telemetry.ClientTelemetry; +import org.apache.kafka.server.telemetry.ClientTelemetryPayload; +import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; @@ -82,9 +87,7 @@ @Tag("integration") public class KafkaStreamsTelemetryIntegrationTest { - private static final int NUM_BROKERS = 1; - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); - + private static EmbeddedKafkaCluster cluster; private String appId; private String inputTopicTwoPartitions; private String outputTopicTwoPartitions; @@ -93,12 +96,16 @@ public class KafkaStreamsTelemetryIntegrationTest { private final List streamsConfigurations = new ArrayList<>(); private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); + private static final int NUM_BROKERS = 1; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; @BeforeAll public static void startCluster() throws IOException { - CLUSTER.start(); + final Properties properties = new Properties(); + properties.put("metric.reporters", TestingClientTelemetry.class.getName()); + cluster = new EmbeddedKafkaCluster(NUM_BROKERS, properties); + cluster.start(); } @BeforeEach @@ -108,15 +115,15 @@ public void setUp(final TestInfo testInfo) throws InterruptedException { outputTopicTwoPartitions = appId + "-output-two"; inputTopicOnePartition = appId + "-input-one"; outputTopicOnePartition = appId + "-output-one"; - CLUSTER.createTopic(inputTopicTwoPartitions, 2, 1); - CLUSTER.createTopic(outputTopicTwoPartitions, 2, 1); - CLUSTER.createTopic(inputTopicOnePartition, 1, 1); - CLUSTER.createTopic(outputTopicOnePartition, 1, 1); + cluster.createTopic(inputTopicTwoPartitions, 2, 1); + cluster.createTopic(outputTopicTwoPartitions, 2, 1); + cluster.createTopic(inputTopicOnePartition, 1, 1); + cluster.createTopic(outputTopicOnePartition, 1, 1); } @AfterAll public static void closeCluster() { - CLUSTER.stop(); + cluster.stop(); } @AfterEach @@ -145,7 +152,6 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter } } - @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Consumer") @@ -328,7 +334,7 @@ private Properties props(final Properties extraProperties) { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); - streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); @@ -426,4 +432,43 @@ public void unregisterMetricFromSubscription(final KafkaMetric metric) { } } + public static class TestingClientTelemetry implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver { + public static final List SUBSCRIBED_METRICS = new ArrayList<>(); + + public TestingClientTelemetry() { + } + + @Override + public void init(final List metrics) { + } + + @Override + public void metricChange(final KafkaMetric metric) { + } + + @Override + public void metricRemoval(final KafkaMetric metric) { + } + + @Override + public void close() { + + } + + @Override + public void configure(final Map configs) { + + } + + @Override + public ClientTelemetryReceiver clientReceiver() { + return this; + } + + @Override + public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) { + SUBSCRIBED_METRICS.add(payload); + } + } + } From 39145e5923281ea3c86dab3226b5a2bd642cb978 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Fri, 13 Sep 2024 13:00:39 -0400 Subject: [PATCH 26/54] Updates per review comments --- .../kafka/clients/admin/KafkaAdminClient.java | 5 +++-- .../kafka/clients/consumer/KafkaConsumer.java | 16 +++++++++++----- .../kafka/clients/producer/KafkaProducer.java | 16 +++++++++++----- 3 files changed, 25 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 0dc48f5f33780..c5d66fcf6d6d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -562,10 +562,10 @@ static KafkaAdminClient createInternal( time, 1, (int) TimeUnit.HOURS.toMillis(1), - null, + null, metadataManager.updater(), (hostResolver == null) ? new DefaultHostResolver() : hostResolver, - null, + null, clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); @@ -704,6 +704,7 @@ public void close(Duration timeout) { // Wait for the thread to be joined. thread.join(waitTimeMs); } + metrics.close(); log.debug("Kafka admin client closed."); } catch (InterruptedException e) { log.debug("Interrupted while joining I/O thread", e); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index bb45a4769c39b..974d5164b06cf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1428,15 +1428,21 @@ public void resume(Collection partitions) { } /** - * An application metric provided for subscription. + * Add the provided application metric for subscription. * This metric will be added to this client's metrics * that are available for subscription and sent as * telemetry data to the broker. * The provided metric must map to an OTLP metric data point * type in the OpenTelemetry v1 metrics protobuf message types. * Specifically, the metric should be one of the following: - * - `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. - * - `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + *

    + *
  • + * `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. + *
  • + *
  • + * `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + *
  • + *
* Metrics not matching these types are silently ignored. * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. * @@ -1448,11 +1454,11 @@ public void registerMetricForSubscription(KafkaMetric metric) { } /** - * An application to be removed from subscription. + * Remove the provided application metric for subscription. * This metric is removed from this client's metrics * and will not be available for subscription any longer. * Executing this method with a metric that has not been registered is a - * benign operation and does not result in any action taken (no-op) + * benign operation and does not result in any action taken (no-op). * * @param metric The application metric to remove */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index ee8ac221d8554..e1bd7f03aca80 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1303,15 +1303,21 @@ public List partitionsFor(String topic) { /** - * An application metric provided for subscription. + * Add the provided application metric for subscription. * This metric will be added to this client's metrics * that are available for subscription and sent as * telemetry data to the broker. * The provided metric must map to an OTLP metric data point * type in the OpenTelemetry v1 metrics protobuf message types. * Specifically, the metric should be one of the following: - * - `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. - * - `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + *
    + *
  • + * `Sum`: Monotonic total count meter (Counter). Suitable for metrics like total number of X, e.g., total bytes sent. + *
  • + *
  • + * `Gauge`: Non-monotonic current value meter (UpDownCounter). Suitable for metrics like current value of Y, e.g., current queue count. + *
  • + *
* Metrics not matching these types are silently ignored. * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. * @@ -1326,11 +1332,11 @@ public void registerMetricForSubscription(KafkaMetric metric) { } /** - * An application to be removed from subscription. + * Remove the provided application metric for subscription. * This metric is removed from this client's metrics * and will not be available for subscription any longer. * Executing this method with a metric that has not been registered is a - * benign operation and does not result in any action taken (no-op) + * benign operation and does not result in any action taken (no-op). * * @param metric The application metric to remove */ From 5f92c6d75515c37133b446edfa020c33221bf640 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Wed, 25 Sep 2024 15:38:05 -0400 Subject: [PATCH 27/54] Add the telemetry reporter to the reporters list after creation --- .../java/org/apache/kafka/clients/admin/KafkaAdminClient.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c5d66fcf6d6d4..76668efed60e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -543,6 +543,8 @@ static KafkaAdminClient createInternal( adminAddresses.usingBootstrapControllers()); metadataManager.update(Cluster.bootstrap(adminAddresses.addresses()), time.milliseconds()); List reporters = CommonClientConfigs.metricsReporters(clientId, config); + clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); + clientTelemetryReporter.ifPresent(reporters::add); Map metricTags = Collections.singletonMap("client-id", clientId); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(AdminClientConfig.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(config.getLong(AdminClientConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) @@ -551,8 +553,6 @@ static KafkaAdminClient createInternal( MetricsContext metricsContext = new KafkaMetricsContext(JMX_PREFIX, config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)); metrics = new Metrics(metricConfig, reporters, time, metricsContext); - clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); - clientTelemetryReporter.ifPresent(telemetryReporter -> telemetryReporter.contextChange(metricsContext)); networkClient = ClientUtils.createNetworkClient(config, clientId, metrics, From 8a65d2830b6f1572ad656525867737cb159243a5 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Thu, 26 Sep 2024 15:53:59 -0400 Subject: [PATCH 28/54] Fix merge conflic from Fix broken tests in KafkaAdminClientTest --- .../apache/kafka/clients/admin/KafkaAdminClientTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index a864503e1c928..7c7870040c1bd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -453,7 +453,8 @@ public void testMetricsReporterAutoGeneratedClientId() { public void testDisableJmxReporter() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, ""); + props.setProperty(AdminClientConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); + props.setProperty(CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG, "false"); KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); assertTrue(admin.metrics.reporters().isEmpty()); admin.close(); @@ -464,8 +465,9 @@ public void testExplicitlyEnableJmxReporter() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); + props.setProperty(CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG, "true"); KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); - assertEquals(1, admin.metrics.reporters().size()); + assertEquals(2, admin.metrics.reporters().size()); admin.close(); } From 87e4c6750afb5a6e499a1c6a9a760a8d91aa7e2d Mon Sep 17 00:00:00 2001 From: bbejeck Date: Thu, 26 Sep 2024 17:47:25 -0400 Subject: [PATCH 29/54] Minor cleanup --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 5d66366530c92..b23f46219eec4 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -332,7 +332,6 @@ private Properties props(final boolean stateUpdaterEnabled) { private Properties props(final Properties extraProperties) { final Properties streamsConfiguration = new Properties(); - streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); @@ -342,7 +341,6 @@ private Properties props(final Properties extraProperties) { streamsConfiguration.put(StreamsConfig.DEFAULT_CLIENT_SUPPLIER_CONFIG, TestClientSupplier.class); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.putAll(extraProperties); - streamsConfigurations.add(streamsConfiguration); return streamsConfiguration; From 4201cb570a4340af4b30f1992b3553342d81115a Mon Sep 17 00:00:00 2001 From: bbejeck Date: Fri, 27 Sep 2024 14:58:35 -0400 Subject: [PATCH 30/54] Fix merge conflict from Disable metrics push in AdminClient by default; update tests --- .../clients/admin/KafkaAdminClientTest.java | 17 +++++++++++++++-- .../KafkaStreamsTelemetryIntegrationTest.java | 2 ++ .../org/apache/kafka/streams/StreamsConfig.java | 1 + 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 7c7870040c1bd..08803f165d023 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -235,6 +235,7 @@ import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -445,7 +446,7 @@ public void testMetricsReporterAutoGeneratedClientId() { MockMetricsReporter mockMetricsReporter = (MockMetricsReporter) admin.metrics.reporters().get(0); assertEquals(admin.getClientId(), mockMetricsReporter.clientId); - assertEquals(1, admin.metrics.reporters().size()); + assertEquals(2, admin.metrics.reporters().size()); admin.close(); } @@ -465,9 +466,21 @@ public void testExplicitlyEnableJmxReporter() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); - props.setProperty(CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG, "true"); + KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); + assertEquals(1, admin.metrics.reporters().size()); + admin.close(); + } + + @Test + public void testExplicitlyEnableTelemetryReporter() { + Properties props = new Properties(); + props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); + props.setProperty(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true"); KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); assertEquals(2, admin.metrics.reporters().size()); + //ClientTelemetryReporter always added after metrics reporters created with JmxReporter + assertInstanceOf(ClientTelemetryReporter.class, admin.metrics.reporters().get(1)); admin.close(); } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index b23f46219eec4..dea695027251b 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.integration; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -391,6 +392,7 @@ public Consumer getGlobalConsumer(final Map conf @Override public Admin getAdmin(final Map config) { + assertTrue((Boolean) config.get(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG)); final TestingMetricsInterceptingAdminClient adminClient = new TestingMetricsInterceptingAdminClient(config); INTERCEPTING_ADMIN_CLIENTS.add(adminClient); return adminClient; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index ae12240f9976d..35dcf31347d65 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1857,6 +1857,7 @@ public Map getAdminConfigs(final String clientId) { // add client id with stream client id prefix props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId); + props.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); return props; } From 631b7201f270ab16730149ed2a98f0256e366e45 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Sat, 28 Sep 2024 11:36:34 -0400 Subject: [PATCH 31/54] Fix merge conflict from fix tests, start of end-to-end metrics test --- .../java/kafka/admin/ClientTelemetryTest.java | 2 +- .../KafkaStreamsTelemetryIntegrationTest.java | 27 +++++++++++++++++-- .../apache/kafka/streams/KafkaStreams.java | 5 +++- .../apache/kafka/streams/StreamsConfig.java | 2 -- 4 files changed, 30 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/kafka/admin/ClientTelemetryTest.java b/core/src/test/java/kafka/admin/ClientTelemetryTest.java index f9dc1a4330076..3426dcc4d99ca 100644 --- a/core/src/test/java/kafka/admin/ClientTelemetryTest.java +++ b/core/src/test/java/kafka/admin/ClientTelemetryTest.java @@ -80,7 +80,7 @@ public class ClientTelemetryTest { public void testClientInstanceId(ClusterInstance clusterInstance) throws InterruptedException, ExecutionException { Map configs = new HashMap<>(); configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()); - configs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); + configs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true"); try (Admin admin = Admin.create(configs)) { String testTopicName = "test_topic"; admin.createTopics(Collections.singletonList(new NewTopic(testTopicName, 1, (short) 1))); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index dea695027251b..2b7b25216c12b 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; @@ -39,6 +40,7 @@ import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryPayload; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; +import org.apache.kafka.streams.ClientInstanceIds; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; @@ -64,6 +66,7 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -82,12 +85,12 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(600) @Tag("integration") public class KafkaStreamsTelemetryIntegrationTest { - private static EmbeddedKafkaCluster cluster; private String appId; private String inputTopicTwoPartitions; @@ -97,7 +100,7 @@ public class KafkaStreamsTelemetryIntegrationTest { private final List streamsConfigurations = new ArrayList<>(); private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); - private static final int NUM_BROKERS = 1; + private static final int NUM_BROKERS = 3; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; @@ -153,6 +156,26 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter } } + @Test + @DisplayName("End-to-end test validating metrics pushed to broker") + public void shouldPushMetricsToBroker() throws Exception { + final Properties properties = props(true); + final Topology topology = complexTopology(); + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); + + final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); + final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); + final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() + .filter(entry -> entry.getKey().endsWith("1-consumer")) + .map(Map.Entry::getValue) + .findFirst().get(); + assertNotNull(adminInstanceId); + assertNotNull(mainConsumerInstanceId); + } + } + + @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Consumer") diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index caf8bf23af467..c5699d6820930 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.clients.admin.MemberToRemove; import org.apache.kafka.clients.admin.RemoveMembersFromConsumerGroupOptions; @@ -1011,7 +1012,9 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, // use client id instead of thread client id since this admin client may be shared among threads this.clientSupplier = clientSupplier; - adminClient = clientSupplier.getAdmin(applicationConfigs.getAdminConfigs(ClientUtils.adminClientId(clientId))); + final Map adminConfigs = applicationConfigs.getAdminConfigs(ClientUtils.adminClientId(clientId)); + adminConfigs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); + adminClient = clientSupplier.getAdmin(adminConfigs); log.info("Kafka Streams version: {}", ClientMetrics.version()); log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 35dcf31347d65..18d45875fd3dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1857,8 +1857,6 @@ public Map getAdminConfigs(final String clientId) { // add client id with stream client id prefix props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId); - props.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); - return props; } From 637ad3563ff8b19013bda297cb1087116473425d Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 30 Sep 2024 13:37:04 -0400 Subject: [PATCH 32/54] Call initiate close on ClientTelemetryReporter during admin close --- .../java/org/apache/kafka/clients/admin/KafkaAdminClient.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 76668efed60e2..cac5968ac76e9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -676,6 +676,8 @@ public void close(Duration timeout) { long now = time.milliseconds(); long newHardShutdownTimeMs = now + waitTimeMs; long prev = INVALID_SHUTDOWN_TIME; + final long reporterTimeoutMs = waitTimeMs; + clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(reporterTimeoutMs)); while (true) { if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { if (prev == INVALID_SHUTDOWN_TIME) { From 5400f6253101ffe9b758cbf252d90d93998ca5cb Mon Sep 17 00:00:00 2001 From: bbejeck Date: Tue, 1 Oct 2024 13:44:55 -0400 Subject: [PATCH 33/54] Updates per comments --- .../apache/kafka/streams/processor/internals/StreamThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index c6eeb5098de66..446932e35cd0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -474,7 +474,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, taskManager.setMainConsumer(mainConsumer); referenceContainer.mainConsumer = mainConsumer; - final String stateUpdaterId = stateUpdaterEnabled ? threadId.replace("-StreamThread-", "-StateUpdater-") : "NA"; + final String stateUpdaterId = threadId.replace("-StreamThread-", "-StateUpdater-"); final StreamsThreadMetricsDelegatingReporter reporter = new StreamsThreadMetricsDelegatingReporter(mainConsumer, threadId, stateUpdaterId); streamsMetrics.metricsRegistry().addReporter(reporter); From 3091485aa07021171be00bcf47c43dc639e1f6e6 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Sat, 5 Oct 2024 18:57:54 -0400 Subject: [PATCH 34/54] Fix missed items during rebase --- .../org/apache/kafka/clients/consumer/Consumer.java | 9 --------- .../apache/kafka/clients/consumer/KafkaConsumer.java | 10 ---------- 2 files changed, 19 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 86a00ff6d9e6c..4201395578390 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -122,15 +122,6 @@ public interface Consumer extends Closeable { */ void registerMetricForSubscription(KafkaMetric metric); - /** - * @see KafkaConsumer#unregisterMetricFromSubscription(KafkaMetric) - */ - void unregisterMetricFromSubscription(KafkaMetric metric); - /** - * @see KafkaConsumer#registerMetricForSubscription(KafkaMetric) - */ - void registerMetricForSubscription(KafkaMetric metric); - /** * @see KafkaConsumer#unregisterMetricFromSubscription(KafkaMetric) */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 974d5164b06cf..8297a5aaed637 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1746,14 +1746,4 @@ KafkaConsumerMetrics kafkaConsumerMetrics() { boolean updateAssignmentMetadataIfNeeded(final Timer timer) { return delegate.updateAssignmentMetadataIfNeeded(timer); } - - @Override - public void registerMetricForSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); - } - - @Override - public void unregisterMetricFromSubscription(KafkaMetric metric) { - throw new UnsupportedOperationException("not implemented"); - } } From 69f9cc9a863ebac35fa172b7af8cb8c9566f6869 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Mon, 7 Oct 2024 16:55:49 -0400 Subject: [PATCH 35/54] Fix constructor that got mangled during rebase --- .../src/main/java/org/apache/kafka/streams/KafkaStreams.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index c5699d6820930..1d99d54981f15 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1020,7 +1020,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); metrics = createMetrics(applicationConfigs, time, clientId); - final StreamsClientMetricsDelegatingReporter reporter = new StreamsClientMetricsDelegatingReporter(adminClient); + final StreamsClientMetricsDelegatingReporter reporter = new StreamsClientMetricsDelegatingReporter(adminClient, clientId); metrics.addReporter(reporter); streamsMetrics = new StreamsMetricsImpl( From a88edc2170694853f46df98c1e5e67674bba18a5 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 16 Oct 2024 15:42:02 -0400 Subject: [PATCH 36/54] Fixes for resolve conflict merge mistakes during rebase --- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 3 +-- .../org/apache/kafka/clients/admin/KafkaAdminClientTest.java | 5 ++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index cac5968ac76e9..fb195c9a3dd4a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -676,8 +676,7 @@ public void close(Duration timeout) { long now = time.milliseconds(); long newHardShutdownTimeMs = now + waitTimeMs; long prev = INVALID_SHUTDOWN_TIME; - final long reporterTimeoutMs = waitTimeMs; - clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(reporterTimeoutMs)); + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); while (true) { if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { if (prev == INVALID_SHUTDOWN_TIME) { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 08803f165d023..d143c3a76e3d0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -446,7 +446,7 @@ public void testMetricsReporterAutoGeneratedClientId() { MockMetricsReporter mockMetricsReporter = (MockMetricsReporter) admin.metrics.reporters().get(0); assertEquals(admin.getClientId(), mockMetricsReporter.clientId); - assertEquals(2, admin.metrics.reporters().size()); + assertEquals(1, admin.metrics.reporters().size()); admin.close(); } @@ -454,8 +454,7 @@ public void testMetricsReporterAutoGeneratedClientId() { public void testDisableJmxReporter() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(AdminClientConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); - props.setProperty(CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG, "false"); + props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, ""); KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); assertTrue(admin.metrics.reporters().isEmpty()); admin.close(); From 9b3e22330c6f88b10a168fc54b17bd98ee8703ec Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 16 Oct 2024 18:01:02 -0400 Subject: [PATCH 37/54] Address review comments --- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 8 +++----- .../main/java/org/apache/kafka/streams/KafkaStreams.java | 4 +--- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index fb195c9a3dd4a..5220db68ca254 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -561,11 +561,9 @@ static KafkaAdminClient createInternal( apiVersions, time, 1, - (int) TimeUnit.HOURS.toMillis(1), - null, + (int) TimeUnit.HOURS.toMillis(1), null, metadataManager.updater(), - (hostResolver == null) ? new DefaultHostResolver() : hostResolver, - null, + (hostResolver == null) ? new DefaultHostResolver() : hostResolver, null, clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); @@ -677,6 +675,7 @@ public void close(Duration timeout) { long newHardShutdownTimeMs = now + waitTimeMs; long prev = INVALID_SHUTDOWN_TIME; clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); + metrics.close(); while (true) { if (hardShutdownTimeMs.compareAndSet(prev, newHardShutdownTimeMs)) { if (prev == INVALID_SHUTDOWN_TIME) { @@ -705,7 +704,6 @@ public void close(Duration timeout) { // Wait for the thread to be joined. thread.join(waitTimeMs); } - metrics.close(); log.debug("Kafka admin client closed."); } catch (InterruptedException e) { log.debug("Interrupted while joining I/O thread", e); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 1d99d54981f15..770f08c710774 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1012,9 +1012,7 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, // use client id instead of thread client id since this admin client may be shared among threads this.clientSupplier = clientSupplier; - final Map adminConfigs = applicationConfigs.getAdminConfigs(ClientUtils.adminClientId(clientId)); - adminConfigs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); - adminClient = clientSupplier.getAdmin(adminConfigs); + adminClient = clientSupplier.getAdmin(applicationConfigs.getAdminConfigs(ClientUtils.adminClientId(clientId))); log.info("Kafka Streams version: {}", ClientMetrics.version()); log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); From 838957604d009dfbe069f2be907750cc022f14d2 Mon Sep 17 00:00:00 2001 From: Bill Date: Thu, 17 Oct 2024 09:16:42 -0400 Subject: [PATCH 38/54] Checkstyle fixes --- streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 770f08c710774..5a59a79ef5aca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.clients.admin.MemberToRemove; import org.apache.kafka.clients.admin.RemoveMembersFromConsumerGroupOptions; From 6d64484515d818f515221fbbe8d463b44ccd06c6 Mon Sep 17 00:00:00 2001 From: Bill Date: Thu, 17 Oct 2024 11:11:18 -0400 Subject: [PATCH 39/54] Address Comments --- .../kafka/clients/admin/KafkaAdminClientTest.java | 12 ++++++++++++ .../KafkaStreamsTelemetryIntegrationTest.java | 13 ++++--------- .../org/apache/kafka/streams/StreamsConfig.java | 2 +- .../streams/processor/internals/StreamThread.java | 10 +++++++--- .../org/apache/kafka/streams/StreamsConfigTest.java | 2 +- 5 files changed, 25 insertions(+), 14 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index d143c3a76e3d0..0ff71d7369885 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -151,6 +151,7 @@ import org.apache.kafka.common.message.ShareGroupDescribeResponseData; import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; +import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -483,6 +484,17 @@ public void testExplicitlyEnableTelemetryReporter() { admin.close(); } + @Test + public void testExplicitlyTelemetryReporterIsDisabled() { + Properties props = new Properties(); + props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); + KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); + assertEquals(1, admin.metrics.reporters().size()); + assertInstanceOf(JmxReporter.class, admin.metrics.reporters().get(0)); + admin.close(); + } + private static Cluster mockCluster(int numNodes, int controllerIndex) { HashMap nodes = new HashMap<>(); for (int i = 0; i < numNodes; i++) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2b7b25216c12b..f51638d25311d 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -167,7 +167,7 @@ public void shouldPushMetricsToBroker() throws Exception { final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() - .filter(entry -> entry.getKey().endsWith("1-consumer")) + .filter(entry -> !entry.getKey().contains("restore")) .map(Map.Entry::getValue) .findFirst().get(); assertNotNull(adminInstanceId); @@ -178,13 +178,11 @@ public void shouldPushMetricsToBroker() throws Exception { @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") - @DisplayName("Streams metrics should get passed to Consumer") + @DisplayName("Streams metrics should get passed to Admin and Consumer") public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { final Properties properties = props(stateUpdaterEnabled); final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); - /* - This test verifies that all Kafka Streams metrics with a thread-id tag get passed to the consumer - */ + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { streams.start(); waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), @@ -223,10 +221,7 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE final Properties properties2 = props(stateUpdaterEnabled); properties2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); properties2.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); - - /* - This test ensures metrics are registered and removed correctly with Kafka Steams dynamic membership changes - */ + final Topology topology = complexTopology(); try (final KafkaStreams streamsOne = new KafkaStreams(topology, properties1)) { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 18d45875fd3dd..820a79e52c46a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1230,7 +1230,7 @@ public class StreamsConfig extends AbstractConfig { private static final Map ADMIN_CLIENT_OVERRIDES; static { final Map tempAdminClientDefaultOverrides = new HashMap<>(); - tempAdminClientDefaultOverrides.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true"); + tempAdminClientDefaultOverrides.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); ADMIN_CLIENT_OVERRIDES = Collections.unmodifiableMap(tempAdminClientDefaultOverrides); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 446932e35cd0e..e492d180175ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -88,6 +88,9 @@ public class StreamThread extends Thread implements ProcessingThread { + private static final String THREAD_ID_SUBSTRING = "-StreamThread-"; + private static final String STATE_UPDATER_ID_SUBSTRING = "-StateUpdater-"; + /** * Stream thread states are the possible states that a stream thread can be in. * A thread must only be in one state at a time @@ -368,7 +371,8 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, final int threadIdx, final Runnable shutdownErrorHook, final BiConsumer streamsUncaughtExceptionHandler) { - final String threadId = clientId + "-StreamThread-" + threadIdx; + + final String threadId = clientId + THREAD_ID_SUBSTRING + threadIdx; final String logPrefix = String.format("stream-thread [%s] ", threadId); final LogContext logContext = new LogContext(logPrefix); @@ -474,7 +478,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata, taskManager.setMainConsumer(mainConsumer); referenceContainer.mainConsumer = mainConsumer; - final String stateUpdaterId = threadId.replace("-StreamThread-", "-StateUpdater-"); + final String stateUpdaterId = threadId.replace(THREAD_ID_SUBSTRING, STATE_UPDATER_ID_SUBSTRING); final StreamsThreadMetricsDelegatingReporter reporter = new StreamsThreadMetricsDelegatingReporter(mainConsumer, threadId, stateUpdaterId); streamsMetrics.metricsRegistry().addReporter(reporter); @@ -538,7 +542,7 @@ private static StateUpdater maybeCreateAndStartStateUpdater(final boolean stateU final String clientId, final int threadIdx) { if (stateUpdaterEnabled) { - final String name = clientId + "-StateUpdater-" + threadIdx; + final String name = clientId + STATE_UPDATER_ID_SUBSTRING + threadIdx; final StateUpdater stateUpdater = new DefaultStateUpdater( name, streamsMetrics.metricsRegistry(), diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 95948de7f4d46..31e755c3b1f8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -383,7 +383,7 @@ public void shouldOverrideNonPrefixedCustomConfigsWithPrefixedConfigs() { @Test public void shouldOverrideAdminDefaultAdminClientEnableTelemetry() { final Map returnedProps = streamsConfig.getAdminConfigs(clientId); - assertThat(returnedProps.get(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG), equalTo("true")); + assertTrue((boolean) returnedProps.get(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG)); } @Test From 4eec2b12d40ab86a951a416303caeb25d45eb982 Mon Sep 17 00:00:00 2001 From: Bill Date: Thu, 17 Oct 2024 16:44:57 -0400 Subject: [PATCH 40/54] Completed integration test --- .../KafkaStreamsTelemetryIntegrationTest.java | 45 ++++++++++++++++--- 1 file changed, 40 insertions(+), 5 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index f51638d25311d..1955409c9d391 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryPayload; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; +import org.apache.kafka.shaded.io.opentelemetry.proto.metrics.v1.MetricsData; import org.apache.kafka.streams.ClientInstanceIds; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; @@ -51,6 +52,7 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.test.TestUtils; +import org.apache.kafka.tools.ClientMetricsCommand; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -64,6 +66,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.time.Duration; @@ -91,18 +95,21 @@ @Timeout(600) @Tag("integration") public class KafkaStreamsTelemetryIntegrationTest { - private static EmbeddedKafkaCluster cluster; private String appId; private String inputTopicTwoPartitions; private String outputTopicTwoPartitions; private String inputTopicOnePartition; private String outputTopicOnePartition; private final List streamsConfigurations = new ArrayList<>(); + + private static EmbeddedKafkaCluster cluster; private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); private static final int NUM_BROKERS = 3; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; + private static final List SUBSCRIBED_METRICS = new ArrayList<>(); + private static final Logger log = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); @BeforeAll public static void startCluster() throws IOException { @@ -160,8 +167,11 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { final Properties properties = props(true); + final Properties singleValueProps = new Properties(); + singleValueProps.put("bootstrap.servers", cluster.bootstrapServers()); final Topology topology = complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + try (final KafkaStreams streams = new KafkaStreams(topology, properties); + final ClientMetricsCommand.ClientMetricsService clientMetricsService = new ClientMetricsCommand.ClientMetricsService(singleValueProps)) { IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); @@ -172,6 +182,23 @@ public void shouldPushMetricsToBroker() throws Exception { .findFirst().get(); assertNotNull(adminInstanceId); assertNotNull(mainConsumerInstanceId); + + final String[] subscribeCommands = new String[] { + "--bootstrap-server", + cluster.bootstrapServers(), + "--metrics", + "org.apache.kafka.stream", + "--alter", + "--name", + mainConsumerInstanceId.toString(), + "--interval", + "1000" + }; + final ClientMetricsCommand.ClientMetricsCommandOptions commandOptions = new ClientMetricsCommand.ClientMetricsCommandOptions(subscribeCommands); + clientMetricsService.alterClientMetrics(commandOptions); + TestUtils.waitForCondition(() -> !SUBSCRIBED_METRICS.isEmpty(), + 60_000, + "Never subscribed metrics"); } } @@ -451,7 +478,6 @@ public void unregisterMetricFromSubscription(final KafkaMetric metric) { } public static class TestingClientTelemetry implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver { - public static final List SUBSCRIBED_METRICS = new ArrayList<>(); public TestingClientTelemetry() { } @@ -485,8 +511,17 @@ public ClientTelemetryReceiver clientReceiver() { @Override public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) { - SUBSCRIBED_METRICS.add(payload); + try { + MetricsData data = MetricsData.parseFrom(payload.data()); + List names = data.getResourceMetricsList() + .stream() + .map(rm -> rm.getScopeMetricsList().get(0).getMetrics(0).getName()) + .collect(Collectors.toList()); + log.info("Found metrics {}", names); + SUBSCRIBED_METRICS.addAll(names); + } catch (Exception e) { + e.printStackTrace(System.out); + } } } - } From 1c5884cbb93245d9877768c3c8214769f72f240d Mon Sep 17 00:00:00 2001 From: Bill Date: Fri, 18 Oct 2024 16:20:05 -0400 Subject: [PATCH 41/54] Get initial end-to-end integration test running --- .../KafkaStreamsTelemetryIntegrationTest.java | 68 ++++++++++--------- 1 file changed, 37 insertions(+), 31 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 1955409c9d391..2cfa42f7e76da 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Uuid; @@ -34,16 +35,19 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryPayload; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; -import org.apache.kafka.shaded.io.opentelemetry.proto.metrics.v1.MetricsData; import org.apache.kafka.streams.ClientInstanceIds; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; @@ -66,8 +70,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.time.Duration; @@ -108,8 +110,7 @@ public class KafkaStreamsTelemetryIntegrationTest { private static final int NUM_BROKERS = 3; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; - private static final List SUBSCRIBED_METRICS = new ArrayList<>(); - private static final Logger log = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); + public static final Map SUBSCRIBED_CLIENT_METRIC_DATA_COUNT = new HashMap<>(); @BeforeAll public static void startCluster() throws IOException { @@ -167,11 +168,11 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { final Properties properties = props(true); - final Properties singleValueProps = new Properties(); - singleValueProps.put("bootstrap.servers", cluster.bootstrapServers()); + final Properties clientProps = new Properties(); + clientProps.put("bootstrap.servers", cluster.bootstrapServers()); final Topology topology = complexTopology(); try (final KafkaStreams streams = new KafkaStreams(topology, properties); - final ClientMetricsCommand.ClientMetricsService clientMetricsService = new ClientMetricsCommand.ClientMetricsService(singleValueProps)) { + final ClientMetricsCommand.ClientMetricsService clientMetricsService = new ClientMetricsCommand.ClientMetricsService(clientProps)) { IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); @@ -183,22 +184,29 @@ public void shouldPushMetricsToBroker() throws Exception { assertNotNull(adminInstanceId); assertNotNull(mainConsumerInstanceId); - final String[] subscribeCommands = new String[] { - "--bootstrap-server", - cluster.bootstrapServers(), - "--metrics", - "org.apache.kafka.stream", - "--alter", - "--name", - mainConsumerInstanceId.toString(), - "--interval", - "1000" - }; - final ClientMetricsCommand.ClientMetricsCommandOptions commandOptions = new ClientMetricsCommand.ClientMetricsCommandOptions(subscribeCommands); + final String[] metricsSubscriptionParameters = new String[]{"--bootstrap-server", cluster.bootstrapServers(), "--metrics", "org.apache.kafka.stream", + "--alter", "--name", mainConsumerInstanceId.toString(), "--interval", "1000"}; + final ClientMetricsCommand.ClientMetricsCommandOptions commandOptions = new ClientMetricsCommand.ClientMetricsCommandOptions(metricsSubscriptionParameters); clientMetricsService.alterClientMetrics(commandOptions); - TestUtils.waitForCondition(() -> !SUBSCRIBED_METRICS.isEmpty(), - 60_000, - "Never subscribed metrics"); + final List words = Arrays.asList("foo", "bar", "baz", "all", "streams", "lead", "to", "kafka"); + clientProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + clientProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + + IntegrationTestUtils.produceValuesSynchronously(inputTopicTwoPartitions, + words, + clientProps, + cluster.time); + + clientProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + clientProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); + clientProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consumer-group"); + + final List> actualKeyValues = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(clientProps, outputTopicTwoPartitions, words.size()); + assertEquals(words.size(), actualKeyValues.size()); + + TestUtils.waitForCondition(() -> SUBSCRIBED_CLIENT_METRIC_DATA_COUNT.get(mainConsumerInstanceId) >= 10_000, + 30_000 * 10, //Temporary until a workaround for getting past the initial push interval of 5 minutes then set at 30 seconds + "Never received subscribed metrics"); } } @@ -512,14 +520,12 @@ public ClientTelemetryReceiver clientReceiver() { @Override public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) { try { - MetricsData data = MetricsData.parseFrom(payload.data()); - List names = data.getResourceMetricsList() - .stream() - .map(rm -> rm.getScopeMetricsList().get(0).getMetrics(0).getName()) - .collect(Collectors.toList()); - log.info("Found metrics {}", names); - SUBSCRIBED_METRICS.addAll(names); - } catch (Exception e) { + final Uuid clientId = payload.clientInstanceId(); + //Temporary until a solution is found for getting shaded MetricData visible to gradle + final int dataSize = payload.data().array().length; + SUBSCRIBED_CLIENT_METRIC_DATA_COUNT.put(clientId, dataSize); + + } catch (final Exception e) { e.printStackTrace(System.out); } } From 427c65e2f5815e0093a206b6dbe95ecf81047197 Mon Sep 17 00:00:00 2001 From: Bill Date: Fri, 18 Oct 2024 17:32:42 -0400 Subject: [PATCH 42/54] Respond to comments --- .../src/main/java/org/apache/kafka/clients/admin/Admin.java | 2 +- .../org/apache/kafka/clients/admin/KafkaAdminClient.java | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index e2c7c37b7a3bb..d6695566bc2fa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1864,7 +1864,7 @@ default ListShareGroupsResult listShareGroups() { * Metrics not matching these types are silently ignored. * Executing this method for a previously registered metric is a benign operation and results in updating that metrics entry. * - * @param metric, the application metric to register + * @param metric The application metric to register */ void registerMetricForSubscription(KafkaMetric metric); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 5220db68ca254..c04caf482bdb9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -561,9 +561,11 @@ static KafkaAdminClient createInternal( apiVersions, time, 1, - (int) TimeUnit.HOURS.toMillis(1), null, + (int) TimeUnit.HOURS.toMillis(1), + null, metadataManager.updater(), - (hostResolver == null) ? new DefaultHostResolver() : hostResolver, null, + (hostResolver == null) ? new DefaultHostResolver() : hostResolver, + null, clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null)); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, networkClient, timeoutProcessorFactory, logContext, clientTelemetryReporter); From 78cef95cedf86733a30fd68067b7073d6f30bf9e Mon Sep 17 00:00:00 2001 From: Bill Date: Fri, 18 Oct 2024 18:30:10 -0400 Subject: [PATCH 43/54] More descriptive name --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2cfa42f7e76da..b198c4b4d2a40 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -110,7 +110,7 @@ public class KafkaStreamsTelemetryIntegrationTest { private static final int NUM_BROKERS = 3; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; - public static final Map SUBSCRIBED_CLIENT_METRIC_DATA_COUNT = new HashMap<>(); + public static final Map SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT = new HashMap<>(); @BeforeAll public static void startCluster() throws IOException { @@ -204,7 +204,7 @@ public void shouldPushMetricsToBroker() throws Exception { final List> actualKeyValues = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(clientProps, outputTopicTwoPartitions, words.size()); assertEquals(words.size(), actualKeyValues.size()); - TestUtils.waitForCondition(() -> SUBSCRIBED_CLIENT_METRIC_DATA_COUNT.get(mainConsumerInstanceId) >= 10_000, + TestUtils.waitForCondition(() -> SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT.get(mainConsumerInstanceId) >= 10_000, 30_000 * 10, //Temporary until a workaround for getting past the initial push interval of 5 minutes then set at 30 seconds "Never received subscribed metrics"); } @@ -523,7 +523,7 @@ public void exportMetrics(final AuthorizableRequestContext context, final Client final Uuid clientId = payload.clientInstanceId(); //Temporary until a solution is found for getting shaded MetricData visible to gradle final int dataSize = payload.data().array().length; - SUBSCRIBED_CLIENT_METRIC_DATA_COUNT.put(clientId, dataSize); + SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT.put(clientId, dataSize); } catch (final Exception e) { e.printStackTrace(System.out); From f61fb2011553e66a27b91e4e0516da7169f5b6e9 Mon Sep 17 00:00:00 2001 From: Bill Date: Mon, 21 Oct 2024 15:27:35 -0400 Subject: [PATCH 44/54] Reslove conflict from this commit Add open telemetry protobuf as test dependency, update import control, fix test --- build.gradle | 2 + checkstyle/import-control.xml | 1 + .../KafkaStreamsTelemetryIntegrationTest.java | 92 +++++++++++-------- 3 files changed, 56 insertions(+), 39 deletions(-) diff --git a/build.gradle b/build.gradle index d062351938a77..6662b47d1db62 100644 --- a/build.gradle +++ b/build.gradle @@ -2659,6 +2659,8 @@ project(':streams') { testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension testImplementation libs.junitPlatformSuiteEngine // supports suite test + testImplementation libs.opentelemetryProto + testImplementation project(':group-coordinator') testRuntimeOnly project(':streams:test-utils') testRuntimeOnly runtimeTestLibs diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 1f47117b5aa27..df5af363aafc8 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -419,6 +419,7 @@ + diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index b198c4b4d2a40..2705539fd1ace 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Uuid; @@ -35,10 +34,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.telemetry.ClientTelemetry; @@ -47,7 +43,6 @@ import org.apache.kafka.streams.ClientInstanceIds; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; @@ -70,6 +65,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.time.Duration; @@ -80,9 +77,12 @@ import java.util.Locale; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; import java.util.stream.Stream; +import io.opentelemetry.proto.metrics.v1.MetricsData; + import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkObjectProperties; @@ -110,7 +110,8 @@ public class KafkaStreamsTelemetryIntegrationTest { private static final int NUM_BROKERS = 3; private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; - public static final Map SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT = new HashMap<>(); + private static final Logger LOG = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); + public static final Map> SUBSCRIBED_CLIENT_METRICS = new ConcurrentHashMap<>(); @BeforeAll public static void startCluster() throws IOException { @@ -168,13 +169,10 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { final Properties properties = props(true); - final Properties clientProps = new Properties(); - clientProps.put("bootstrap.servers", cluster.bootstrapServers()); - final Topology topology = complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties); - final ClientMetricsCommand.ClientMetricsService clientMetricsService = new ClientMetricsCommand.ClientMetricsService(clientProps)) { + final Topology topology = simpleTopology(); + subscribeForStreamsMetrics(); + try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); - final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() @@ -184,33 +182,14 @@ public void shouldPushMetricsToBroker() throws Exception { assertNotNull(adminInstanceId); assertNotNull(mainConsumerInstanceId); - final String[] metricsSubscriptionParameters = new String[]{"--bootstrap-server", cluster.bootstrapServers(), "--metrics", "org.apache.kafka.stream", - "--alter", "--name", mainConsumerInstanceId.toString(), "--interval", "1000"}; - final ClientMetricsCommand.ClientMetricsCommandOptions commandOptions = new ClientMetricsCommand.ClientMetricsCommandOptions(metricsSubscriptionParameters); - clientMetricsService.alterClientMetrics(commandOptions); - final List words = Arrays.asList("foo", "bar", "baz", "all", "streams", "lead", "to", "kafka"); - clientProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - clientProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - - IntegrationTestUtils.produceValuesSynchronously(inputTopicTwoPartitions, - words, - clientProps, - cluster.time); - - clientProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - clientProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); - clientProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consumer-group"); - - final List> actualKeyValues = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(clientProps, outputTopicTwoPartitions, words.size()); - assertEquals(words.size(), actualKeyValues.size()); - - TestUtils.waitForCondition(() -> SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT.get(mainConsumerInstanceId) >= 10_000, - 30_000 * 10, //Temporary until a workaround for getting past the initial push interval of 5 minutes then set at 30 seconds + TestUtils.waitForCondition(() -> !SUBSCRIBED_CLIENT_METRICS.get(mainConsumerInstanceId).isEmpty(), + 30_000, "Never received subscribed metrics"); + final List actualTaskMetrics = SUBSCRIBED_CLIENT_METRICS.get(mainConsumerInstanceId).stream().filter(metricName -> metricName.startsWith("org.apache.kafka.stream.task")).collect(Collectors.toList()); + assertEquals(EXPECTED_MAIN_CONSUMER_TASK_METRICS, actualTaskMetrics); } } - @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Admin and Consumer") @@ -361,6 +340,15 @@ public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws InterruptedEx } } + private void subscribeForStreamsMetrics() throws Exception { + final Properties clientProps = new Properties(); + clientProps.put("bootstrap.servers", cluster.bootstrapServers()); + try (final ClientMetricsCommand.ClientMetricsService clientMetricsService = new ClientMetricsCommand.ClientMetricsService(clientProps)) { + final String[] metricsSubscriptionParameters = new String[]{"--bootstrap-server", cluster.bootstrapServers(), "--metrics", "org.apache.kafka.stream", "--alter", "--name", "streams-task-metrics-subscription", "--interval", "1000"}; + final ClientMetricsCommand.ClientMetricsCommandOptions commandOptions = new ClientMetricsCommand.ClientMetricsCommandOptions(metricsSubscriptionParameters); + clientMetricsService.alterClientMetrics(commandOptions); + } + } private List getTaskIdsAsStrings(final KafkaStreams streams) { return streams.metadataForLocalThreads().stream() .flatMap(threadMeta -> threadMeta.activeTasks().stream() @@ -520,14 +508,40 @@ public ClientTelemetryReceiver clientReceiver() { @Override public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) { try { + final MetricsData data = MetricsData.parseFrom(payload.data()); final Uuid clientId = payload.clientInstanceId(); - //Temporary until a solution is found for getting shaded MetricData visible to gradle - final int dataSize = payload.data().array().length; - SUBSCRIBED_CLIENT_METRICS_BYTES_AMOUNT.put(clientId, dataSize); - + final List metricNames = data.getResourceMetricsList() + .stream() + .map(rm -> rm.getScopeMetricsList().get(0).getMetrics(0).getName()) + .sorted() + .collect(Collectors.toList()); + LOG.info("Found metrics {} for clientId={}", metricNames, clientId); + SUBSCRIBED_CLIENT_METRICS.put(clientId, metricNames); } catch (final Exception e) { e.printStackTrace(System.out); } } } + + private static final List EXPECTED_MAIN_CONSUMER_TASK_METRICS = Arrays.asList( + "org.apache.kafka.stream.task.active.buffer.count", + "org.apache.kafka.stream.task.active.process.ratio", + "org.apache.kafka.stream.task.dropped.records.rate", + "org.apache.kafka.stream.task.dropped.records.total", + "org.apache.kafka.stream.task.enforced.processing.rate", + "org.apache.kafka.stream.task.enforced.processing.total", + "org.apache.kafka.stream.task.process.latency.avg", + "org.apache.kafka.stream.task.process.latency.max", + "org.apache.kafka.stream.task.process.rate", + "org.apache.kafka.stream.task.process.total", + "org.apache.kafka.stream.task.punctuate.latency.avg", + "org.apache.kafka.stream.task.punctuate.latency.max", + "org.apache.kafka.stream.task.punctuate.rate", + "org.apache.kafka.stream.task.punctuate.total", + "org.apache.kafka.stream.task.record.lateness.avg", + "org.apache.kafka.stream.task.record.lateness.max", + "org.apache.kafka.stream.task.restore.rate", + "org.apache.kafka.stream.task.restore.remaining.records.total", + "org.apache.kafka.stream.task.restore.total" + ); } From d5c5a7b0f7aef3b9ddf42508a934faa184b19597 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 23 Oct 2024 14:35:03 -0400 Subject: [PATCH 45/54] Address comments, fix a potential bug and side cleanup --- build.gradle | 3 +- checkstyle/import-control.xml | 2 +- .../kafka/clients/admin/KafkaAdminClient.java | 47 ++------ .../clients/admin/KafkaAdminClientTest.java | 35 +++--- .../KafkaStreamsTelemetryIntegrationTest.java | 110 ++++++++---------- 5 files changed, 81 insertions(+), 116 deletions(-) diff --git a/build.gradle b/build.gradle index 6662b47d1db62..44194856dddc9 100644 --- a/build.gradle +++ b/build.gradle @@ -2639,7 +2639,7 @@ project(':streams') { } dependencies { - api project(':clients') + api project(path: ':clients', configuration: 'shadow') // `org.rocksdb.Options` is part of Kafka Streams public api via `RocksDBConfigSetter` api libs.rocksDBJni @@ -2659,7 +2659,6 @@ project(':streams') { testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension testImplementation libs.junitPlatformSuiteEngine // supports suite test - testImplementation libs.opentelemetryProto testImplementation project(':group-coordinator') testRuntimeOnly project(':streams:test-utils') diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index df5af363aafc8..36159347f4dd3 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -419,7 +419,7 @@ - + diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c04caf482bdb9..726a955a48ff9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -153,7 +153,6 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData.UserName; import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.message.ExpireDelegationTokenRequestData; -import org.apache.kafka.common.message.GetTelemetrySubscriptionsRequestData; import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; import org.apache.kafka.common.message.ListClientMetricsResourcesRequestData; import org.apache.kafka.common.message.ListGroupsRequestData; @@ -228,8 +227,6 @@ import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.ExpireDelegationTokenRequest; import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; -import org.apache.kafka.common.requests.GetTelemetrySubscriptionsRequest; -import org.apache.kafka.common.requests.GetTelemetrySubscriptionsResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsRequest; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; import org.apache.kafka.common.requests.JoinGroupRequest; @@ -255,6 +252,7 @@ import org.apache.kafka.common.security.token.delegation.DelegationToken; import org.apache.kafka.common.security.token.delegation.TokenInformation; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.KafkaThread; @@ -583,12 +581,13 @@ static KafkaAdminClient createInternal(AdminClientConfig config, Time time) { Metrics metrics = null; String clientId = generateClientId(config); - + Optional clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config); + try { metrics = new Metrics(new MetricConfig(), new LinkedList<>(), time); LogContext logContext = createLogContext(clientId); return new KafkaAdminClient(config, clientId, time, metadataManager, metrics, - client, null, logContext, Optional.empty()); + client, null, logContext, clientTelemetryReporter); } catch (Throwable exc) { closeQuietly(metrics, "Metrics"); throw new KafkaException("Failed to create new KafkaAdminClient", exc); @@ -5070,48 +5069,16 @@ public Uuid clientInstanceId(Duration timeout) { throw new IllegalArgumentException("The timeout cannot be negative."); } - if (!clientTelemetryEnabled) { + if (clientTelemetryReporter.isEmpty()) { throw new IllegalStateException("Telemetry is not enabled. Set config `" + AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG + "` to `true`."); + } if (clientInstanceId != null) { return clientInstanceId; } - final long now = time.milliseconds(); - final KafkaFutureImpl future = new KafkaFutureImpl<>(); - runnable.call(new Call("getTelemetrySubscriptions", calcDeadlineMs(now, (int) timeout.toMillis()), - new LeastLoadedNodeProvider()) { - - @Override - GetTelemetrySubscriptionsRequest.Builder createRequest(int timeoutMs) { - return new GetTelemetrySubscriptionsRequest.Builder(new GetTelemetrySubscriptionsRequestData(), true); - } - - @Override - void handleResponse(AbstractResponse abstractResponse) { - GetTelemetrySubscriptionsResponse response = (GetTelemetrySubscriptionsResponse) abstractResponse; - if (response.error() != Errors.NONE) { - future.completeExceptionally(response.error().exception()); - } else { - future.complete(response.data().clientInstanceId()); - } - } - - @Override - void handleFailure(Throwable throwable) { - future.completeExceptionally(throwable); - } - }, now); - - try { - clientInstanceId = future.get(); - } catch (Exception e) { - log.error("Error occurred while fetching client instance id", e); - throw new KafkaException("Error occurred while fetching client instance id", e); - } - - return clientInstanceId; + return ClientTelemetryUtils.fetchClientInstanceId(clientTelemetryReporter.get(), timeout); } private void invokeDriver( diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 0ff71d7369885..395104d04644e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -121,7 +121,6 @@ import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; -import org.apache.kafka.common.message.GetTelemetrySubscriptionsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.message.InitProducerIdResponseData; @@ -199,8 +198,6 @@ import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; -import org.apache.kafka.common.requests.GetTelemetrySubscriptionsRequest; -import org.apache.kafka.common.requests.GetTelemetrySubscriptionsResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; import org.apache.kafka.common.requests.InitProducerIdRequest; import org.apache.kafka.common.requests.InitProducerIdResponse; @@ -237,6 +234,7 @@ import org.apache.kafka.common.resource.ResourcePatternFilter; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; +import org.apache.kafka.common.telemetry.internals.ClientTelemetrySender; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -250,6 +248,8 @@ import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.MockedStatic; +import org.mockito.internal.stubbing.answers.CallsRealMethods; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -302,10 +302,14 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; /** * A unit test for KafkaAdminClient. @@ -8359,18 +8363,21 @@ public void testFenceProducers() throws Exception { @Test public void testClientInstanceId() { - try (AdminClientUnitTestEnv env = mockClientEnv(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true")) { - Uuid expected = Uuid.randomUuid(); - GetTelemetrySubscriptionsResponseData responseData = - new GetTelemetrySubscriptionsResponseData().setClientInstanceId(expected).setErrorCode(Errors.NONE.code()); - - env.kafkaClient().prepareResponse( - request -> request instanceof GetTelemetrySubscriptionsRequest, - new GetTelemetrySubscriptionsResponse(responseData)); - - Uuid result = env.adminClient().clientInstanceId(Duration.ofSeconds(1)); - assertEquals(expected, result); + try (MockedStatic mockedCommonClientConfigs = mockStatic(CommonClientConfigs.class, new CallsRealMethods())) { + ClientTelemetryReporter clientTelemetryReporter = mock(ClientTelemetryReporter.class); + clientTelemetryReporter.configure(any()); + mockedCommonClientConfigs.when(() -> CommonClientConfigs.telemetryReporter(anyString(), any())).thenReturn(Optional.of(clientTelemetryReporter)); + + try (AdminClientUnitTestEnv env = mockClientEnv(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true")) { + ClientTelemetrySender clientTelemetrySender = mock(ClientTelemetrySender.class); + Uuid expectedUuid = Uuid.randomUuid(); + when(clientTelemetryReporter.telemetrySender()).thenReturn(clientTelemetrySender); + when(clientTelemetrySender.clientInstanceId(any())).thenReturn(Optional.of(expectedUuid)); + + Uuid result = env.adminClient().clientInstanceId(Duration.ofSeconds(1)); + assertEquals(expectedUuid, result); + } } } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2705539fd1ace..38f97b7b71285 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryPayload; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; +import org.apache.kafka.shaded.io.opentelemetry.proto.metrics.v1.MetricsData; import org.apache.kafka.streams.ClientInstanceIds; import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.KafkaStreams; @@ -81,8 +82,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import io.opentelemetry.proto.metrics.v1.MetricsData; - import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkObjectProperties; @@ -102,7 +101,8 @@ public class KafkaStreamsTelemetryIntegrationTest { private String outputTopicTwoPartitions; private String inputTopicOnePartition; private String outputTopicOnePartition; - private final List streamsConfigurations = new ArrayList<>(); + private Properties streamsApplicationProperties = new Properties(); + private Properties streamsSecondApplicationProperties = new Properties(); private static EmbeddedKafkaCluster cluster; private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); @@ -111,12 +111,12 @@ public class KafkaStreamsTelemetryIntegrationTest { private static final int FIRST_INSTANCE_CONSUMER = 0; private static final int SECOND_INSTANCE_CONSUMER = 1; private static final Logger LOG = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); - public static final Map> SUBSCRIBED_CLIENT_METRICS = new ConcurrentHashMap<>(); + @BeforeAll public static void startCluster() throws IOException { final Properties properties = new Properties(); - properties.put("metric.reporters", TestingClientTelemetry.class.getName()); + properties.put("metric.reporters", TelemetryPlugin.class.getName()); cluster = new EmbeddedKafkaCluster(NUM_BROKERS, properties); cluster.start(); } @@ -143,16 +143,18 @@ public static void closeCluster() { public void tearDown() throws Exception { INTERCEPTING_CONSUMERS.clear(); INTERCEPTING_ADMIN_CLIENTS.clear(); - IntegrationTestUtils.purgeLocalStreamsState(streamsConfigurations); - streamsConfigurations.clear(); + IntegrationTestUtils.purgeLocalStreamsState(streamsApplicationProperties); + if (!streamsSecondApplicationProperties.isEmpty()) { + IntegrationTestUtils.purgeLocalStreamsState(streamsSecondApplicationProperties); + } } @Test @DisplayName("Calling unregisterMetric on metrics not registered should not cause an error") public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedException { - final Properties properties = props(true); + streamsApplicationProperties = props(true); final Topology topology = complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { streams.start(); waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), IntegrationTestUtils.DEFAULT_TIMEOUT, @@ -168,40 +170,44 @@ public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws Inter @Test @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { - final Properties properties = props(true); + streamsApplicationProperties = props(true); final Topology topology = simpleTopology(); subscribeForStreamsMetrics(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { + try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() .filter(entry -> !entry.getKey().contains("restore")) .map(Map.Entry::getValue) - .findFirst().get(); + .findFirst().orElseThrow(); assertNotNull(adminInstanceId); assertNotNull(mainConsumerInstanceId); - TestUtils.waitForCondition(() -> !SUBSCRIBED_CLIENT_METRICS.get(mainConsumerInstanceId).isEmpty(), + TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).isEmpty(), 30_000, "Never received subscribed metrics"); - final List actualTaskMetrics = SUBSCRIBED_CLIENT_METRICS.get(mainConsumerInstanceId).stream().filter(metricName -> metricName.startsWith("org.apache.kafka.stream.task")).collect(Collectors.toList()); + final List actualTaskMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).stream().filter(metricName -> metricName.startsWith("org.apache.kafka.stream.task")).collect(Collectors.toList()); assertEquals(EXPECTED_MAIN_CONSUMER_TASK_METRICS, actualTaskMetrics); + + TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId).isEmpty(), + 30_000, + "Never received subscribed metrics"); + final List actualInstanceMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId); + final List expectedInstanceMetrics = Arrays.asList("org.apache.kafka.stream.alive.stream.threads", "org.apache.kafka.stream.failed.stream.threads"); + assertEquals(expectedInstanceMetrics, actualInstanceMetrics); } } @ParameterizedTest @MethodSource("singleAndMultiTaskParameters") @DisplayName("Streams metrics should get passed to Admin and Consumer") - public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws InterruptedException { - final Properties properties = props(stateUpdaterEnabled); + public void shouldPassMetrics(final String topologyType, final boolean stateUpdaterEnabled) throws Exception { + streamsApplicationProperties = props(stateUpdaterEnabled); final Topology topology = topologyType.equals("simple") ? simpleTopology() : complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { - streams.start(); - waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), - IntegrationTestUtils.DEFAULT_TIMEOUT, - () -> "Kafka Streams never transitioned to a RUNNING state."); + try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); @@ -226,23 +232,20 @@ public void shouldPassMetrics(final String topologyType, final boolean stateUpda @ParameterizedTest @MethodSource("multiTaskParameters") @DisplayName("Correct streams metrics should get passed with dynamic membership") - public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws InterruptedException { - final Properties properties1 = props(stateUpdaterEnabled); - properties1.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); - properties1.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); + public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterEnabled) throws Exception { + streamsApplicationProperties = props(stateUpdaterEnabled); + streamsApplicationProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks1"); + streamsApplicationProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks1"); - final Properties properties2 = props(stateUpdaterEnabled); - properties2.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); - properties2.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); + streamsSecondApplicationProperties = props(stateUpdaterEnabled); + streamsSecondApplicationProperties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath() + "-ks2"); + streamsSecondApplicationProperties.put(StreamsConfig.CLIENT_ID_CONFIG, appId + "-ks2"); final Topology topology = complexTopology(); - try (final KafkaStreams streamsOne = new KafkaStreams(topology, properties1)) { - streamsOne.start(); - waitForCondition(() -> KafkaStreams.State.RUNNING == streamsOne.state(), - IntegrationTestUtils.DEFAULT_TIMEOUT, - () -> "Kafka Streams never transitioned to a RUNNING state."); + try (final KafkaStreams streamsOne = new KafkaStreams(topology, streamsApplicationProperties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(streamsOne); final List streamsTaskMetricNames = streamsOne.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); @@ -259,7 +262,7 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE assertEquals(consumerPassedTaskMetricCount, streamsTaskMetricNames.size()); - try (final KafkaStreams streamsTwo = new KafkaStreams(topology, properties2)) { + try (final KafkaStreams streamsTwo = new KafkaStreams(topology, streamsSecondApplicationProperties)) { streamsTwo.start(); waitForCondition(() -> KafkaStreams.State.RUNNING == streamsTwo.state() && KafkaStreams.State.RUNNING == streamsOne.state(), IntegrationTestUtils.DEFAULT_TIMEOUT, @@ -321,22 +324,24 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE @Test @DisplayName("Streams metrics should not be visible in consumer metrics") - public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws InterruptedException { - final Properties properties = props(true); + public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws Exception { + streamsApplicationProperties = props(true); final Topology topology = complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, properties)) { - streams.start(); - waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), - IntegrationTestUtils.DEFAULT_TIMEOUT, - () -> "Kafka Streams never transitioned to a RUNNING state."); + try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final List streamsThreadMetrics = streams.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("thread-id")).collect(Collectors.toList()); + final List streamsClientMetrics = streams.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.group().equals("stream-metrics")).collect(Collectors.toList()); + final Map embeddedConsumerMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).metrics(); + final Map embeddedAdminMetrics = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CONSUMER).metrics(); streamsThreadMetrics.forEach(metricName -> assertFalse(embeddedConsumerMetrics.containsKey(metricName), "Stream thread metric found in client metrics" + metricName)); + streamsClientMetrics.forEach(metricName -> assertFalse(embeddedAdminMetrics.containsKey(metricName), "Stream client metric found in client metrics" + metricName)); } } @@ -383,8 +388,6 @@ private Properties props(final Properties extraProperties) { streamsConfiguration.put(StreamsConfig.DEFAULT_CLIENT_SUPPLIER_CONFIG, TestClientSupplier.class); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.putAll(extraProperties); - streamsConfigurations.add(streamsConfiguration); - return streamsConfiguration; } @@ -444,18 +447,6 @@ public static class MetricsInterceptingConsumer extends KafkaConsumer passedMetrics = new ArrayList<>(); - public MetricsInterceptingConsumer(final Map configs) { - super(configs); - } - - public MetricsInterceptingConsumer(final Properties properties) { - super(properties); - } - - public MetricsInterceptingConsumer(final Properties properties, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { - super(properties, keyDeserializer, valueDeserializer); - } - public MetricsInterceptingConsumer(final Map configs, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { super(configs, keyDeserializer, valueDeserializer); } @@ -473,9 +464,10 @@ public void unregisterMetricFromSubscription(final KafkaMetric metric) { } } - public static class TestingClientTelemetry implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver { + public static class TelemetryPlugin implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver { - public TestingClientTelemetry() { + public static final Map> SUBSCRIBED_METRICS = new ConcurrentHashMap<>(); + public TelemetryPlugin() { } @Override @@ -516,9 +508,9 @@ public void exportMetrics(final AuthorizableRequestContext context, final Client .sorted() .collect(Collectors.toList()); LOG.info("Found metrics {} for clientId={}", metricNames, clientId); - SUBSCRIBED_CLIENT_METRICS.put(clientId, metricNames); + SUBSCRIBED_METRICS.put(clientId, metricNames); } catch (final Exception e) { - e.printStackTrace(System.out); + e.printStackTrace(System.err); } } } From 3f64e7da48373ed78f303f97a9832300075c8694 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 23 Oct 2024 17:13:05 -0400 Subject: [PATCH 46/54] Store clientInstanceId before returning --- .../java/org/apache/kafka/clients/admin/KafkaAdminClient.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 726a955a48ff9..7c7cbb99baaa3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -5078,7 +5078,8 @@ public Uuid clientInstanceId(Duration timeout) { return clientInstanceId; } - return ClientTelemetryUtils.fetchClientInstanceId(clientTelemetryReporter.get(), timeout); + clientInstanceId = ClientTelemetryUtils.fetchClientInstanceId(clientTelemetryReporter.get(), timeout); + return clientInstanceId; } private void invokeDriver( From 6704891034b5a3c20acf60abafc627436559f771 Mon Sep 17 00:00:00 2001 From: Bill Date: Thu, 24 Oct 2024 16:05:27 -0400 Subject: [PATCH 47/54] Address review comments --- .../clients/admin/KafkaAdminClientTest.java | 30 +++++++++++-------- .../java/kafka/admin/ClientTelemetryTest.java | 2 +- .../KafkaStreamsTelemetryIntegrationTest.java | 23 -------------- .../apache/kafka/streams/StreamsConfig.java | 1 - 4 files changed, 18 insertions(+), 38 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 395104d04644e..bf7a624cc4ea8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -150,7 +150,6 @@ import org.apache.kafka.common.message.ShareGroupDescribeResponseData; import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.message.WriteTxnMarkersResponseData; -import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -479,24 +478,29 @@ public void testExplicitlyEnableJmxReporter() { public void testExplicitlyEnableTelemetryReporter() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); props.setProperty(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true"); - KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); - assertEquals(2, admin.metrics.reporters().size()); - //ClientTelemetryReporter always added after metrics reporters created with JmxReporter - assertInstanceOf(ClientTelemetryReporter.class, admin.metrics.reporters().get(1)); - admin.close(); + try (KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props)) { + List telemetryReporterList = admin.metrics.reporters().stream() + .filter(r -> r instanceof ClientTelemetryReporter) + .map(r -> (ClientTelemetryReporter) r) + .collect(Collectors.toList()); + + assertEquals(telemetryReporterList.size(), 1); + } } @Test - public void testExplicitlyTelemetryReporterIsDisabled() { + public void testTelemetryReporterIsDisabledByDefault() { Properties props = new Properties(); props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(AdminClientConfig.METRIC_REPORTER_CLASSES_CONFIG, "org.apache.kafka.common.metrics.JmxReporter"); - KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props); - assertEquals(1, admin.metrics.reporters().size()); - assertInstanceOf(JmxReporter.class, admin.metrics.reporters().get(0)); - admin.close(); + try (KafkaAdminClient admin = (KafkaAdminClient) AdminClient.create(props)) { + List telemetryReporterList = admin.metrics.reporters().stream() + .filter(r -> r instanceof ClientTelemetryReporter) + .map(r -> (ClientTelemetryReporter) r) + .collect(Collectors.toList()); + + assertTrue(telemetryReporterList.isEmpty()); + } } private static Cluster mockCluster(int numNodes, int controllerIndex) { diff --git a/core/src/test/java/kafka/admin/ClientTelemetryTest.java b/core/src/test/java/kafka/admin/ClientTelemetryTest.java index 3426dcc4d99ca..f9dc1a4330076 100644 --- a/core/src/test/java/kafka/admin/ClientTelemetryTest.java +++ b/core/src/test/java/kafka/admin/ClientTelemetryTest.java @@ -80,7 +80,7 @@ public class ClientTelemetryTest { public void testClientInstanceId(ClusterInstance clusterInstance) throws InterruptedException, ExecutionException { Map configs = new HashMap<>(); configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()); - configs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, "true"); + configs.put(AdminClientConfig.ENABLE_METRICS_PUSH_CONFIG, true); try (Admin admin = Admin.create(configs)) { String testTopicName = "test_topic"; admin.createTopics(Collections.singletonList(new NewTopic(testTopicName, 1, (short) 1))); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 38f97b7b71285..5f5985c5f490f 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -28,14 +28,11 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.metrics.KafkaMetric; -import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryPayload; @@ -73,7 +70,6 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -87,7 +83,6 @@ import static org.apache.kafka.common.utils.Utils.mkObjectProperties; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -149,24 +144,6 @@ public void tearDown() throws Exception { } } - @Test - @DisplayName("Calling unregisterMetric on metrics not registered should not cause an error") - public void shouldNotThrowExceptionWhenRemovingNonExistingMetrics() throws InterruptedException { - streamsApplicationProperties = props(true); - final Topology topology = complexTopology(); - try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { - streams.start(); - waitForCondition(() -> KafkaStreams.State.RUNNING == streams.state(), - IntegrationTestUtils.DEFAULT_TIMEOUT, - () -> "Kafka Streams never transitioned to a RUNNING state."); - - final Consumer embeddedConsumer = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER); - final MetricName metricName = new MetricName("fakeMetric", "fakeGroup", "It's a fake metric", new HashMap<>()); - final KafkaMetric nonExitingMetric = new KafkaMetric(new Object(), metricName, (Measurable) (m, now) -> 1.0, new MetricConfig(), Time.SYSTEM); - assertDoesNotThrow(() -> embeddedConsumer.unregisterMetricFromSubscription(nonExitingMetric)); - } - } - @Test @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 820a79e52c46a..d24575a05d88f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1811,7 +1811,6 @@ public Map getGlobalConsumerConfigs(final String clientId) { // add client id with stream client id prefix baseConsumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-global-consumer"); baseConsumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); - return baseConsumerProps; } From ec481192d8feb6afca33b33eeafd4d5eb1d9e043 Mon Sep 17 00:00:00 2001 From: Bill Date: Mon, 28 Oct 2024 17:17:56 -0400 Subject: [PATCH 48/54] Update end-to-end test --- .../KafkaStreamsTelemetryIntegrationTest.java | 34 ++++++------------- 1 file changed, 10 insertions(+), 24 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 5f5985c5f490f..a42c6f6633c18 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -148,6 +148,7 @@ public void tearDown() throws Exception { @DisplayName("End-to-end test validating metrics pushed to broker") public void shouldPushMetricsToBroker() throws Exception { streamsApplicationProperties = props(true); + streamsApplicationProperties.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "TRACE"); final Topology topology = simpleTopology(); subscribeForStreamsMetrics(); try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { @@ -160,12 +161,19 @@ public void shouldPushMetricsToBroker() throws Exception { .findFirst().orElseThrow(); assertNotNull(adminInstanceId); assertNotNull(mainConsumerInstanceId); + LOG.info("Main consumer instance id {}", mainConsumerInstanceId); TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).isEmpty(), 30_000, "Never received subscribed metrics"); - final List actualTaskMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).stream().filter(metricName -> metricName.startsWith("org.apache.kafka.stream.task")).collect(Collectors.toList()); - assertEquals(EXPECTED_MAIN_CONSUMER_TASK_METRICS, actualTaskMetrics); + final List expectedMetrics = streams.metrics().values().stream().map(Metric::metricName) + .filter(metricName -> metricName.tags().containsKey("thread-id")).map(mn -> { + final String name = mn.name().replace('-', '.'); + final String group = mn.group().replace("-metrics", "").replace('-', '.'); + return "org.apache.kafka." + group + "." + name; + }).sorted().collect(Collectors.toList()); + final List actualTaskMetrics = new ArrayList<>(TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId)); + assertEquals(expectedMetrics, actualTaskMetrics); TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId).isEmpty(), 30_000, @@ -491,26 +499,4 @@ public void exportMetrics(final AuthorizableRequestContext context, final Client } } } - - private static final List EXPECTED_MAIN_CONSUMER_TASK_METRICS = Arrays.asList( - "org.apache.kafka.stream.task.active.buffer.count", - "org.apache.kafka.stream.task.active.process.ratio", - "org.apache.kafka.stream.task.dropped.records.rate", - "org.apache.kafka.stream.task.dropped.records.total", - "org.apache.kafka.stream.task.enforced.processing.rate", - "org.apache.kafka.stream.task.enforced.processing.total", - "org.apache.kafka.stream.task.process.latency.avg", - "org.apache.kafka.stream.task.process.latency.max", - "org.apache.kafka.stream.task.process.rate", - "org.apache.kafka.stream.task.process.total", - "org.apache.kafka.stream.task.punctuate.latency.avg", - "org.apache.kafka.stream.task.punctuate.latency.max", - "org.apache.kafka.stream.task.punctuate.rate", - "org.apache.kafka.stream.task.punctuate.total", - "org.apache.kafka.stream.task.record.lateness.avg", - "org.apache.kafka.stream.task.record.lateness.max", - "org.apache.kafka.stream.task.restore.rate", - "org.apache.kafka.stream.task.restore.remaining.records.total", - "org.apache.kafka.stream.task.restore.total" - ); } From 52d389f910780c8688c4c0cf95976d4c00c12e41 Mon Sep 17 00:00:00 2001 From: Bill Date: Tue, 29 Oct 2024 11:49:11 -0400 Subject: [PATCH 49/54] Update end-to-end test to validate metrics for INFO, DEBUG and TRACE --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index a42c6f6633c18..73a3d3d72427d 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -63,6 +63,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -144,11 +145,12 @@ public void tearDown() throws Exception { } } - @Test + @ParameterizedTest + @ValueSource(strings = {"INFO", "DEBUG", "TRACE"}) @DisplayName("End-to-end test validating metrics pushed to broker") - public void shouldPushMetricsToBroker() throws Exception { + public void shouldPushMetricsToBroker(final String recordingLevel) throws Exception { streamsApplicationProperties = props(true); - streamsApplicationProperties.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "TRACE"); + streamsApplicationProperties.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, recordingLevel); final Topology topology = simpleTopology(); subscribeForStreamsMetrics(); try (final KafkaStreams streams = new KafkaStreams(topology, streamsApplicationProperties)) { From 03de445c93a557bac7ff2082e01d6909c30c27e3 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 30 Oct 2024 15:57:46 -0400 Subject: [PATCH 50/54] commit f30f2d03e9064cb9a167fe35b0c5d63470405ce6 puts all clients in the ClientsInstanceIds in consumerInstanceIds map. This PR makes sure only the main consumer client instance id is selected. --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 73a3d3d72427d..d0e6bac586478 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -157,8 +157,11 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams); final ClientInstanceIds clientInstanceIds = streams.clientInstanceIds(Duration.ofSeconds(60)); final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); + final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() - .filter(entry -> !entry.getKey().contains("restore")) + .filter(entry -> !entry.getKey().contains("restore") + && !entry.getKey().contains("producer") + && !entry.getKey().contains("Global")) .map(Map.Entry::getValue) .findFirst().orElseThrow(); assertNotNull(adminInstanceId); From 6a40bbab5add04c92c6260bfe0eff4532aedc395 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 30 Oct 2024 16:08:23 -0400 Subject: [PATCH 51/54] Standardize on lower-case if names ever change --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index d0e6bac586478..11a83ecdbc36d 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -159,9 +159,9 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() - .filter(entry -> !entry.getKey().contains("restore") - && !entry.getKey().contains("producer") - && !entry.getKey().contains("Global")) + .filter(entry -> !entry.getKey().toLowerCase().contains("restore") + && !entry.getKey().toLowerCase().contains("producer") + && !entry.getKey().toLowerCase().contains("global")) .map(Map.Entry::getValue) .findFirst().orElseThrow(); assertNotNull(adminInstanceId); From 26ae0b56278e47b703bf75d2ff056bce1d373285 Mon Sep 17 00:00:00 2001 From: Bill Date: Wed, 30 Oct 2024 16:10:33 -0400 Subject: [PATCH 52/54] Standardize on lower-case if names ever change --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 11a83ecdbc36d..6a897d431a188 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -159,9 +159,9 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() - .filter(entry -> !entry.getKey().toLowerCase().contains("restore") - && !entry.getKey().toLowerCase().contains("producer") - && !entry.getKey().toLowerCase().contains("global")) + .filter(entry -> !entry.getKey().toLowerCase(Locale.getDefault()).contains("restore") + && !entry.getKey().toLowerCase(Locale.getDefault()).contains("producer") + && !entry.getKey().toLowerCase(Locale.getDefault()).contains("global")) .map(Map.Entry::getValue) .findFirst().orElseThrow(); assertNotNull(adminInstanceId); From 21667fee1b98fce52b712768d728027f6d988b59 Mon Sep 17 00:00:00 2001 From: Bill Date: Sat, 2 Nov 2024 12:52:07 -0400 Subject: [PATCH 53/54] Rebased trunk and updated test --- .../integration/KafkaStreamsTelemetryIntegrationTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 6a897d431a188..2a0161809b5af 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -82,7 +82,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkObjectProperties; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -159,9 +159,8 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except final Uuid adminInstanceId = clientInstanceIds.adminInstanceId(); final Uuid mainConsumerInstanceId = clientInstanceIds.consumerInstanceIds().entrySet().stream() - .filter(entry -> !entry.getKey().toLowerCase(Locale.getDefault()).contains("restore") - && !entry.getKey().toLowerCase(Locale.getDefault()).contains("producer") - && !entry.getKey().toLowerCase(Locale.getDefault()).contains("global")) + .filter(entry -> !entry.getKey().endsWith("-restore-consumer") + && !entry.getKey().endsWith("GlobalStreamThread")) .map(Map.Entry::getValue) .findFirst().orElseThrow(); assertNotNull(adminInstanceId); From 8b5682daee1467cc2aab5040d37ab84a1c83e2f0 Mon Sep 17 00:00:00 2001 From: Bill Date: Mon, 4 Nov 2024 17:52:53 -0500 Subject: [PATCH 54/54] switch back to combined mode - getting not enough resources error --- .../KafkaStreamsTelemetryIntegrationTest.java | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java index 2a0161809b5af..9b1854811b716 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsTelemetryIntegrationTest.java @@ -101,11 +101,11 @@ public class KafkaStreamsTelemetryIntegrationTest { private Properties streamsSecondApplicationProperties = new Properties(); private static EmbeddedKafkaCluster cluster; - private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); + private static final List> INTERCEPTING_CONSUMERS = new ArrayList<>(); private static final List INTERCEPTING_ADMIN_CLIENTS = new ArrayList<>(); private static final int NUM_BROKERS = 3; - private static final int FIRST_INSTANCE_CONSUMER = 0; - private static final int SECOND_INSTANCE_CONSUMER = 1; + private static final int FIRST_INSTANCE_CLIENT = 0; + private static final int SECOND_INSTANCE_CLIENT = 1; private static final Logger LOG = LoggerFactory.getLogger(KafkaStreamsTelemetryIntegrationTest.class); @@ -176,8 +176,8 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except final String group = mn.group().replace("-metrics", "").replace('-', '.'); return "org.apache.kafka." + group + "." + name; }).sorted().collect(Collectors.toList()); - final List actualTaskMetrics = new ArrayList<>(TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId)); - assertEquals(expectedMetrics, actualTaskMetrics); + final List actualMetrics = new ArrayList<>(TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId)); + assertEquals(expectedMetrics, actualMetrics); TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId).isEmpty(), 30_000, @@ -206,8 +206,8 @@ public void shouldPassMetrics(final String topologyType, final boolean stateUpda - final List consumerPassedStreamThreadMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); - final List adminPassedStreamClientMetricNames = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + final List consumerPassedStreamThreadMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); + final List adminPassedStreamClientMetricNames = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CLIENT).passedMetrics.stream().map(KafkaMetric::metricName).collect(Collectors.toList()); assertEquals(streamsThreadMetrics.size(), consumerPassedStreamThreadMetricNames.size()); @@ -239,7 +239,7 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE final List streamsTaskMetricNames = streamsOne.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); - final List consumerPassedStreamTaskMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).passedMetrics.stream().map(KafkaMetric::metricName) + final List consumerPassedStreamTaskMetricNames = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT).passedMetrics.stream().map(KafkaMetric::metricName) .filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); /* @@ -268,9 +268,9 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE final List streamsOneStateMetrics = streamsOne.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); - final List consumerOnePassedTaskMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER) + final List consumerOnePassedTaskMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); - final List consumerOnePassedStateMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER) + final List consumerOnePassedStateMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); final List streamsTwoTaskMetrics = streamsTwo.metrics().values().stream().map(Metric::metricName) @@ -278,9 +278,9 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE final List streamsTwoStateMetrics = streamsTwo.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); - final List consumerTwoPassedTaskMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) + final List consumerTwoPassedTaskMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CLIENT) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.tags().containsKey("task-id")).collect(Collectors.toList()); - final List consumerTwoPassedStateMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CONSUMER) + final List consumerTwoPassedStateMetrics = INTERCEPTING_CONSUMERS.get(SECOND_INSTANCE_CLIENT) .passedMetrics.stream().map(KafkaMetric::metricName).filter(metricName -> metricName.group().equals("stream-state-metrics")).collect(Collectors.toList()); /* Confirm pre-existing KafkaStreams instance one only passes metrics for its tasks and has no metrics for previous tasks @@ -312,8 +312,8 @@ public void shouldPassCorrectMetricsDynamicInstances(final boolean stateUpdaterE } @Test - @DisplayName("Streams metrics should not be visible in consumer metrics") - public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws Exception { + @DisplayName("Streams metrics should not be visible in client metrics") + public void passedMetricsShouldNotLeakIntoClientMetrics() throws Exception { streamsApplicationProperties = props(true); final Topology topology = complexTopology(); @@ -326,8 +326,8 @@ public void passedMetricsShouldNotLeakIntoConsumerMetrics() throws Exception { final List streamsClientMetrics = streams.metrics().values().stream().map(Metric::metricName) .filter(metricName -> metricName.group().equals("stream-metrics")).collect(Collectors.toList()); - final Map embeddedConsumerMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CONSUMER).metrics(); - final Map embeddedAdminMetrics = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CONSUMER).metrics(); + final Map embeddedConsumerMetrics = INTERCEPTING_CONSUMERS.get(FIRST_INSTANCE_CLIENT).metrics(); + final Map embeddedAdminMetrics = INTERCEPTING_ADMIN_CLIENTS.get(FIRST_INSTANCE_CLIENT).metrics(); streamsThreadMetrics.forEach(metricName -> assertFalse(embeddedConsumerMetrics.containsKey(metricName), "Stream thread metric found in client metrics" + metricName)); streamsClientMetrics.forEach(metricName -> assertFalse(embeddedAdminMetrics.containsKey(metricName), "Stream client metric found in client metrics" + metricName)); @@ -408,7 +408,7 @@ public Producer getProducer(final Map config) { @Override public Consumer getConsumer(final Map config) { - final MetricsInterceptingConsumer consumer = new MetricsInterceptingConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + final TestingMetricsInterceptingConsumer consumer = new TestingMetricsInterceptingConsumer<>(config, new ByteArrayDeserializer(), new ByteArrayDeserializer()); INTERCEPTING_CONSUMERS.add(consumer); return consumer; } @@ -432,11 +432,11 @@ public Admin getAdmin(final Map config) { } } - public static class MetricsInterceptingConsumer extends KafkaConsumer { + public static class TestingMetricsInterceptingConsumer extends KafkaConsumer { public List passedMetrics = new ArrayList<>(); - public MetricsInterceptingConsumer(final Map configs, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { + public TestingMetricsInterceptingConsumer(final Map configs, final Deserializer keyDeserializer, final Deserializer valueDeserializer) { super(configs, keyDeserializer, valueDeserializer); }