Skip to content

Commit

Permalink
KAFKA-16143: New JMX metrics for AsyncKafkaConsumer (#17199)
Browse files Browse the repository at this point in the history
Reviewers: Andrew Schofield <[email protected]>, Kirk True <[email protected]>, Lianet Magrans <[email protected]>
  • Loading branch information
FrankYang0529 authored Dec 13, 2024
1 parent d5ad922 commit 770d64d
Show file tree
Hide file tree
Showing 22 changed files with 1,036 additions and 88 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@
import org.apache.kafka.clients.consumer.internals.events.TopicSubscriptionChangeEvent;
import org.apache.kafka.clients.consumer.internals.events.UnsubscribeEvent;
import org.apache.kafka.clients.consumer.internals.events.UpdatePatternSubscriptionEvent;
import org.apache.kafka.clients.consumer.internals.metrics.KafkaConsumerMetrics;
import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics;
import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.IsolationLevel;
Expand Down Expand Up @@ -109,7 +109,6 @@
import java.util.ConcurrentModificationException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand Down Expand Up @@ -215,10 +214,11 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
private final ApplicationEventHandler applicationEventHandler;
private final Time time;
private final AtomicReference<Optional<ConsumerGroupMetadata>> groupMetadata = new AtomicReference<>(Optional.empty());
private final KafkaConsumerMetrics kafkaConsumerMetrics;
private final AsyncConsumerMetrics kafkaConsumerMetrics;
private Logger log;
private final String clientId;
private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
private final BackgroundEventHandler backgroundEventHandler;
private final BackgroundEventProcessor backgroundEventProcessor;
private final CompletableEventReaper backgroundEventReaper;
private final Deserializers<K, V> deserializers;
Expand Down Expand Up @@ -320,6 +320,7 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
this.clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config);
this.clientTelemetryReporter.ifPresent(reporters::add);
this.metrics = createMetrics(config, time, reporters);
this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);

List<ConsumerInterceptor<K, V>> interceptorList = configuredConsumerInterceptors(config);
Expand All @@ -339,7 +340,11 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {

ApiVersions apiVersions = new ApiVersions();
final BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
final BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue);
this.backgroundEventHandler = new BackgroundEventHandler(
backgroundEventQueue,
time,
kafkaConsumerMetrics
);

// This FetchBuffer is shared between the application and network threads.
this.fetchBuffer = new FetchBuffer(logContext);
Expand All @@ -352,7 +357,9 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
fetchMetricsManager.throttleTimeSensor(),
clientTelemetryReporter.map(ClientTelemetryReporter::telemetrySender).orElse(null),
backgroundEventHandler,
false);
false,
kafkaConsumerMetrics
);
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig));
final Supplier<RequestManagers> requestManagersSupplier = RequestManagers.supplier(time,
Expand Down Expand Up @@ -382,7 +389,9 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
new CompletableEventReaper(logContext),
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
requestManagersSupplier,
kafkaConsumerMetrics
);

this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
Expand All @@ -402,8 +411,6 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
fetchMetricsManager,
time);

this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);

if (groupMetadata.get().isPresent() &&
GroupProtocol.of(config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)) == GroupProtocol.CONSUMER) {
config.ignore(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG); // Used by background thread
Expand Down Expand Up @@ -460,10 +467,15 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
this.defaultApiTimeoutMs = Duration.ofMillis(defaultApiTimeoutMs);
this.deserializers = deserializers;
this.applicationEventHandler = applicationEventHandler;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics);
this.clientTelemetryReporter = Optional.empty();
this.autoCommitEnabled = autoCommitEnabled;
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
this.backgroundEventHandler = new BackgroundEventHandler(
backgroundEventQueue,
time,
kafkaConsumerMetrics
);
}

AsyncKafkaConsumer(LogContext logContext,
Expand Down Expand Up @@ -498,7 +510,7 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
deserializers,
fetchMetricsManager,
time);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaConsumerMetrics = new AsyncConsumerMetrics(metrics);

GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
Expand All @@ -509,7 +521,11 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {

BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
this.backgroundEventQueue = new LinkedBlockingQueue<>();
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue);
this.backgroundEventHandler = new BackgroundEventHandler(
backgroundEventQueue,
time,
kafkaConsumerMetrics
);
this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
Expand All @@ -524,7 +540,8 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
client,
metadata,
backgroundEventHandler,
false
false,
kafkaConsumerMetrics
);
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
Supplier<RequestManagers> requestManagersSupplier = RequestManagers.supplier(
Expand Down Expand Up @@ -556,7 +573,8 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
new CompletableEventReaper(logContext),
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
requestManagersSupplier,
kafkaConsumerMetrics);
this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = new CompletableEventReaper(logContext);
}
Expand All @@ -571,7 +589,8 @@ ApplicationEventHandler build(
final CompletableEventReaper applicationEventReaper,
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
final Supplier<RequestManagers> requestManagersSupplier
final Supplier<RequestManagers> requestManagersSupplier,
final AsyncConsumerMetrics asyncConsumerMetrics
);

}
Expand Down Expand Up @@ -1941,25 +1960,30 @@ private void subscribeInternal(Collection<String> topics, Optional<ConsumerRebal
* It is possible that {@link ErrorEvent an error}
* could occur when processing the events. In such cases, the processor will take a reference to the first
* error, continue to process the remaining events, and then throw the first error that occurred.
*
* Visible for testing.
*/
private boolean processBackgroundEvents() {
boolean processBackgroundEvents() {
AtomicReference<KafkaException> firstError = new AtomicReference<>();

LinkedList<BackgroundEvent> events = new LinkedList<>();
backgroundEventQueue.drainTo(events);

for (BackgroundEvent event : events) {
try {
if (event instanceof CompletableEvent)
backgroundEventReaper.add((CompletableEvent<?>) event);

backgroundEventProcessor.process(event);
} catch (Throwable t) {
KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t);

if (!firstError.compareAndSet(null, e))
log.warn("An error occurred when processing the background event: {}", e.getMessage(), e);
List<BackgroundEvent> events = backgroundEventHandler.drainEvents();
if (!events.isEmpty()) {
long startMs = time.milliseconds();
for (BackgroundEvent event : events) {
kafkaConsumerMetrics.recordBackgroundEventQueueTime(time.milliseconds() - event.enqueuedMs());
try {
if (event instanceof CompletableEvent)
backgroundEventReaper.add((CompletableEvent<?>) event);

backgroundEventProcessor.process(event);
} catch (Throwable t) {
KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t);

if (!firstError.compareAndSet(null, e))
log.warn("An error occurred when processing the background event: {}", e.getMessage(), e);
}
}
kafkaConsumerMetrics.recordBackgroundEventQueueProcessingTime(time.milliseconds() - startMs);
}

backgroundEventReaper.reap(time.milliseconds());
Expand Down Expand Up @@ -2088,7 +2112,7 @@ public Metrics metricsRegistry() {
}

@Override
public KafkaConsumerMetrics kafkaConsumerMetrics() {
public AsyncConsumerMetrics kafkaConsumerMetrics() {
return kafkaConsumerMetrics;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEvent;
import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper;
import org.apache.kafka.clients.consumer.internals.metrics.AsyncConsumerMetrics;
import org.apache.kafka.common.internals.IdempotentCloser;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.utils.KafkaThread;
Expand Down Expand Up @@ -62,21 +63,24 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable {
private final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier;
private final Supplier<NetworkClientDelegate> networkClientDelegateSupplier;
private final Supplier<RequestManagers> requestManagersSupplier;
private final AsyncConsumerMetrics asyncConsumerMetrics;
private ApplicationEventProcessor applicationEventProcessor;
private NetworkClientDelegate networkClientDelegate;
private RequestManagers requestManagers;
private volatile boolean running;
private final IdempotentCloser closer = new IdempotentCloser();
private volatile Duration closeTimeout = Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS);
private volatile long cachedMaximumTimeToWait = MAX_POLL_TIMEOUT_MS;
private long lastPollTimeMs = 0L;

public ConsumerNetworkThread(LogContext logContext,
Time time,
BlockingQueue<ApplicationEvent> applicationEventQueue,
CompletableEventReaper applicationEventReaper,
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
Supplier<RequestManagers> requestManagersSupplier) {
Supplier<RequestManagers> requestManagersSupplier,
AsyncConsumerMetrics asyncConsumerMetrics) {
super(BACKGROUND_THREAD_NAME, true);
this.time = time;
this.log = logContext.logger(getClass());
Expand All @@ -86,6 +90,7 @@ public ConsumerNetworkThread(LogContext logContext,
this.networkClientDelegateSupplier = networkClientDelegateSupplier;
this.requestManagersSupplier = requestManagersSupplier;
this.running = true;
this.asyncConsumerMetrics = asyncConsumerMetrics;
}

@Override
Expand Down Expand Up @@ -141,6 +146,11 @@ void runOnce() {
processApplicationEvents();

final long currentTimeMs = time.milliseconds();
if (lastPollTimeMs != 0L) {
asyncConsumerMetrics.recordTimeBetweenNetworkThreadPoll(currentTimeMs - lastPollTimeMs);
}
lastPollTimeMs = currentTimeMs;

final long pollWaitTimeMs = requestManagers.entries().stream()
.filter(Optional::isPresent)
.map(Optional::get)
Expand All @@ -166,8 +176,13 @@ void runOnce() {
private void processApplicationEvents() {
LinkedList<ApplicationEvent> events = new LinkedList<>();
applicationEventQueue.drainTo(events);
if (events.isEmpty())
return;

asyncConsumerMetrics.recordApplicationEventQueueSize(0);
long startMs = time.milliseconds();
for (ApplicationEvent event : events) {
asyncConsumerMetrics.recordApplicationEventQueueTime(time.milliseconds() - event.enqueuedMs());
try {
if (event instanceof CompletableEvent) {
applicationEventReaper.add((CompletableEvent<?>) event);
Expand All @@ -181,6 +196,7 @@ private void processApplicationEvents() {
log.warn("Error processing event {}", t.getMessage(), t);
}
}
asyncConsumerMetrics.recordApplicationEventQueueProcessingTime(time.milliseconds() - startMs);
}

/**
Expand All @@ -189,7 +205,7 @@ private void processApplicationEvents() {
* is given least one attempt to satisfy any network requests <em>before</em> checking if a timeout has expired.
*/
private void reapExpiredApplicationEvents(long currentTimeMs) {
applicationEventReaper.reap(currentTimeMs);
asyncConsumerMetrics.recordApplicationEventExpiredSize(applicationEventReaper.reap(currentTimeMs));
}

/**
Expand Down Expand Up @@ -326,7 +342,7 @@ void cleanup() {
log.error("Unexpected error during shutdown. Proceed with closing.", e);
} finally {
sendUnsentRequests(timer);
applicationEventReaper.reap(applicationEventQueue);
asyncConsumerMetrics.recordApplicationEventExpiredSize(applicationEventReaper.reap(applicationEventQueue));

closeQuietly(requestManagers, "request managers");
closeQuietly(networkClientDelegate, "network client delegate");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ public final class ConsumerUtils {
public static final String CONSUMER_SHARE_METRIC_GROUP_PREFIX = "consumer-share";
public static final String COORDINATOR_METRICS_SUFFIX = "-coordinator-metrics";
public static final String CONSUMER_METRICS_SUFFIX = "-metrics";
public static final String CONSUMER_METRIC_GROUP = CONSUMER_METRIC_GROUP_PREFIX + CONSUMER_METRICS_SUFFIX;

/**
* A fixed, large enough value will suffice for max.
Expand Down
Loading

0 comments on commit 770d64d

Please sign in to comment.