Skip to content
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-17783: Adding listeners to remove share partition on partition changes #17796

Open
wants to merge 4 commits into
base: trunk
Choose a base branch
from
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
86 changes: 78 additions & 8 deletions core/src/main/java/kafka/server/share/SharePartitionManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package kafka.server.share;

import kafka.cluster.PartitionListener;
import kafka.server.ReplicaManager;

import org.apache.kafka.clients.consumer.AcknowledgeType;
Expand Down Expand Up @@ -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.
replicaManager.maybeAddListener(sharePartitionKey.topicIdPartition().topicPartition(),
new SharePartitionListener(sharePartitionKey));
SharePartition partition = new SharePartition(
sharePartitionKey.groupId(),
sharePartitionKey.topicIdPartition(),
Expand Down Expand Up @@ -640,10 +647,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);
// 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);
Expand All @@ -665,17 +669,83 @@ 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);
}
}

private SharePartitionKey sharePartitionKey(String groupId, TopicIdPartition topicIdPartition) {
return new SharePartitionKey(groupId, topicIdPartition);
}

private void removeSharePartitionFromCache(SharePartitionKey sharePartitionKey) {
SharePartition sharePartition = partitionCacheMap.remove(sharePartitionKey);
if (sharePartition != null) {
sharePartition.markFenced();
}
}

/**
* 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.
*/
private class SharePartitionListener implements PartitionListener {

private final SharePartitionKey sharePartitionKey;

private SharePartitionListener(SharePartitionKey sharePartitionKey) {
this.sharePartitionKey = sharePartitionKey;
}

/**
* The onFailed method is called when a Partition is marked offline.
*
* @param topicPartition The topic-partition that has been marked offline.
*/
@Override
public void onFailed(TopicPartition topicPartition) {
log.info("The share partition failed listener is invoked for the topic-partition: {}, share-partition: {}",
topicPartition, sharePartitionKey);
onUpdate(topicPartition);
}

/**
* The onDeleted method is called when a Partition is deleted.
*
* @param topicPartition The topic-partition that has been deleted.
*/
@Override
public void onDeleted(TopicPartition topicPartition) {
log.info("The share partition delete listener is invoked for the topic-partition: {}, share-partition: {}",
topicPartition, sharePartitionKey);
onUpdate(topicPartition);
}

/**
* The onFollower method is called when a Partition is marked follower.
*
* @param topicPartition The topic-partition that has been marked as follower.
*/
@Override
public void onFollower(TopicPartition topicPartition) {
log.info("The share partition leader change listener is invoked for the topic-partition: {}, share-partition: {}",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

INFO is probably too noisy for this as we expect it any time the topic leadership changes.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, can make it debug as well.

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);
}
}

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).
Expand Down
16 changes: 16 additions & 0 deletions core/src/main/scala/kafka/cluster/Partition.scala
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,11 @@ trait PartitionListener {
* that the partition was deleted but only that this broker does not host a replica of it any more.
*/
def onDeleted(partition: TopicPartition): Unit = {}

/**
* Called when the Partition on this broker is marked as follower.
*/
def onFollower(partition: TopicPartition): Unit = {}
}

trait AlterPartitionListener {
Expand Down Expand Up @@ -701,6 +706,15 @@ class Partition(val topicPartition: TopicPartition,
}
}

/**
* Invoke the partition listeners when the partition has been marked as follower.
*/
def invokeFollowerListeners(): Unit = {
listeners.forEach { listener =>
listener.onFollower(topicPartition)
}
}

private def clear(): Unit = {
remoteReplicasMap.clear()
assignmentState = SimpleAssignmentState(Seq.empty)
Expand Down Expand Up @@ -891,6 +905,8 @@ class Partition(val topicPartition: TopicPartition,
s"and partition state $partitionState since it is already a follower with leader epoch $leaderEpoch.")
}

// Invoke the follower transition listeners for the partition.
invokeFollowerListeners()
// We must restart the fetchers when the leader epoch changed regardless of
// whether the leader changed as well.
isNewLeaderEpoch
Expand Down