-
Notifications
You must be signed in to change notification settings - Fork 14.6k
KAFKA-19691: Add metrics corresponding to consumer rebalance listener metrics #20464
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
RaidenE1
wants to merge
4
commits into
apache:trunk
Choose a base branch
from
RaidenE1:consumer-rebalance-listener-metrics
base: trunk
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+682
−18
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
156 changes: 156 additions & 0 deletions
156
.../src/main/java/org/apache/kafka/streams/processor/internals/metrics/RebalanceMetrics.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,156 @@ | ||
/* | ||
* 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.processor.internals.metrics; | ||
|
||
import org.apache.kafka.common.metrics.Sensor; | ||
import org.apache.kafka.common.metrics.Sensor.RecordingLevel; | ||
|
||
import java.util.Map; | ||
|
||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATENCY_SUFFIX; | ||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.THREAD_LEVEL_GROUP; | ||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addAvgAndMaxToSensor; | ||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addValueMetricToSensor; | ||
|
||
public class RebalanceMetrics { | ||
private RebalanceMetrics() {} | ||
|
||
private static final String TASKS_REVOKED = "tasks-revoked"; | ||
private static final String TASKS_ASSIGNED = "tasks-assigned"; | ||
private static final String TASKS_LOST = "tasks-lost"; | ||
|
||
private static final String TASKS_REVOKED_AVG_LATENCY_DESCRIPTION = "The average time taken for tasks-revoked rebalance listener callback"; | ||
private static final String TASKS_REVOKED_MAX_LATENCY_DESCRIPTION = "The max time taken for tasks-revoked rebalance listener callback"; | ||
private static final String TASKS_ASSIGNED_AVG_LATENCY_DESCRIPTION = "The average time taken for tasks-assigned rebalance listener callback"; | ||
private static final String TASKS_ASSIGNED_MAX_LATENCY_DESCRIPTION = "The max time taken for tasks-assigned rebalance listener callback"; | ||
private static final String TASKS_LOST_AVG_LATENCY_DESCRIPTION = "The average time taken for tasks-lost rebalance listener callback"; | ||
private static final String TASKS_LOST_MAX_LATENCY_DESCRIPTION = "The max time taken for tasks-lost rebalance listener callback"; | ||
|
||
// Task count metric descriptions | ||
private static final String TASKS_REVOKED_TOTAL_DESCRIPTION = "The total number of tasks revoked during rebalance"; | ||
private static final String TASKS_ASSIGNED_ACTIVE_TOTAL_DESCRIPTION = "The total number of active tasks assigned during rebalance"; | ||
private static final String TASKS_ASSIGNED_STANDBY_TOTAL_DESCRIPTION = "The total number of standby tasks assigned during rebalance"; | ||
private static final String TASKS_LOST_TOTAL_DESCRIPTION = "The total number of tasks lost during rebalance"; | ||
|
||
public static Sensor tasksRevokedSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return rebalanceLatencySensor( | ||
threadId, | ||
TASKS_REVOKED, | ||
TASKS_REVOKED_AVG_LATENCY_DESCRIPTION, | ||
TASKS_REVOKED_MAX_LATENCY_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
public static Sensor tasksAssignedSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return rebalanceLatencySensor( | ||
threadId, | ||
TASKS_ASSIGNED, | ||
TASKS_ASSIGNED_AVG_LATENCY_DESCRIPTION, | ||
TASKS_ASSIGNED_MAX_LATENCY_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
public static Sensor tasksLostSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return rebalanceLatencySensor( | ||
threadId, | ||
TASKS_LOST, | ||
TASKS_LOST_AVG_LATENCY_DESCRIPTION, | ||
TASKS_LOST_MAX_LATENCY_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
private static Sensor rebalanceLatencySensor(final String threadId, | ||
final String operation, | ||
final String avgDescription, | ||
final String maxDescription, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
final Sensor sensor = streamsMetrics.threadLevelSensor(threadId, operation + LATENCY_SUFFIX, RecordingLevel.INFO); | ||
final Map<String, String> tagMap = streamsMetrics.threadLevelTagMap(threadId); | ||
addAvgAndMaxToSensor( | ||
sensor, | ||
THREAD_LEVEL_GROUP, | ||
tagMap, | ||
operation + LATENCY_SUFFIX, | ||
avgDescription, | ||
maxDescription | ||
); | ||
return sensor; | ||
} | ||
|
||
// Task count metrics (DEBUG level) | ||
public static Sensor tasksRevokedTotalSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return taskCountSensor( | ||
threadId, | ||
TASKS_REVOKED + "-total", | ||
TASKS_REVOKED_TOTAL_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
public static Sensor tasksAssignedActiveTotalSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return taskCountSensor( | ||
threadId, | ||
TASKS_ASSIGNED + "-active-total", | ||
TASKS_ASSIGNED_ACTIVE_TOTAL_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
public static Sensor tasksAssignedStandbyTotalSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return taskCountSensor( | ||
threadId, | ||
TASKS_ASSIGNED + "-standby-total", | ||
TASKS_ASSIGNED_STANDBY_TOTAL_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
public static Sensor tasksLostTotalSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
return taskCountSensor( | ||
threadId, | ||
TASKS_LOST + "-total", | ||
TASKS_LOST_TOTAL_DESCRIPTION, | ||
streamsMetrics | ||
); | ||
} | ||
|
||
private static Sensor taskCountSensor(final String threadId, | ||
final String metricName, | ||
final String description, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
final Sensor sensor = streamsMetrics.threadLevelSensor(threadId, metricName, RecordingLevel.DEBUG); | ||
final Map<String, String> tagMap = streamsMetrics.threadLevelTagMap(threadId); | ||
addValueMetricToSensor( | ||
sensor, | ||
THREAD_LEVEL_GROUP, | ||
tagMap, | ||
metricName, | ||
description | ||
); | ||
return sensor; | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The start time is captured outside the try block, but if an exception occurs during the method execution, the sensor will still record the latency in the finally block. This could lead to recording metrics for failed operations that didn't complete normally, potentially skewing the latency measurements.
Copilot uses AI. Check for mistakes.