-
Notifications
You must be signed in to change notification settings - Fork 13.8k
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
KAFKA-16933: New consumer unsubscribe close commit fixes #16272
Changes from 13 commits
27fdd0d
a1f17a6
3fadad7
bfae065
a44e2ac
19b7e9d
51d1bf9
e46e071
238d907
20fc8a8
64327dd
fe5ddf6
3dcd2b1
46c75fc
5a4402b
56ac969
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1278,12 +1278,10 @@ void prepareShutdown(final Timer timer, final AtomicReference<Throwable> firstEx | |
autoCommitSync(timer); | ||
|
||
applicationEventHandler.add(new CommitOnCloseEvent()); | ||
completeQuietly( | ||
() -> { | ||
maybeRevokePartitions(); | ||
applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer))); | ||
}, | ||
"Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); | ||
completeQuietly(() -> maybeRevokePartitions(), | ||
"Failed to execute callback to release assignment", firstException); | ||
completeQuietly(() -> applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer))), | ||
"Failed to send leave group heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); | ||
} | ||
|
||
// Visible for testing | ||
|
@@ -1324,6 +1322,7 @@ void completeQuietly(final Utils.ThrowingRunnable function, | |
} catch (TimeoutException e) { | ||
log.debug("Timeout expired before the {} operation could complete.", msg); | ||
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. Should we update 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. Totally, good catch. |
||
} catch (Exception e) { | ||
log.error(msg, e); | ||
firstException.compareAndSet(null, e); | ||
} | ||
} | ||
|
@@ -1502,7 +1501,8 @@ public void unsubscribe() { | |
Timer timer = time.timer(Long.MAX_VALUE); | ||
UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer)); | ||
applicationEventHandler.add(unsubscribeEvent); | ||
log.info("Unsubscribing all topics or patterns and assigned partitions"); | ||
log.info("Unsubscribing all topics or patterns and assigned partitions {}", | ||
subscriptions.assignedPartitions()); | ||
|
||
try { | ||
processBackgroundEvents(unsubscribeEvent.future(), timer); | ||
|
@@ -1512,7 +1512,9 @@ public void unsubscribe() { | |
} | ||
resetGroupMetadata(); | ||
} | ||
subscriptions.unsubscribe(); | ||
} catch (Exception e) { | ||
log.error("Unsubscribe failed", e); | ||
throw e; | ||
} finally { | ||
release(); | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -520,12 +520,15 @@ private void replaceTargetAssignmentWithNewAssignment( | |
@Override | ||
public void transitionToFenced() { | ||
if (state == MemberState.PREPARE_LEAVING) { | ||
log.debug("Member {} with epoch {} got fenced but it is already preparing to leave " + | ||
log.info("Member {} with epoch {} got fenced but it is already preparing to leave " + | ||
"the group, so it will stop sending heartbeat and won't attempt to rejoin.", | ||
memberId, memberEpoch); | ||
// Transition to UNSUBSCRIBED, ensuring that the member (that is not part of the | ||
// group anymore from the broker point of view) will stop sending heartbeats while it | ||
// completes the ongoing leaving operation. | ||
// Briefly transition to LEAVING to ensure all required actions are applied even | ||
// though there is no need to send a leave group heartbeat (ex. clear epoch and | ||
// notify epoch listeners). Then transition to UNSUBSCRIBED, ensuring that the member | ||
// (that is not part of the group anymore from the broker point of view) will stop | ||
// sending heartbeats while it completes the ongoing leaving operation. | ||
transitionToSendingLeaveGroup(false); | ||
transitionTo(MemberState.UNSUBSCRIBED); | ||
return; | ||
} | ||
|
@@ -664,6 +667,7 @@ public CompletableFuture<Void> leaveGroup() { | |
if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) { | ||
// Member already leaving. No-op and return existing leave group future that will | ||
// complete when the ongoing leave operation completes. | ||
log.debug("Leave group operation already in progress for member {}", memberId); | ||
return leaveGroupInProgress.get(); | ||
} | ||
|
||
|
@@ -673,7 +677,15 @@ public CompletableFuture<Void> leaveGroup() { | |
|
||
CompletableFuture<Void> callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); | ||
callbackResult.whenComplete((result, error) -> { | ||
if (error != null) { | ||
log.error("Member {} callback to release assignment failed. Member will proceed " + | ||
"to send leave group heartbeat", memberId, error); | ||
} else { | ||
log.debug("Member {} completed callback to release assignment and will send leave " + | ||
"group heartbeat", memberId); | ||
} | ||
// Clear the subscription, no matter if the callback execution failed or succeeded. | ||
subscriptions.unsubscribe(); | ||
clearSubscription(); | ||
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. Is the name 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. you're right, it's clearing assignments, renamed. |
||
|
||
// Transition to ensure that a heartbeat request is sent out to effectively leave the | ||
|
@@ -705,6 +717,9 @@ private CompletableFuture<Void> invokeOnPartitionsRevokedOrLostToReleaseAssignme | |
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); | ||
droppedPartitions.addAll(subscriptions.assignedPartitions()); | ||
|
||
log.debug("Member {} is triggering callbacks to release assignment {} and leave group", | ||
memberId, droppedPartitions); | ||
|
||
CompletableFuture<Void> callbackResult; | ||
if (droppedPartitions.isEmpty()) { | ||
// No assignment to release. | ||
|
@@ -764,7 +779,7 @@ public void transitionToSendingLeaveGroup(boolean dueToExpiredPollTimer) { | |
* This also includes the latest member ID in the notification. If the member fails or leaves | ||
* the group, this will be invoked with empty epoch and member ID. | ||
*/ | ||
private void notifyEpochChange(Optional<Integer> epoch, Optional<String> memberId) { | ||
void notifyEpochChange(Optional<Integer> epoch, Optional<String> memberId) { | ||
stateUpdatesListeners.forEach(stateListener -> stateListener.onMemberEpochUpdated(epoch, memberId)); | ||
} | ||
|
||
|
@@ -794,8 +809,12 @@ public void onHeartbeatRequestSent() { | |
} | ||
} else if (state == MemberState.LEAVING) { | ||
if (isPollTimerExpired) { | ||
log.debug("Member {} sent heartbeat to leave due to expired poll timer. It will " + | ||
"remain stale (no heartbeat) until it rejoins the group on the next consumer " + | ||
"poll.", memberId); | ||
transitionToStale(); | ||
} else { | ||
log.debug("Member {} sent heartbeat to leave group.", memberId); | ||
transitionToUnsubscribed(); | ||
} | ||
} | ||
|
@@ -939,11 +958,13 @@ void maybeReconcile() { | |
revokedPartitions.addAll(ownedPartitions); | ||
revokedPartitions.removeAll(assignedTopicPartitions); | ||
|
||
log.info("Updating assignment with local epoch {}\n" + | ||
log.info("Reconciling assignment with local epoch {}\n" + | ||
"\tMember: {}\n" + | ||
"\tAssigned partitions: {}\n" + | ||
"\tCurrent owned partitions: {}\n" + | ||
"\tAdded partitions (assigned - owned): {}\n" + | ||
"\tRevoked partitions (owned - assigned): {}\n", | ||
memberId, | ||
resolvedAssignment.localEpoch, | ||
assignedTopicPartitions, | ||
ownedPartitions, | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -158,7 +158,12 @@ public class AsyncKafkaConsumerTest { | |
public void resetAll() { | ||
backgroundEventQueue.clear(); | ||
if (consumer != null) { | ||
consumer.close(Duration.ZERO); | ||
try { | ||
consumer.close(Duration.ZERO); | ||
} catch (Exception e) { | ||
// best effort to clean up after each test, but may throw (ex. if callbacks where | ||
// throwing errors) | ||
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. I did this before but was asked to not do it, see https://github.com/apache/kafka/pull/15613/files/9fb917e4b1e60f238183c92d1ad3bc2565a7e1ea#r1559907295. That's why I added a "clean-up close" in the tests where close fails, with an expected exception (search for "clean-up" in this file). I'd also be fine with your (and my original approach) to have a best-effort clean up and ignore exceptions here. But then, let's remove the "clean-up close" code in the other tests. Any consistent approach is fine with me here. 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. Oh I see, good point. Agree with going with a consistent approach here, but could I bring it in a follow-up PR right after this? (it wouldn't need to get into 3.8). The trick is that now the close may throw for any of the tests that throw on the callback, so I would prefer to play safe, make sure the close does not throw here, and I'll review all the tests that we expect could rightfully throw on close now (Also the other clean-up logic has some Timeout related checks so better not removing blindly, could have some value it seems) |
||
} | ||
} | ||
consumer = null; | ||
Mockito.framework().clearInlineMocks(); | ||
|
@@ -832,8 +837,8 @@ public void testPartitionRevocationOnClose() { | |
|
||
@Test | ||
public void testFailedPartitionRevocationOnClose() { | ||
// If rebalance listener failed to execute during close, we will skip sending leave group and proceed with | ||
// closing the consumer. | ||
// If rebalance listener failed to execute during close, we still send the leave group, | ||
// and proceed with closing the consumer. | ||
ConsumerRebalanceListener listener = mock(ConsumerRebalanceListener.class); | ||
SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); | ||
consumer = newConsumer( | ||
|
@@ -848,7 +853,7 @@ public void testFailedPartitionRevocationOnClose() { | |
subscriptions.assignFromSubscribed(singleton(tp)); | ||
doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); | ||
assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); | ||
verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class)); | ||
verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class)); | ||
verify(listener).onPartitionsRevoked(eq(singleton(tp))); | ||
assertEquals(emptySet(), subscriptions.assignedPartitions()); | ||
} | ||
|
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.
So the point here seems to be, if
maybeRevokePartitions
fails, likely due to rebalancelistener, still leave the group.It looks like a good change to me. I'm just surprised to see that the legacy consumer does not seem to do this? If
onPrepareLeave
inAbstractCoordinator
fails, we won't reachmaybeLeaveGroup
.So is this a bug also in the legacy consumer?
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.
You got it right, in this case the legacy consumer won't send a leave group either. We discussed it with @dajac and it seemed right to ensure the new consumer leaves on close (even if the callbacks fail). No changes on the legacy for now though (would require more thought).