Skip to content

Commit

Permalink
KAFKA-16143: New JMX metrics for AsyncKafkaConsumer
Browse files Browse the repository at this point in the history
Signed-off-by: PoAn Yang <[email protected]>
  • Loading branch information
FrankYang0529 committed Nov 15, 2024
1 parent cc20e78 commit 20951a9
Show file tree
Hide file tree
Showing 19 changed files with 769 additions and 51 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,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.KafkaAsyncConsumerMetrics;
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 @@ -213,7 +213,7 @@ 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 KafkaAsyncConsumerMetrics kafkaAsyncConsumerMetrics;
private Logger log;
private final String clientId;
private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
Expand Down Expand Up @@ -318,6 +318,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.kafkaAsyncConsumerMetrics = new KafkaAsyncConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);

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

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

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

this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
Expand All @@ -399,8 +403,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 @@ -457,7 +459,7 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.deserializers = deserializers;
this.applicationEventHandler = applicationEventHandler;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaAsyncConsumerMetrics = new KafkaAsyncConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
this.clientTelemetryReporter = Optional.empty();
this.autoCommitEnabled = autoCommitEnabled;
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
Expand Down Expand Up @@ -495,7 +497,7 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
deserializers,
fetchMetricsManager,
time);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaAsyncConsumerMetrics = new KafkaAsyncConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);

GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
Expand All @@ -506,7 +508,8 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {

BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
this.backgroundEventQueue = new LinkedBlockingQueue<>();
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue);
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(
backgroundEventQueue, time, Optional.of(kafkaAsyncConsumerMetrics));
this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
Expand All @@ -520,7 +523,8 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
logContext,
client,
metadata,
backgroundEventHandler
backgroundEventHandler,
Optional.of(kafkaAsyncConsumerMetrics)
);
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
Supplier<RequestManagers> requestManagersSupplier = RequestManagers.supplier(
Expand Down Expand Up @@ -552,7 +556,8 @@ public void onGroupAssignmentUpdated(Set<TopicPartition> partitions) {
new CompletableEventReaper(logContext),
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
requestManagersSupplier,
Optional.of(kafkaAsyncConsumerMetrics));
this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = new CompletableEventReaper(logContext);
}
Expand All @@ -567,7 +572,8 @@ ApplicationEventHandler build(
final CompletableEventReaper applicationEventReaper,
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
final Supplier<RequestManagers> requestManagersSupplier
final Supplier<RequestManagers> requestManagersSupplier,
final Optional<KafkaAsyncConsumerMetrics> kafkaConsumerMetrics
);

}
Expand Down Expand Up @@ -709,7 +715,7 @@ public ConsumerRecords<K, V> poll(final Duration timeout) {

acquireAndEnsureOpen();
try {
kafkaConsumerMetrics.recordPollStart(timer.currentTimeMs());
kafkaAsyncConsumerMetrics.recordPollStart(timer.currentTimeMs());

if (subscriptions.hasNoSubscriptionOrUserAssignment()) {
throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions");
Expand Down Expand Up @@ -749,7 +755,7 @@ public ConsumerRecords<K, V> poll(final Duration timeout) {

return ConsumerRecords.empty();
} finally {
kafkaConsumerMetrics.recordPollEnd(timer.currentTimeMs());
kafkaAsyncConsumerMetrics.recordPollEnd(timer.currentTimeMs());
release();
}
}
Expand Down Expand Up @@ -943,7 +949,7 @@ public Map<TopicPartition, OffsetAndMetadata> committed(final Set<TopicPartition
wakeupTrigger.clearTask();
}
} finally {
kafkaConsumerMetrics.recordCommitted(time.nanoseconds() - start);
kafkaAsyncConsumerMetrics.recordCommitted(time.nanoseconds() - start);
release();
}
}
Expand Down Expand Up @@ -1330,7 +1336,7 @@ private void close(Duration timeout, boolean swallowException) {
backgroundEventReaper.reap(backgroundEventQueue);

closeQuietly(interceptors, "consumer interceptors", firstException);
closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException);
closeQuietly(kafkaAsyncConsumerMetrics, "kafka consumer metrics", firstException);
closeQuietly(metrics, "consumer metrics", firstException);
closeQuietly(deserializers, "consumer deserializers", firstException);
clientTelemetryReporter.ifPresent(reporter -> closeQuietly(reporter, "async consumer telemetry reporter", firstException));
Expand Down Expand Up @@ -1455,7 +1461,7 @@ private void commitSync(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets
interceptors.onCommit(committedOffsets);
} finally {
wakeupTrigger.clearTask();
kafkaConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart);
kafkaAsyncConsumerMetrics.recordCommitSync(time.nanoseconds() - commitStart);
release();
}
}
Expand Down Expand Up @@ -1893,14 +1899,19 @@ 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);
kafkaAsyncConsumerMetrics.recordBackgroundEventQueueSize(backgroundEventQueue.size());

