Skip to content

KAFKA-17503: Fix incorrect calculation of poll-idle-ratio-avg for con… #17139

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 4 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,7 @@
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.configuredConsumerInterceptors;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetricConfigWithoutTags;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs;
Expand Down Expand Up @@ -467,7 +468,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.isolationLevel = IsolationLevel.READ_UNCOMMITTED;
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList());
this.time = time;
this.metrics = new Metrics(time);
this.metrics = new Metrics(createMetricConfigWithoutTags(config), time);
this.metadata = metadata;
this.metadataVersionSnapshot = metadata.updateVersion();
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createConsumerNetworkClient;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createFetchMetricsManager;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createLogContext;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetricConfigWithoutTags;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createMetrics;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.createSubscriptionState;
import static org.apache.kafka.common.utils.Utils.closeQuietly;
Expand Down Expand Up @@ -286,7 +287,7 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.time = time;
this.subscriptions = subscriptions;
this.metadata = metadata;
this.metrics = new Metrics(time);
this.metrics = new Metrics(createMetricConfigWithoutTags(config), time);
this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
this.groupId = Optional.ofNullable(config.getString(ConsumerConfig.GROUP_ID_CONFIG));
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,16 +143,20 @@ public static Metrics createMetrics(ConsumerConfig config, Time time) {
public static Metrics createMetrics(ConsumerConfig config, Time time, List<MetricsReporter> reporters) {
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
Map<String, String> metricsTags = Collections.singletonMap(CONSUMER_CLIENT_ID_METRIC_TAG, clientId);
MetricConfig metricConfig = new MetricConfig()
.samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS)
.recordLevel(Sensor.RecordingLevel.forName(config.getString(ConsumerConfig.METRICS_RECORDING_LEVEL_CONFIG)))
MetricConfig metricConfig = createMetricConfigWithoutTags(config)
.tags(metricsTags);
MetricsContext metricsContext = new KafkaMetricsContext(CONSUMER_JMX_PREFIX,
config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX));
return new Metrics(metricConfig, reporters, time, metricsContext);
}

public static MetricConfig createMetricConfigWithoutTags(ConsumerConfig config) {
return new MetricConfig()
.samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS)
.recordLevel(Sensor.RecordingLevel.forName(config.getString(ConsumerConfig.METRICS_RECORDING_LEVEL_CONFIG)));
}

public static FetchMetricsManager createFetchMetricsManager(Metrics metrics) {
Set<String> metricsTags = Collections.singleton(CONSUMER_CLIENT_ID_METRIC_TAG);
FetchMetricsRegistry metricsRegistry = new FetchMetricsRegistry(metricsTags, CONSUMER_METRIC_GROUP_PREFIX);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,8 @@ public class KafkaConsumerMetrics implements AutoCloseable {
private final Sensor pollIdleSensor;
private final Sensor committedSensor;
private final Sensor commitSyncSensor;
private final TimeRatio2 pollIdleRatio;
private long lastPollMs;
private long pollStartMs;
private long timeSinceLastPollMs;

public KafkaConsumerMetrics(Metrics metrics, String metricGrpPrefix) {
this.metrics = metrics;
Expand Down Expand Up @@ -68,6 +67,7 @@ public KafkaConsumerMetrics(Metrics metrics, String metricGrpPrefix) {
metricGroupName,
"The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records."),
new Avg());
this.pollIdleRatio = new TimeRatio2(metrics.config().timeWindowMs(), pollIdleSensor);
Copy link
Contributor Author

@srdo srdo Dec 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a bit unsure whether it would be better to select a smaller time window here. The defaults for the metrics system is a window of 30 seconds, which is a bit large for this purpose, and it also only keeps 2 samples. It might make sense to aim for e.g. a sample every few seconds, or 1/5th the window size the metric system uses, or something like that.


this.commitSyncSensor = metrics.sensor("commit-sync-time-ns-total");
this.commitSyncSensor.add(
Expand All @@ -91,16 +91,14 @@ public KafkaConsumerMetrics(Metrics metrics, String metricGrpPrefix) {
}

public void recordPollStart(long pollStartMs) {
this.pollStartMs = pollStartMs;
this.timeSinceLastPollMs = lastPollMs != 0L ? pollStartMs - lastPollMs : 0;
this.pollIdleRatio.recordEventStart(pollStartMs);
long timeSinceLastPollMs = lastPollMs != 0L ? pollStartMs - lastPollMs : 0;
this.timeBetweenPollSensor.record(timeSinceLastPollMs);
this.lastPollMs = pollStartMs;
}

public void recordPollEnd(long pollEndMs) {
long pollTimeMs = pollEndMs - pollStartMs;
double pollIdleRatio = pollTimeMs * 1.0 / (pollTimeMs + timeSinceLastPollMs);
this.pollIdleSensor.record(pollIdleRatio);
this.pollIdleRatio.recordEventEnd(pollEndMs);
}

public void recordCommitSync(long duration) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals.metrics;

import org.apache.kafka.common.metrics.Sensor;

/**
* Maintains a ratio of the duration of a periodic event
* over a rolling fixed-size window, emitting the measured ratio to the specified sensor for each window.
* For example, this can be used to compute the ratio of
* time that a thread is busy or idle.
*/
public class TimeRatio2 {

private final long windowSizeMs;
private final Sensor sensor;
private long lastWindowEndMs;
private long msSpentInEventInCurrentWindow;
private long msSpentOutsideEventInCurrentWindow;
private long lastEventStartMs;
private long lastEventEndMs;

public TimeRatio2(long windowSizeMs, Sensor sensor) {
if (windowSizeMs < 0.0) {
throw new IllegalArgumentException("Invalid window size: value " + windowSizeMs + " is less than 0.");
}
this.windowSizeMs = windowSizeMs;
this.sensor = sensor;
}

private void updateWindow(long nowMs, long msSpentInEvent, long msSpentOutsideEvent) {
long msSinceLastSamplingIntervalEnd = nowMs - this.lastWindowEndMs;
if (msSinceLastSamplingIntervalEnd < this.windowSizeMs) {
// We can't fill a window yet, so just add the last event's millis to the counters
this.msSpentInEventInCurrentWindow += msSpentInEvent;
this.msSpentOutsideEventInCurrentWindow += msSpentOutsideEvent;
} else {
// We can fill at least one window.
// Add millis from the last event to fill this window and flush the ratio to the sensor,
// then start a new window with any remaining millis we couldn't fit in the flushed window.
// As each event period starts with "inside event" time and ends with "outside event" time,
// we prefer to insert "inside event" time into the window first.
long missingMsToCompleteWindow = this.windowSizeMs - (this.msSpentInEventInCurrentWindow + this.msSpentOutsideEventInCurrentWindow);
long inEventMsFittingInWindow = Math.min(missingMsToCompleteWindow, msSpentInEvent);
long outsideEventMsFittingInWindow = missingMsToCompleteWindow - inEventMsFittingInWindow;

double eventRatio = (this.msSpentInEventInCurrentWindow + inEventMsFittingInWindow) * 1.0 / this.windowSizeMs;
long currentWindowEndMs = this.lastWindowEndMs + this.windowSizeMs;
this.sensor.record(eventRatio, currentWindowEndMs);

this.msSpentInEventInCurrentWindow = 0;
this.msSpentOutsideEventInCurrentWindow = 0;
this.lastWindowEndMs = currentWindowEndMs;
updateWindow(nowMs, msSpentInEvent - inEventMsFittingInWindow, msSpentOutsideEvent - outsideEventMsFittingInWindow);
}
}

public void recordEventStart(long nowMs) {
if (this.lastWindowEndMs == 0) {
this.lastWindowEndMs = nowMs;
} else {
long msSpentInEvent = this.lastEventEndMs - this.lastEventStartMs;
long msSpentOutsideEvent = nowMs - this.lastEventEndMs;
updateWindow(nowMs, msSpentInEvent, msSpentOutsideEvent);
}
this.lastEventStartMs = nowMs;
}

public void recordEventEnd(long nowMs) {
this.lastEventEndMs = nowMs;
}

}
Loading
Loading