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 Sep 15, 2024
1 parent 7a321f2 commit b1b848f
Show file tree
Hide file tree
Showing 11 changed files with 293 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.clientTelemetryReporter = CommonClientConfigs.telemetryReporter(clientId, config);
this.clientTelemetryReporter.ifPresent(reporters::add);
this.metrics = createMetrics(config, time, reporters);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX, GroupProtocol.CONSUMER);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);

List<ConsumerInterceptor<K, V>> interceptorList = configuredConsumerInterceptors(config);
Expand All @@ -320,7 +321,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {

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

// This FetchBuffer is shared between the application and network threads.
this.fetchBuffer = new FetchBuffer(logContext);
Expand Down Expand Up @@ -363,7 +364,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
new CompletableEventReaper(logContext),
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
requestManagersSupplier,
Optional.of(kafkaConsumerMetrics));

ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
Expand All @@ -385,8 +387,6 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
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 @@ -443,7 +443,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.deserializers = deserializers;
this.applicationEventHandler = applicationEventHandler;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX, GroupProtocol.CONSUMER);
this.clientTelemetryReporter = Optional.empty();
this.autoCommitEnabled = autoCommitEnabled;
this.offsetCommitCallbackInvoker = new OffsetCommitCallbackInvoker(interceptors);
Expand Down Expand Up @@ -483,7 +483,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
deserializers,
fetchMetricsManager,
time);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX, GroupProtocol.CONSUMER);

GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(
config,
Expand All @@ -494,7 +494,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {

BlockingQueue<ApplicationEvent> applicationEventQueue = new LinkedBlockingQueue<>();
this.backgroundEventQueue = new LinkedBlockingQueue<>();
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue);
BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, Optional.of(kafkaConsumerMetrics));
ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
logContext,
subscriptions,
Expand Down Expand Up @@ -541,7 +541,8 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
new CompletableEventReaper(logContext),
applicationEventProcessorSupplier,
networkClientDelegateSupplier,
requestManagersSupplier);
requestManagersSupplier,
Optional.of(kafkaConsumerMetrics));
this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker);
this.backgroundEventReaper = new CompletableEventReaper(logContext);
}
Expand All @@ -556,7 +557,8 @@ ApplicationEventHandler build(
final CompletableEventReaper applicationEventReaper,
final Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
final Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
final Supplier<RequestManagers> requestManagersSupplier
final Supplier<RequestManagers> requestManagersSupplier,
final Optional<KafkaConsumerMetrics> kafkaConsumerMetrics
);

}
Expand Down Expand Up @@ -1846,8 +1848,11 @@ private boolean processBackgroundEvents() {

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

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

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
retryBackoffMs,
retryBackoffMaxMs);

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

