-
Notifications
You must be signed in to change notification settings - Fork 14.5k
KAFKA-17783: Adding listeners to remove share partition on partition changes #17796
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
373ecb8
9e0efbb
49ff3dd
568092e
df45863
5e1d78f
ad68da8
fa6b01e
1a1eace
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 |
---|---|---|
|
@@ -16,6 +16,7 @@ | |
*/ | ||
package kafka.server.share; | ||
|
||
import kafka.cluster.PartitionListener; | ||
import kafka.server.ReplicaManager; | ||
|
||
import org.apache.kafka.clients.consumer.AcknowledgeType; | ||
|
@@ -609,6 +610,12 @@ private SharePartition getOrCreateSharePartition(SharePartitionKey sharePartitio | |
k -> { | ||
long start = time.hiResClockMs(); | ||
int leaderEpoch = ShareFetchUtils.leaderEpoch(replicaManager, sharePartitionKey.topicIdPartition().topicPartition()); | ||
// Attach listener to Partition which shall invoke partition change handlers. | ||
// However, as there could be multiple share partitions (per group name) for a single topic-partition, | ||
// hence create separate listeners per share partition which holds the share partition key | ||
// to identify the respective share partition. | ||
SharePartitionListener listener = new SharePartitionListener(sharePartitionKey, replicaManager, partitionCacheMap); | ||
replicaManager.maybeAddListener(sharePartitionKey.topicIdPartition().topicPartition(), listener); | ||
SharePartition partition = new SharePartition( | ||
sharePartitionKey.groupId(), | ||
sharePartitionKey.topicIdPartition(), | ||
|
@@ -620,7 +627,8 @@ private SharePartition getOrCreateSharePartition(SharePartitionKey sharePartitio | |
time, | ||
persister, | ||
replicaManager, | ||
groupConfigManager | ||
groupConfigManager, | ||
listener | ||
); | ||
this.shareGroupMetrics.partitionLoadTime(start); | ||
return partition; | ||
|
@@ -640,10 +648,7 @@ private void handleInitializationException( | |
} | ||
|
||
// Remove the partition from the cache as it's failed to initialize. | ||
SharePartition sharePartition = partitionCacheMap.remove(sharePartitionKey); | ||
if (sharePartition != null) { | ||
sharePartition.markFenced(); | ||
} | ||
removeSharePartitionFromCache(sharePartitionKey, partitionCacheMap, replicaManager); | ||
// The partition initialization failed, so add the partition to the erroneous partitions. | ||
log.debug("Error initializing share partition with key {}", sharePartitionKey, throwable); | ||
shareFetch.addErroneous(sharePartitionKey.topicIdPartition(), throwable); | ||
|
@@ -665,17 +670,81 @@ public void handleFencedSharePartitionException( | |
// The share partition is fenced hence remove the partition from map and let the client retry. | ||
// But surface the error to the client so client might take some action i.e. re-fetch | ||
// the metadata and retry the fetch on new leader. | ||
SharePartition sharePartition = partitionCacheMap.remove(sharePartitionKey); | ||
if (sharePartition != null) { | ||
sharePartition.markFenced(); | ||
} | ||
removeSharePartitionFromCache(sharePartitionKey, partitionCacheMap, replicaManager); | ||
} | ||
} | ||
|
||
private SharePartitionKey sharePartitionKey(String groupId, TopicIdPartition topicIdPartition) { | ||
return new SharePartitionKey(groupId, topicIdPartition); | ||
} | ||
|
||
private static void removeSharePartitionFromCache( | ||
SharePartitionKey sharePartitionKey, | ||
Map<SharePartitionKey, SharePartition> map, | ||
ReplicaManager replicaManager | ||
) { | ||
SharePartition sharePartition = map.remove(sharePartitionKey); | ||
if (sharePartition != null) { | ||
sharePartition.markFenced(); | ||
replicaManager.removeListener(sharePartitionKey.topicIdPartition().topicPartition(), sharePartition.listener()); | ||
} | ||
} | ||
|
||
/** | ||
* The SharePartitionListener is used to listen for partition events. The share partition is associated with | ||
* the topic-partition, we need to handle the partition events for the share partition. | ||
* <p> | ||
* The partition cache map stores share partitions against share partition key which comprises | ||
* group and topic-partition. Instead of maintaining a separate map for topic-partition to share partitions, | ||
* we can maintain the share partition key in the listener and create a new listener for each share partition. | ||
*/ | ||
static class SharePartitionListener implements PartitionListener { | ||
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. Would it be better to make this a private class? This way, we don't need to pass in the states in SharePartitionManager. 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. That's what I was initially thinking to do but it made writing tests hard as now I have to create an instance of SharePartitionManager and invoke calls to create SharePartition as well which will provide me the listener. Hence thought to make it static itself. |
||
|
||
private final SharePartitionKey sharePartitionKey; | ||
private final ReplicaManager replicaManager; | ||
private final Map<SharePartitionKey, SharePartition> partitionCacheMap; | ||
|
||
SharePartitionListener( | ||
SharePartitionKey sharePartitionKey, | ||
ReplicaManager replicaManager, | ||
Map<SharePartitionKey, SharePartition> partitionCacheMap | ||
) { | ||
this.sharePartitionKey = sharePartitionKey; | ||
this.replicaManager = replicaManager; | ||
this.partitionCacheMap = partitionCacheMap; | ||
} | ||
|
||
@Override | ||
public void onFailed(TopicPartition topicPartition) { | ||
log.debug("The share partition failed listener is invoked for the topic-partition: {}, share-partition: {}", | ||
topicPartition, sharePartitionKey); | ||
onUpdate(topicPartition); | ||
} | ||
|
||
@Override | ||
public void onDeleted(TopicPartition topicPartition) { | ||
log.debug("The share partition delete listener is invoked for the topic-partition: {}, share-partition: {}", | ||
topicPartition, sharePartitionKey); | ||
onUpdate(topicPartition); | ||
} | ||
|
||
@Override | ||
public void onBecomingFollower(TopicPartition topicPartition) { | ||
log.debug("The share partition becoming follower listener is invoked for the topic-partition: {}, share-partition: {}", | ||
topicPartition, sharePartitionKey); | ||
onUpdate(topicPartition); | ||
} | ||
|
||
private void onUpdate(TopicPartition topicPartition) { | ||
if (!sharePartitionKey.topicIdPartition().topicPartition().equals(topicPartition)) { | ||
log.error("The share partition listener is invoked for the wrong topic-partition: {}, share-partition: {}", | ||
topicPartition, sharePartitionKey); | ||
return; | ||
} | ||
removeSharePartitionFromCache(sharePartitionKey, partitionCacheMap, replicaManager); | ||
} | ||
} | ||
|
||
static class ShareGroupMetrics { | ||
/** | ||
* share-acknowledgement (share-acknowledgement-rate and share-acknowledgement-count) - The total number of offsets acknowledged for share groups (requests to be ack). | ||
|
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.
Nit: A line break before the ) would separate the argument declarations from the statements more clearly.
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.
Done.