for (BackgroundEvent event : events) {
kafkaAsyncConsumerMetrics.recordBackgroundEventQueueTime(time.milliseconds() - event.enqueuedMs());
long startMs = time.milliseconds();
try {
if (event instanceof CompletableEvent)
backgroundEventReaper.add((CompletableEvent<?>) event);
Expand All @@ -1911,6 +1922,8 @@ private boolean processBackgroundEvents() {

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

Expand Down Expand Up @@ -2040,8 +2053,8 @@ public Metrics metricsRegistry() {
}

@Override
public KafkaConsumerMetrics kafkaConsumerMetrics() {
return kafkaConsumerMetrics;
public KafkaAsyncConsumerMetrics kafkaConsumerMetrics() {
return kafkaAsyncConsumerMetrics;
}

// Visible for testing
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
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.KafkaAsyncConsumerMetrics;
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 @@ -60,21 +61,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 Optional<KafkaAsyncConsumerMetrics> kafkaAsyncConsumerMetrics;
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,
Optional<KafkaAsyncConsumerMetrics> kafkaAsyncConsumerMetrics) {
super(BACKGROUND_THREAD_NAME, true);
this.time = time;
this.log = logContext.logger(getClass());
Expand All @@ -84,6 +88,7 @@ public ConsumerNetworkThread(LogContext logContext,
this.networkClientDelegateSupplier = networkClientDelegateSupplier;
this.requestManagersSupplier = requestManagersSupplier;
this.running = true;
this.kafkaAsyncConsumerMetrics = kafkaAsyncConsumerMetrics;
}

@Override
Expand Down Expand Up @@ -139,6 +144,10 @@ void runOnce() {
processApplicationEvents();

final long currentTimeMs = time.milliseconds();
final long timeSinceLastPollMs = lastPollTimeMs != 0L ? currentTimeMs - lastPollTimeMs : 0;
lastPollTimeMs = currentTimeMs;
kafkaAsyncConsumerMetrics.ifPresent(metrics -> metrics.recordTimeBetweenNetworkThreadPoll(timeSinceLastPollMs));

final long pollWaitTimeMs = requestManagers.entries().stream()
.filter(Optional::isPresent)
.map(Optional::get)
Expand All @@ -162,15 +171,20 @@ void runOnce() {
private void processApplicationEvents() {
LinkedList<ApplicationEvent> events = new LinkedList<>();
applicationEventQueue.drainTo(events);
kafkaAsyncConsumerMetrics.ifPresent(metrics -> metrics.recordApplicationEventQueueSize(0));

for (ApplicationEvent event : events) {
long startMs = time.milliseconds();
kafkaAsyncConsumerMetrics.ifPresent(metrics -> metrics.recordApplicationEventQueueTime(startMs - event.enqueuedMs()));
try {
if (event instanceof CompletableEvent)
applicationEventReaper.add((CompletableEvent<?>) event);

applicationEventProcessor.process(event);
} catch (Throwable t) {
log.warn("Error processing event {}", t.getMessage(), t);
} finally {
kafkaAsyncConsumerMetrics.ifPresent(metrics -> metrics.recordApplicationEventQueueProcessingTime(time.milliseconds() - startMs));
}
}
}
Expand Down
Loading

0 comments on commit 20951a9

Please sign in to comment.