Skip to content

Commit

Permalink
KAFKA-17703: Moved DelayedActionsQueue outside DelayedShareFetch (apa…
Browse files Browse the repository at this point in the history
…che#17380)

Move ActionQueue outside DelayedShareFetch class to SharePartitionManager where SharePartitionManager has the ability to add a delayed action to the ActionQueue.

Add TopicPartition as a key for delayed share fetch along with SharePartition (that is already present right now). This will be utilized later when we add the code to know if new data has been added to the topic partition.

Reviewers: Apoorv Mittal <[email protected]>, Jun Rao <[email protected]>
  • Loading branch information
adixitconfluent authored and tedyu committed Jan 6, 2025
1 parent 011c3aa commit d1f3885
Show file tree
Hide file tree
Showing 11 changed files with 307 additions and 192 deletions.
41 changes: 18 additions & 23 deletions core/src/main/java/kafka/server/share/DelayedShareFetch.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,13 @@
*/
package kafka.server.share;

import kafka.server.ActionQueue;
import kafka.server.DelayedOperation;
import kafka.server.DelayedOperationPurgatory;
import kafka.server.LogReadResult;
import kafka.server.QuotaFactory;
import kafka.server.ReplicaManager;

import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.share.fetch.ShareFetchData;
import org.apache.kafka.server.storage.log.FetchPartitionData;

Expand Down Expand Up @@ -55,25 +52,19 @@ public class DelayedShareFetch extends DelayedOperation {

private final ShareFetchData shareFetchData;
private final ReplicaManager replicaManager;
private final Map<SharePartitionKey, SharePartition> partitionCacheMap;
private final ActionQueue delayedActionQueue;
private final DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory;

private Map<TopicIdPartition, FetchRequest.PartitionData> topicPartitionDataFromTryComplete;
private final SharePartitionManager sharePartitionManager;

DelayedShareFetch(
ShareFetchData shareFetchData,
ReplicaManager replicaManager,
Map<SharePartitionKey, SharePartition> partitionCacheMap,
ActionQueue delayedActionQueue,
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory) {
SharePartitionManager sharePartitionManager) {
super(shareFetchData.fetchParams().maxWaitMs, Option.empty());
this.shareFetchData = shareFetchData;
this.replicaManager = replicaManager;
this.partitionCacheMap = partitionCacheMap;
this.delayedActionQueue = delayedActionQueue;
this.delayedShareFetchPurgatory = delayedShareFetchPurgatory;
this.topicPartitionDataFromTryComplete = new LinkedHashMap<>();
this.sharePartitionManager = sharePartitionManager;
}

@Override
Expand Down Expand Up @@ -129,7 +120,7 @@ public void onComplete() {
});

log.trace("Data successfully retrieved by replica manager: {}", responseData);
ShareFetchUtils.processFetchResponse(shareFetchData, responseData, partitionCacheMap, replicaManager)
ShareFetchUtils.processFetchResponse(shareFetchData, responseData, sharePartitionManager, replicaManager)
.whenComplete((result, throwable) -> {
if (throwable != null) {
log.error("Error processing fetch response for share partitions", throwable);
Expand All @@ -143,12 +134,7 @@ public void onComplete() {
// then we should check if there is a pending share fetch request for the topic-partition and complete it.
// We add the action to delayed actions queue to avoid an infinite call stack, which could happen if
// we directly call delayedShareFetchPurgatory.checkAndComplete
delayedActionQueue.add(() -> {
result.keySet().forEach(topicIdPartition ->
delayedShareFetchPurgatory.checkAndComplete(
new DelayedShareFetchKey(shareFetchData.groupId(), topicIdPartition)));
return BoxedUnit.UNIT;
});
sharePartitionManager.addPurgatoryCheckAndCompleteDelayedActionToActionQueue(result.keySet(), shareFetchData.groupId());
});

} catch (Exception e) {
Expand Down Expand Up @@ -187,8 +173,11 @@ Map<TopicIdPartition, FetchRequest.PartitionData> acquirablePartitions() {
Map<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData = new LinkedHashMap<>();

shareFetchData.partitionMaxBytes().keySet().forEach(topicIdPartition -> {
SharePartition sharePartition = partitionCacheMap.get(new SharePartitionKey(
shareFetchData.groupId(), topicIdPartition));
SharePartition sharePartition = sharePartitionManager.sharePartition(shareFetchData.groupId(), topicIdPartition);
if (sharePartition == null) {
log.error("Encountered null share partition for groupId={}, topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
return;
}

int partitionMaxBytes = shareFetchData.partitionMaxBytes().getOrDefault(topicIdPartition, 0);
// Add the share partition to the list of partitions to be fetched only if we can
Expand Down Expand Up @@ -217,7 +206,13 @@ Map<TopicIdPartition, FetchRequest.PartitionData> acquirablePartitions() {
}

private void releasePartitionLocks(String groupId, Set<TopicIdPartition> topicIdPartitions) {
topicIdPartitions.forEach(tp -> partitionCacheMap.get(new
SharePartitionKey(groupId, tp)).releaseFetchLock());
topicIdPartitions.forEach(tp -> {
SharePartition sharePartition = sharePartitionManager.sharePartition(groupId, tp);
if (sharePartition == null) {
log.error("Encountered null share partition for groupId={}, topicIdPartition={}. Skipping it.", shareFetchData.groupId(), tp);
return;
}
sharePartition.releaseFetchLock();
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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 kafka.server.share;

import kafka.server.DelayedOperationKey;

import org.apache.kafka.common.Uuid;

import java.util.Objects;

/**
* A key for delayed share fetch purgatory that refers to the share partition.
*/
public class DelayedShareFetchGroupKey implements DelayedShareFetchKey, DelayedOperationKey {
private final String groupId;
private final Uuid topicId;
private final int partition;

DelayedShareFetchGroupKey(String groupId, Uuid topicId, int partition) {
this.groupId = groupId;
this.topicId = topicId;
this.partition = partition;
}

@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DelayedShareFetchGroupKey that = (DelayedShareFetchGroupKey) o;
return topicId.equals(that.topicId) && partition == that.partition && groupId.equals(that.groupId);
}

@Override
public int hashCode() {
return Objects.hash(topicId, partition, groupId);
}

@Override
public String toString() {
return "DelayedShareFetchGroupKey(groupId=" + groupId +
", topicId=" + topicId +
", partition=" + partition +
")";
}

@Override
public String keyLabel() {
return String.format("groupId=%s, topicId=%s, partition=%s", groupId, topicId, partition);
}
}
38 changes: 1 addition & 37 deletions core/src/main/java/kafka/server/share/DelayedShareFetchKey.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,44 +16,8 @@
*/
package kafka.server.share;

import kafka.server.DelayedOperationKey;

import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.server.share.SharePartitionKey;

import java.util.Objects;

/**
* A key for delayed operations that fetch data for share consumers.
*/
public class DelayedShareFetchKey extends SharePartitionKey implements DelayedOperationKey {

DelayedShareFetchKey(String groupId, TopicIdPartition topicIdPartition) {
super(groupId, topicIdPartition);
}

@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DelayedShareFetchKey that = (DelayedShareFetchKey) o;
return topicIdPartition.equals(that.topicIdPartition) && groupId.equals(that.groupId);
}

@Override
public int hashCode() {
return Objects.hash(topicIdPartition, groupId);
}

@Override
public String toString() {
return "DelayedShareFetchKey(groupId=" + groupId +
", topicIdPartition=" + topicIdPartition +
")";
}

@Override
public String keyLabel() {
return String.format("groupId=%s, topicIdPartition=%s", groupId, topicIdPartition);
}
public interface DelayedShareFetchKey {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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 kafka.server.share;

import kafka.server.DelayedOperationKey;

import org.apache.kafka.common.Uuid;

import java.util.Objects;

/**
* A key for delayed share fetch purgatory that refers to the topic partition.
*/
public class DelayedShareFetchPartitionKey implements DelayedShareFetchKey, DelayedOperationKey {
private final Uuid topicId;
private final int partition;

DelayedShareFetchPartitionKey(Uuid topicId, int partition) {
this.topicId = topicId;
this.partition = partition;
}

@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DelayedShareFetchPartitionKey that = (DelayedShareFetchPartitionKey) o;
return topicId.equals(that.topicId) && partition == that.partition;
}

@Override
public int hashCode() {
return Objects.hash(topicId, partition);
}

@Override
public String toString() {
return "DelayedShareFetchPartitionKey(topicId=" + topicId +
", partition=" + partition + ")";
}

@Override
public String keyLabel() {
return String.format("topicId=%s, partition=%s", topicId, partition);
}
}
10 changes: 6 additions & 4 deletions core/src/main/java/kafka/server/share/ShareFetchUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.FileRecords;
import org.apache.kafka.common.requests.ListOffsetsRequest;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.share.fetch.ShareFetchData;
import org.apache.kafka.server.storage.log.FetchPartitionData;

Expand All @@ -49,14 +48,17 @@ public class ShareFetchUtils {
static CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> processFetchResponse(
ShareFetchData shareFetchData,
Map<TopicIdPartition, FetchPartitionData> responseData,
Map<SharePartitionKey, SharePartition> partitionCacheMap,
SharePartitionManager sharePartitionManager,
ReplicaManager replicaManager
) {
Map<TopicIdPartition, CompletableFuture<ShareFetchResponseData.PartitionData>> futures = new HashMap<>();
responseData.forEach((topicIdPartition, fetchPartitionData) -> {

SharePartition sharePartition = partitionCacheMap.get(new SharePartitionKey(
shareFetchData.groupId(), topicIdPartition));
SharePartition sharePartition = sharePartitionManager.sharePartition(shareFetchData.groupId(), topicIdPartition);
if (sharePartition == null) {
log.error("Encountered null share partition for groupId={}, topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
return;
}
futures.put(topicIdPartition, sharePartition.acquire(shareFetchData.memberId(), fetchPartitionData)
.handle((acquiredRecords, throwable) -> {
log.trace("Acquired records for topicIdPartition: {} with share fetch data: {}, records: {}",
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/java/kafka/server/share/SharePartition.java
Original file line number Diff line number Diff line change
Expand Up @@ -1788,7 +1788,7 @@ && checkForStartOffsetWithinBatch(inFlightBatch.firstOffset(), inFlightBatch.las

// If we have an acquisition lock timeout for a share-partition, then we should check if
// there is a pending share fetch request for the share-partition and complete it.
DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchKey(groupId, topicIdPartition);
DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition());
delayedShareFetchPurgatory.checkAndComplete(delayedShareFetchKey);
});
}
Expand Down
Loading

0 comments on commit d1f3885

Please sign in to comment.