-
Notifications
You must be signed in to change notification settings - Fork 14.6k
KAFKA-3949: Fix race condition when metadata update arrives during rebalance #1762
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
Changes from all commits
2ce2e1f
b3e2666
3618458
7971c2f
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 |
---|---|---|
|
@@ -78,6 +78,8 @@ public final class ConsumerCoordinator extends AbstractCoordinator { | |
// of offset commit requests, which may be invoked from the heartbeat thread | ||
private final ConcurrentLinkedQueue<OffsetCommitCompletion> completedOffsetCommits; | ||
|
||
private boolean isLeader = false; | ||
private Set<String> joinedSubscription; | ||
private MetadataSnapshot metadataSnapshot; | ||
private MetadataSnapshot assignmentSnapshot; | ||
private long nextAutoCommitDeadline; | ||
|
@@ -137,9 +139,10 @@ public String protocolType() { | |
|
||
@Override | ||
public List<ProtocolMetadata> metadata() { | ||
this.joinedSubscription = subscriptions.subscription(); | ||
List<ProtocolMetadata> metadataList = new ArrayList<>(); | ||
for (PartitionAssignor assignor : assignors) { | ||
Subscription subscription = assignor.subscription(subscriptions.subscription()); | ||
Subscription subscription = assignor.subscription(joinedSubscription); | ||
ByteBuffer metadata = ConsumerProtocol.serializeSubscription(subscription); | ||
metadataList.add(new ProtocolMetadata(assignor.name(), metadata)); | ||
} | ||
|
@@ -155,26 +158,26 @@ public void onMetadataUpdate(Cluster cluster) { | |
throw new TopicAuthorizationException(new HashSet<>(cluster.unauthorizedTopics())); | ||
|
||
if (subscriptions.hasPatternSubscription()) { | ||
final List<String> topicsToSubscribe = new ArrayList<>(); | ||
final Set<String> topicsToSubscribe = new HashSet<>(); | ||
|
||
for (String topic : cluster.topics()) | ||
if (subscriptions.getSubscribedPattern().matcher(topic).matches() && | ||
!(excludeInternalTopics && cluster.internalTopics().contains(topic))) | ||
topicsToSubscribe.add(topic); | ||
|
||
subscriptions.changeSubscription(topicsToSubscribe); | ||
subscriptions.subscribeFromPattern(topicsToSubscribe); | ||
|
||
// note we still need to update the topics contained in the metadata. Although we have | ||
// specified that all topics should be fetched, only those set explicitly will be retained | ||
metadata.setTopics(subscriptions.groupSubscription()); | ||
} | ||
|
||
// check if there are any changes to the metadata which should trigger a rebalance | ||
if (subscriptions.partitionsAutoAssigned()) { | ||
MetadataSnapshot snapshot = new MetadataSnapshot(subscriptions, cluster); | ||
if (!snapshot.equals(metadataSnapshot)) { | ||
if (!snapshot.equals(metadataSnapshot)) | ||
metadataSnapshot = snapshot; | ||
subscriptions.needReassignment(); | ||
} | ||
} | ||
|
||
} | ||
}); | ||
} | ||
|
@@ -192,12 +195,9 @@ protected void onJoinComplete(int generation, | |
String memberId, | ||
String assignmentStrategy, | ||
ByteBuffer assignmentBuffer) { | ||
// if we were the assignor, then we need to make sure that there have been no metadata updates | ||
// since the rebalance begin. Otherwise, we won't rebalance again until the next metadata change | ||
if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) { | ||
subscriptions.needReassignment(); | ||
return; | ||
} | ||
// only the leader is responsible for monitoring for metadata changes (i.e. partition changes) | ||
if (!isLeader) | ||
assignmentSnapshot = null; | ||
|
||
PartitionAssignor assignor = lookupAssignor(assignmentStrategy); | ||
if (assignor == null) | ||
|
@@ -246,13 +246,10 @@ public void poll(long now) { | |
now = time.milliseconds(); | ||
} | ||
|
||
if (subscriptions.partitionsAutoAssigned() && needRejoin()) { | ||
// due to a race condition between the initial metadata fetch and the initial rebalance, we need to ensure that | ||
// the metadata is fresh before joining initially, and then request the metadata update. If metadata update arrives | ||
// while the rebalance is still pending (for example, when the join group is still inflight), then we will lose | ||
// track of the fact that we need to rebalance again to reflect the change to the topic subscription. Without | ||
// ensuring that the metadata is fresh, any metadata update that changes the topic subscriptions and arrives with a | ||
// rebalance in progress will essentially be ignored. See KAFKA-3949 for the complete description of the problem. | ||
if (needRejoin()) { | ||
// due to a race condition between the initial metadata fetch and the initial rebalance, | ||
// we need to ensure that the metadata is fresh before joining initially. This ensures | ||
// that we have matched the pattern against the cluster's topics at least once before joining. | ||
if (subscriptions.hasPatternSubscription()) | ||
client.ensureFreshMetadata(); | ||
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. Do we still need this now that the PR fixes the race condition? 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. Don't we still have a race condition with the initial topic metadata fetch and the regex subscription? Seems like we still need this to ensure that we see at least one topic metadata refresh prior to rebalancing. 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. Since this PR fixes the intermittent issues we saw earlier with pattern subscription unit tests, do you think additional unit tests (not necessarily as part of this PR) are required to verify the behavior for this fix on initial metadata fetch race condition? If we remove this if block all unit tests still seem to pass. 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. @vahidhashemian I've been working on a unit test that I'll post shortly. It uncovered a couple minor problems which I've had to fix. Hooray for unit tests! 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. Actually the test I've been working on is specifically for the metadata refresh with a rebalance in progress. We might still need another test case to verify the metadata race above, but I guess I'll leave that for another patch. |
||
|
||
|
@@ -303,6 +300,8 @@ protected Map<String, ByteBuffer> performAssignment(String leaderId, | |
// update metadata (if needed) and keep track of the metadata used for assignment so that | ||
// we can check after rebalance completion whether anything has changed | ||
client.ensureFreshMetadata(); | ||
|
||
isLeader = true; | ||
assignmentSnapshot = metadataSnapshot; | ||
|
||
log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", | ||
|
@@ -339,14 +338,24 @@ protected void onJoinPrepare(int generation, String memberId) { | |
listener.getClass().getName(), groupId, e); | ||
} | ||
|
||
assignmentSnapshot = null; | ||
subscriptions.needReassignment(); | ||
isLeader = false; | ||
subscriptions.resetGroupSubscription(); | ||
} | ||
|
||
@Override | ||
protected boolean needRejoin() { | ||
return subscriptions.partitionsAutoAssigned() && | ||
(super.needRejoin() || subscriptions.partitionAssignmentNeeded()); | ||
public boolean needRejoin() { | ||
if (!subscriptions.partitionsAutoAssigned()) | ||
return false; | ||
|
||
// we need to rejoin if we performed the assignment and metadata has changed | ||
if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) | ||
return true; | ||
|
||
// we need to join if our subscription has changed since the last join | ||
if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) | ||
return true; | ||
|
||
return super.needRejoin(); | ||
} | ||
|
||
/** | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -350,26 +350,22 @@ private long listOffset(TopicPartition partition, long timestamp) { | |
* the defaultResetPolicy is NONE | ||
*/ | ||
public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() { | ||
if (this.subscriptions.partitionAssignmentNeeded()) { | ||
return Collections.emptyMap(); | ||
} else { | ||
Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<>(); | ||
int recordsRemaining = maxPollRecords; | ||
Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<>(); | ||
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. Why is it safe to remove this check now? 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 had to move the check from here and into |
||
int recordsRemaining = maxPollRecords; | ||
|
||
while (recordsRemaining > 0) { | ||
if (nextInLineRecords == null || nextInLineRecords.isEmpty()) { | ||
CompletedFetch completedFetch = completedFetches.poll(); | ||
if (completedFetch == null) | ||
break; | ||
while (recordsRemaining > 0) { | ||
if (nextInLineRecords == null || nextInLineRecords.isEmpty()) { | ||
CompletedFetch completedFetch = completedFetches.poll(); | ||
if (completedFetch == null) | ||
break; | ||
|
||
nextInLineRecords = parseFetchedData(completedFetch); | ||
} else { | ||
recordsRemaining -= append(drained, nextInLineRecords, recordsRemaining); | ||
} | ||
nextInLineRecords = parseFetchedData(completedFetch); | ||
} else { | ||
recordsRemaining -= append(drained, nextInLineRecords, recordsRemaining); | ||
} | ||
|
||
return drained; | ||
} | ||
|
||
return drained; | ||
} | ||
|
||
private int append(Map<TopicPartition, List<ConsumerRecord<K, V>>> drained, | ||
|
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.
I'm a bit concerned about the overhead here since most implementation of
onJoinPrepare
may involve committing offsets etc which is latency sensitive, while most (if not all??) of such functions only need to be triggered once, since consumers will pause fetching during rebalance anyways.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.
EDIT: nvm.