Skip to content
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 @@ -19,8 +19,11 @@
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData;
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceListener;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.RebalanceMetrics;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;

import org.slf4j.Logger;

Expand All @@ -38,17 +41,37 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener
private final StreamsRebalanceData streamsRebalanceData;
private final TaskManager taskManager;
private final StreamThread streamThread;
private final Sensor tasksRevokedSensor;
private final Sensor tasksAssignedSensor;
private final Sensor tasksLostSensor;
private final Sensor tasksRevokedTotalSensor;
private final Sensor tasksAssignedActiveTotalSensor;
private final Sensor tasksAssignedStandbyTotalSensor;
private final Sensor tasksLostTotalSensor;

public DefaultStreamsRebalanceListener(final Logger log,
final Time time,
final StreamsRebalanceData streamsRebalanceData,
final StreamThread streamThread,
final TaskManager taskManager) {
final TaskManager taskManager,
final StreamsMetricsImpl streamsMetrics,
final String threadId) {
this.log = log;
this.time = time;
this.streamsRebalanceData = streamsRebalanceData;
this.streamThread = streamThread;
this.taskManager = taskManager;

// Create sensors for rebalance metrics
this.tasksRevokedSensor = RebalanceMetrics.tasksRevokedSensor(threadId, streamsMetrics);
this.tasksAssignedSensor = RebalanceMetrics.tasksAssignedSensor(threadId, streamsMetrics);
this.tasksLostSensor = RebalanceMetrics.tasksLostSensor(threadId, streamsMetrics);

// Create sensors for task count metrics (DEBUG level)
this.tasksRevokedTotalSensor = RebalanceMetrics.tasksRevokedTotalSensor(threadId, streamsMetrics);
this.tasksAssignedActiveTotalSensor = RebalanceMetrics.tasksAssignedActiveTotalSensor(threadId, streamsMetrics);
this.tasksAssignedStandbyTotalSensor = RebalanceMetrics.tasksAssignedStandbyTotalSensor(threadId, streamsMetrics);
this.tasksLostTotalSensor = RebalanceMetrics.tasksLostTotalSensor(threadId, streamsMetrics);
}

@Override
Expand All @@ -65,7 +88,10 @@ public Optional<Exception> onTasksRevoked(final Set<StreamsRebalanceData.TaskId>
log.info("Revoking active tasks {}.", tasks);
taskManager.handleRevocation(partitionsToRevoke);
} finally {
log.info("partition revocation took {} ms.", time.milliseconds() - start);
final long latency = time.milliseconds() - start;
tasksRevokedSensor.record(latency);
tasksRevokedTotalSensor.record(tasks.size());
log.info("partition revocation took {} ms for {} tasks.", latency, tasks.size());
}
if (streamThread.state() != StreamThread.State.PENDING_SHUTDOWN) {
streamThread.setState(StreamThread.State.PARTITIONS_REVOKED);
Expand All @@ -78,6 +104,7 @@ public Optional<Exception> onTasksRevoked(final Set<StreamsRebalanceData.TaskId>

@Override
public Optional<Exception> onTasksAssigned(final StreamsRebalanceData.Assignment assignment) {
final long start = time.milliseconds();
try {
Comment on lines 106 to 108
Copy link
Preview

Copilot AI Sep 3, 2025

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.

final Map<TaskId, Set<TopicPartition>> activeTasksWithPartitions =
pairWithTopicPartitions(assignment.activeTasks().stream());
Expand All @@ -91,16 +118,35 @@ public Optional<Exception> onTasksAssigned(final StreamsRebalanceData.Assignment
taskManager.handleRebalanceComplete();
} catch (final Exception exception) {
return Optional.of(exception);
} finally {
final long latency = time.milliseconds() - start;
tasksAssignedSensor.record(latency);
tasksAssignedActiveTotalSensor.record(assignment.activeTasks().size());
tasksAssignedStandbyTotalSensor.record(assignment.standbyTasks().size() + assignment.warmupTasks().size());
log.info("task assignment took {} ms for {} active tasks and {} standby tasks.",
latency, assignment.activeTasks().size(), assignment.standbyTasks().size() + assignment.warmupTasks().size());
}
return Optional.empty();
}

@Override
public Optional<Exception> onAllTasksLost() {
final long start = time.milliseconds();
// Get current task counts before they are lost
final int activeTaskCount = taskManager.activeTaskIds().size();
final int standbyTaskCount = taskManager.standbyTaskIds().size();
final int totalTaskCount = activeTaskCount + standbyTaskCount;

try {
taskManager.handleLostAll();
} catch (final Exception exception) {
return Optional.of(exception);
} finally {
final long latency = time.milliseconds() - start;
tasksLostSensor.record(latency);
tasksLostTotalSensor.record(totalTaskCount);
log.info("handling lost all tasks took {} ms for {} tasks ({} active, {} standby).",
latency, totalTaskCount, activeTaskCount, standbyTaskCount);
}
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1151,7 +1151,9 @@ private void subscribeConsumer() {
time,
streamsRebalanceData.get(),
this,
taskManager
taskManager,
streamsMetrics,
getName()
)
);
} else {
Expand All @@ -1162,7 +1164,9 @@ private void subscribeConsumer() {
time,
streamsRebalanceData.get(),
this,
taskManager
taskManager,
streamsMetrics,
getName()
)
);
}
Expand Down
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;
}
}
Loading
Loading