config.logUnused();
AppInfoParser.registerAppInfo(CONSUMER_JMX_PREFIX, clientId, metrics, time.milliseconds());
Expand Down Expand Up @@ -292,7 +292,7 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.isolationLevel = ConsumerUtils.configuredIsolationLevel(config);
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.assignors = assignors;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX, GroupProtocol.CLASSIC);
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList());
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
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.KafkaConsumerMetrics;
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<KafkaConsumerMetrics> kafkaConsumerMetrics;
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<KafkaConsumerMetrics> kafkaConsumerMetrics) {
super(BACKGROUND_THREAD_NAME, true);
this.time = time;
this.log = logContext.logger(getClass());
Expand All @@ -84,6 +88,18 @@ public ConsumerNetworkThread(LogContext logContext,
this.networkClientDelegateSupplier = networkClientDelegateSupplier;
this.requestManagersSupplier = requestManagersSupplier;
this.running = true;
this.kafkaConsumerMetrics = kafkaConsumerMetrics;
}

public ConsumerNetworkThread(LogContext logContext,
Time time,
BlockingQueue<ApplicationEvent> applicationEventQueue,
CompletableEventReaper applicationEventReaper,
Supplier<ApplicationEventProcessor> applicationEventProcessorSupplier,
Supplier<NetworkClientDelegate> networkClientDelegateSupplier,
Supplier<RequestManagers> requestManagersSupplier) {
this(logContext, time, applicationEventQueue, applicationEventReaper, applicationEventProcessorSupplier,
networkClientDelegateSupplier, requestManagersSupplier, Optional.empty());
}

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

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

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

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

applicationEventProcessor.process(event);
} catch (Throwable t) {
log.warn("Error processing event {}", t.getMessage(), t);
} finally {
kafkaConsumerMetrics.ifPresent(metrics -> metrics.recordApplicationEventQueueProcessingTime(time.milliseconds() - startMs));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler;
import org.apache.kafka.clients.consumer.internals.events.ErrorEvent;
import org.apache.kafka.clients.consumer.internals.metrics.KafkaConsumerMetrics;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.DisconnectException;
Expand Down Expand Up @@ -69,14 +70,16 @@ public class NetworkClientDelegate implements AutoCloseable {
private final int requestTimeoutMs;
private final Queue<UnsentRequest> unsentRequests;
private final long retryBackoffMs;
private final Optional<KafkaConsumerMetrics> kafkaConsumerMetrics;

public NetworkClientDelegate(
final Time time,
final ConsumerConfig config,
final LogContext logContext,
final KafkaClient client,
final Metadata metadata,
final BackgroundEventHandler backgroundEventHandler) {
final BackgroundEventHandler backgroundEventHandler,
final Optional<KafkaConsumerMetrics> kafkaConsumerMetrics) {
this.time = time;
this.client = client;
this.metadata = metadata;
Expand All @@ -85,6 +88,17 @@ public NetworkClientDelegate(
this.unsentRequests = new ArrayDeque<>();
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.kafkaConsumerMetrics = kafkaConsumerMetrics;
}

public NetworkClientDelegate(
final Time time,
final ConsumerConfig config,
final LogContext logContext,
final KafkaClient client,
final Metadata metadata,
final BackgroundEventHandler backgroundEventHandler) {
this(time, config, logContext, client, metadata, backgroundEventHandler, Optional.empty());
}

// Visible for testing
Expand Down Expand Up @@ -170,6 +184,7 @@ private void trySend(final long currentTimeMs) {
Iterator<UnsentRequest> iterator = unsentRequests.iterator();
while (iterator.hasNext()) {
UnsentRequest unsent = iterator.next();
kafkaConsumerMetrics.ifPresent(metrics -> metrics.recordUnsentRequestsQueueTime(currentTimeMs - unsent.addedToQueueMs()));
unsent.timer.update(currentTimeMs);
if (unsent.timer.isExpired()) {
iterator.remove();
Expand Down Expand Up @@ -267,7 +282,9 @@ public void addAll(final List<UnsentRequest> requests) {
public void add(final UnsentRequest r) {
Objects.requireNonNull(r);
r.setTimer(this.time, this.requestTimeoutMs);
r.setAddedToQueueMs(this.time.milliseconds());
unsentRequests.add(r);
kafkaConsumerMetrics.ifPresent(metrics -> metrics.recordUnsentRequestsQueueSize(unsentRequests.size()));
}

public static class PollResult {
Expand Down Expand Up @@ -300,6 +317,7 @@ public static class UnsentRequest {
private final Optional<Node> node; // empty if random node can be chosen

private Timer timer;
private long addedToQueueMs;

public UnsentRequest(final AbstractRequest.Builder<?> requestBuilder,
final Optional<Node> node) {
Expand All @@ -317,6 +335,14 @@ Timer timer() {
return timer;
}

void setAddedToQueueMs(final long addedToQueueMs) {
this.addedToQueueMs = addedToQueueMs;
}

long addedToQueueMs() {
return addedToQueueMs;
}

CompletableFuture<ClientResponse> future() {
return handler.future;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ public enum Type {
* {@link #equals(Object)} and can be used in log messages when debugging.
*/
private final Uuid id;
private long addedToQueueMs;

protected ApplicationEvent(Type type) {
this.type = Objects.requireNonNull(type);
Expand All @@ -60,6 +61,14 @@ public Uuid id() {
return id;
}

public void setAddedToQueueMs(long addedToQueueMs) {
this.addedToQueueMs = addedToQueueMs;
}

public long addedToQueueMs() {
return addedToQueueMs;
}

@Override
public final boolean equals(Object o) {
if (this == o) return true;
Expand Down
Loading

0 comments on commit b1b848f

Please sign in to comment.