-
Notifications
You must be signed in to change notification settings - Fork 14.5k
MINOR: Improved ConsumerRebalanceListenerInvoker for code reusage #18202
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
Serwios
wants to merge
3
commits into
apache:trunk
Choose a base branch
from
Serwios:refactor_consumer_rebalance_listener
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.
Open
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,15 +35,25 @@ | |
/** | ||
* This class encapsulates the invocation of the callback methods defined in the {@link ConsumerRebalanceListener} | ||
* interface. When consumer group partition assignment changes, these methods are invoked. This class wraps those | ||
* callback calls with some logging, optional {@link Sensor} updates, etc. | ||
* callback calls with logging, optional {@link Sensor} updates, and latency recording. | ||
* <p> | ||
* It handles the invocation of partition assignment, revocation, and loss callbacks, ensuring that metrics are | ||
* recorded for the time taken to invoke each callback. | ||
*/ | ||
public class ConsumerRebalanceListenerInvoker { | ||
|
||
private final Logger log; | ||
private final SubscriptionState subscriptions; | ||
private final Time time; | ||
private final RebalanceCallbackMetricsManager metricsManager; | ||
|
||
/** | ||
* Constructs a new {@link ConsumerRebalanceListenerInvoker} with the specified dependencies. | ||
* | ||
* @param logContext the log context to create a logger for this class | ||
* @param subscriptions the subscriptions state for the consumer | ||
* @param time the time utility for obtaining the current time in milliseconds | ||
* @param metricsManager the metrics manager to record callback latency | ||
*/ | ||
ConsumerRebalanceListenerInvoker(LogContext logContext, | ||
SubscriptionState subscriptions, | ||
Time time, | ||
|
@@ -54,77 +64,102 @@ public class ConsumerRebalanceListenerInvoker { | |
this.metricsManager = metricsManager; | ||
} | ||
|
||
/** | ||
* Invokes the onPartitionsAssigned callback method from the rebalance listener and logs the result. | ||
* | ||
* @param assignedPartitions the partitions assigned to the consumer | ||
* @return an exception if an error occurred, or null if no error | ||
*/ | ||
public Exception invokePartitionsAssigned(final SortedSet<TopicPartition> assignedPartitions) { | ||
log.info("Adding newly assigned partitions: {}", assignedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
|
||
Optional<ConsumerRebalanceListener> listener = subscriptions.rebalanceListener(); | ||
return invokeRebalanceCallback("Adding newly assigned partitions", assignedPartitions, | ||
listener -> listener.onPartitionsAssigned(assignedPartitions), | ||
metricsManager::recordPartitionsAssignedLatency); | ||
} | ||
|
||
if (listener.isPresent()) { | ||
try { | ||
final long startMs = time.milliseconds(); | ||
listener.get().onPartitionsAssigned(assignedPartitions); | ||
metricsManager.recordPartitionsAssignedLatency(time.milliseconds() - startMs); | ||
} catch (WakeupException | InterruptException e) { | ||
throw e; | ||
} catch (Exception e) { | ||
log.error("User provided listener {} failed on invocation of onPartitionsAssigned for partitions {}", | ||
listener.get().getClass().getName(), assignedPartitions, e); | ||
return e; | ||
} | ||
} | ||
/** | ||
* Invokes the onPartitionsRevoked callback method from the rebalance listener and logs the result. | ||
* | ||
* @param revokedPartitions the partitions revoked from the consumer | ||
* @return an exception if an error occurred, or null if no error | ||
*/ | ||
public Exception invokePartitionsRevoked(final SortedSet<TopicPartition> revokedPartitions) { | ||
return invokeRebalanceCallback("Revoke previously assigned partitions", revokedPartitions, | ||
listener -> { | ||
Set<TopicPartition> revokePausedPartitions = subscriptions.pausedPartitions(); | ||
revokePausedPartitions.retainAll(revokedPartitions); | ||
if (!revokePausedPartitions.isEmpty()) { | ||
log.info("The pause flag in partitions [{}] will be removed due to revocation.", | ||
revokePausedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
} | ||
listener.onPartitionsRevoked(revokedPartitions); | ||
}, | ||
metricsManager::recordPartitionsRevokedLatency); | ||
} | ||
|
||
return null; | ||
/** | ||
* Invokes the onPartitionsLost callback method from the rebalance listener and logs the result. | ||
* | ||
* @param lostPartitions the partitions lost from the consumer | ||
* @return an exception if an error occurred, or null if no error | ||
*/ | ||
public Exception invokePartitionsLost(final SortedSet<TopicPartition> lostPartitions) { | ||
return invokeRebalanceCallback("Lost previously assigned partitions", lostPartitions, | ||
listener -> { | ||
Set<TopicPartition> lostPausedPartitions = subscriptions.pausedPartitions(); | ||
lostPausedPartitions.retainAll(lostPartitions); | ||
if (!lostPausedPartitions.isEmpty()) { | ||
log.info("The pause flag in partitions [{}] will be removed due to partition lost.", | ||
lostPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
Comment on lines
+111
to
+112
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There's a bug in the existing code: |
||
} | ||
listener.onPartitionsLost(lostPartitions); | ||
}, | ||
metricsManager::recordPartitionsLostLatency); | ||
} | ||
|
||
public Exception invokePartitionsRevoked(final SortedSet<TopicPartition> revokedPartitions) { | ||
log.info("Revoke previously assigned partitions {}", revokedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
Set<TopicPartition> revokePausedPartitions = subscriptions.pausedPartitions(); | ||
revokePausedPartitions.retainAll(revokedPartitions); | ||
if (!revokePausedPartitions.isEmpty()) | ||
log.info("The pause flag in partitions [{}] will be removed due to revocation.", revokePausedPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
/** | ||
* General method to invoke a rebalance callback method (assigned, revoked, or lost) and record latency. | ||
* | ||
* @param logMessage a message describing the action being taken (e.g., adding, revoking, losing) | ||
* @param partitions the partitions affected by the rebalance | ||
* @param invoker the specific method to invoke on the listener | ||
* @param latencyRecorder the method to record latency for the operation | ||
* @return an exception if an error occurred, or null if no error | ||
*/ | ||
private Exception invokeRebalanceCallback(String logMessage, SortedSet<TopicPartition> partitions, | ||
RebalanceListenerInvoker invoker, | ||
LatencyRecorder latencyRecorder) { | ||
log.info("{}: {}", logMessage, partitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
|
||
Optional<ConsumerRebalanceListener> listener = subscriptions.rebalanceListener(); | ||
|
||
if (listener.isPresent()) { | ||
try { | ||
final long startMs = time.milliseconds(); | ||
listener.get().onPartitionsRevoked(revokedPartitions); | ||
metricsManager.recordPartitionsRevokedLatency(time.milliseconds() - startMs); | ||
invoker.invoke(listener.get()); | ||
latencyRecorder.recordLatency(time.milliseconds() - startMs); | ||
} catch (WakeupException | InterruptException e) { | ||
throw e; | ||
} catch (Exception e) { | ||
log.error("User provided listener {} failed on invocation of onPartitionsRevoked for partitions {}", | ||
listener.get().getClass().getName(), revokedPartitions, e); | ||
log.error("User provided listener {} failed on invocation for partitions {}", | ||
listener.get().getClass().getName(), partitions, e); | ||
return e; | ||
} | ||
} | ||
|
||
return null; | ||
} | ||
|
||
public Exception invokePartitionsLost(final SortedSet<TopicPartition> lostPartitions) { | ||
log.info("Lost previously assigned partitions {}", lostPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
Set<TopicPartition> lostPausedPartitions = subscriptions.pausedPartitions(); | ||
lostPausedPartitions.retainAll(lostPartitions); | ||
if (!lostPausedPartitions.isEmpty()) | ||
log.info("The pause flag in partitions [{}] will be removed due to partition lost.", lostPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(", "))); | ||
|
||
Optional<ConsumerRebalanceListener> listener = subscriptions.rebalanceListener(); | ||
|
||
if (listener.isPresent()) { | ||
try { | ||
final long startMs = time.milliseconds(); | ||
listener.get().onPartitionsLost(lostPartitions); | ||
metricsManager.recordPartitionsLostLatency(time.milliseconds() - startMs); | ||
} catch (WakeupException | InterruptException e) { | ||
throw e; | ||
} catch (Exception e) { | ||
log.error("User provided listener {} failed on invocation of onPartitionsLost for partitions {}", | ||
listener.get().getClass().getName(), lostPartitions, e); | ||
return e; | ||
} | ||
} | ||
/** | ||
* Functional interface for invoking a rebalance callback method on the listener. | ||
*/ | ||
@FunctionalInterface | ||
private interface RebalanceListenerInvoker { | ||
void invoke(ConsumerRebalanceListener listener) throws Exception; | ||
} | ||
|
||
return null; | ||
/** | ||
* Functional interface for recording latency during rebalance callback invocations. | ||
*/ | ||
@FunctionalInterface | ||
private interface LatencyRecorder { | ||
void recordLatency(long latency); | ||
} | ||
} |
126 changes: 126 additions & 0 deletions
126
...ava/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvokerTest.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,126 @@ | ||
/* | ||
* 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; | ||
|
||
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; | ||
import org.apache.kafka.clients.consumer.internals.metrics.RebalanceCallbackMetricsManager; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.utils.LogContext; | ||
import org.apache.kafka.common.utils.Time; | ||
|
||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
import org.mockito.Mock; | ||
import org.mockito.MockitoAnnotations; | ||
import org.slf4j.Logger; | ||
|
||
import java.util.Comparator; | ||
import java.util.Optional; | ||
import java.util.SortedSet; | ||
import java.util.TreeSet; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertNull; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.ArgumentMatchers.anyLong; | ||
import static org.mockito.Mockito.never; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.when; | ||
|
||
class ConsumerRebalanceListenerInvokerTest { | ||
|
||
@Mock | ||
private LogContext logContext; | ||
@Mock | ||
private Logger log; | ||
@Mock | ||
private SubscriptionState subscriptions; | ||
@Mock | ||
private Time time; | ||
@Mock | ||
private RebalanceCallbackMetricsManager metricsManager; | ||
@Mock | ||
private ConsumerRebalanceListener listener; | ||
|
||
private ConsumerRebalanceListenerInvoker invoker; | ||
|
||
@BeforeEach | ||
public void setUp() { | ||
MockitoAnnotations.openMocks(this); | ||
|
||
when(logContext.logger(any(Class.class))).thenReturn(log); | ||
|
||
invoker = new ConsumerRebalanceListenerInvoker(logContext, subscriptions, time, metricsManager); | ||
|
||
when(subscriptions.rebalanceListener()).thenReturn(Optional.of(listener)); | ||
} | ||
|
||
@Test | ||
public void testInvokePartitionsAssigned() { | ||
SortedSet<TopicPartition> assignedPartitions = new TreeSet<>(Comparator.comparing(TopicPartition::toString)); | ||
|
||
TopicPartition tp1 = new TopicPartition("testTopic", 0); | ||
assignedPartitions.add(tp1); | ||
|
||
Exception result = invoker.invokePartitionsAssigned(assignedPartitions); | ||
|
||
assertNull(result); | ||
verify(listener).onPartitionsAssigned(assignedPartitions); | ||
verify(metricsManager).recordPartitionsAssignedLatency(anyLong()); | ||
} | ||
|
||
@Test | ||
public void testInvokePartitionsRevoked() { | ||
SortedSet<TopicPartition> revokedPartitions = new TreeSet<>(Comparator.comparing(TopicPartition::toString)); | ||
|
||
TopicPartition tp1 = new TopicPartition("testTopic", 0); | ||
revokedPartitions.add(tp1); | ||
|
||
Exception result = invoker.invokePartitionsRevoked(revokedPartitions); | ||
|
||
assertNull(result); | ||
verify(listener).onPartitionsRevoked(revokedPartitions); | ||
verify(metricsManager).recordPartitionsRevokedLatency(anyLong()); | ||
} | ||
|
||
@Test | ||
public void testInvokePartitionsLost() { | ||
SortedSet<TopicPartition> lostPartitions = new TreeSet<>(Comparator.comparing(TopicPartition::toString)); | ||
|
||
TopicPartition tp1 = new TopicPartition("testTopic", 0); | ||
lostPartitions.add(tp1); | ||
|
||
Exception result = invoker.invokePartitionsLost(lostPartitions); | ||
|
||
assertNull(result); | ||
verify(listener).onPartitionsLost(lostPartitions); | ||
verify(metricsManager).recordPartitionsLostLatency(anyLong()); | ||
} | ||
|
||
@Test | ||
public void testInvokeWithNoListener() { | ||
when(subscriptions.rebalanceListener()).thenReturn(Optional.empty()); | ||
|
||
SortedSet<TopicPartition> assignedPartitions = new TreeSet<>(Comparator.comparing(TopicPartition::toString)); | ||
TopicPartition tp1 = new TopicPartition("testTopic", 0); | ||
assignedPartitions.add(tp1); | ||
|
||
Exception result = invoker.invokePartitionsAssigned(assignedPartitions); | ||
|
||
assertNull(result); | ||
verify(listener, never()).onPartitionsAssigned(assignedPartitions); | ||
} | ||
} |
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.
What do you think about extracting this logic from
invokePartitionsRevoked()
andinvokePartitionsLost()
and making it a separate utility method? Something like: