junrao commented on code in PR #17539:
URL: https://github.com/apache/kafka/pull/17539#discussion_r1832915380
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -207,13 +230,136 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
- 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;
+ private Map<TopicIdPartition, LogReadResult>
maybeReadFromLog(Map<TopicIdPartition, FetchRequest.PartitionData>
topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
partitionsMissingFetchOffsetMetadata = new LinkedHashMap<>();
+ topicPartitionData.forEach((topicIdPartition, partitionData) -> {
+ SharePartition sharePartition =
sharePartitions.get(topicIdPartition);
+ if (sharePartition.fetchOffsetMetadata().isEmpty()) {
+ partitionsMissingFetchOffsetMetadata.put(topicIdPartition,
partitionData);
}
+ });
+ if (partitionsMissingFetchOffsetMetadata.isEmpty()) {
+ return Collections.emptyMap();
+ }
+ // We fetch data from replica manager corresponding to the topic
partitions that have missing fetch offset metadata.
+ return readFromLog(partitionsMissingFetchOffsetMetadata);
+ }
+
+ private void maybeUpdateFetchOffsetMetadata(
+ Map<TopicIdPartition, LogReadResult> replicaManagerReadResponseData) {
+ for (Map.Entry<TopicIdPartition, LogReadResult> entry :
replicaManagerReadResponseData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitions.get(topicIdPartition);
+ LogReadResult replicaManagerLogReadResult = entry.getValue();
+ if (replicaManagerLogReadResult.error().code() !=
Errors.NONE.code()) {
+ log.debug("Replica manager read log result {} errored out for
topic partition {}",
+ replicaManagerLogReadResult, topicIdPartition);
+ continue;
+ }
+
sharePartition.updateFetchOffsetMetadata(Optional.of(replicaManagerLogReadResult.info().fetchOffsetMetadata));
+ }
+ }
+
+ // minByes estimation currently assumes the common case where all fetched
data is acquirable.
+ private boolean isMinBytesSatisfied(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ long accumulatedSize = 0;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ FetchRequest.PartitionData partitionData = entry.getValue();
+ LogOffsetMetadata endOffsetMetadata =
endOffsetMetadataForTopicPartition(topicIdPartition);
+
+ if (endOffsetMetadata == LogOffsetMetadata.UNKNOWN_OFFSET_METADATA)
+ continue;
+
+ SharePartition sharePartition =
sharePartitions.get(topicIdPartition);
+
+ Optional<LogOffsetMetadata> optionalFetchOffsetMetadata =
sharePartition.fetchOffsetMetadata();
+ if (optionalFetchOffsetMetadata.isEmpty() ||
optionalFetchOffsetMetadata.get() == LogOffsetMetadata.UNKNOWN_OFFSET_METADATA)
+ continue;
+ LogOffsetMetadata fetchOffsetMetadata =
optionalFetchOffsetMetadata.get();
+
+ if (fetchOffsetMetadata.messageOffset >
endOffsetMetadata.messageOffset) {
+ log.debug("Satisfying delayed share fetch request for group
{}, member {} since it is fetching later segments of " +
+ "topicIdPartition {}", shareFetchData.groupId(),
shareFetchData.memberId(), topicIdPartition);
+ return true;
+ } else if (fetchOffsetMetadata.messageOffset <
endOffsetMetadata.messageOffset) {
+ if (fetchOffsetMetadata.onOlderSegment(endOffsetMetadata)) {
+ // This can happen when the fetch operation is falling
behind the current segment or the partition
+ // has just rolled a new segment.
+ log.debug("Satisfying delayed share fetch request for
group {}, member {} immediately since it is fetching older " +
+ "segments of topicIdPartition {}",
shareFetchData.groupId(), shareFetchData.memberId(), topicIdPartition);
+ return true;
+ } else if
(fetchOffsetMetadata.onSameSegment(endOffsetMetadata)) {
+ // we take the partition fetch size as upper bound when
accumulating the bytes.
+ long bytesAvailable =
Math.min(endOffsetMetadata.positionDiff(fetchOffsetMetadata),
partitionData.maxBytes);
+ accumulatedSize += bytesAvailable;
+ }
+ }
+ }
+ return accumulatedSize >= shareFetchData.fetchParams().minBytes;
+ }
+
+ private LogOffsetMetadata
endOffsetMetadataForTopicPartition(TopicIdPartition topicIdPartition) {
+ Partition partition =
replicaManager.getPartitionOrException(topicIdPartition.topicPartition());
+ LogOffsetSnapshot offsetSnapshot =
partition.fetchOffsetSnapshot(Optional.empty(), true);
+ // The FetchIsolation type that we use for share fetch is
FetchIsolation.HIGH_WATERMARK. In the future, we can
+ // extend it to support other FetchIsolation types.
+ FetchIsolation isolationType = shareFetchData.fetchParams().isolation;
+ if (isolationType == FetchIsolation.LOG_END)
+ return offsetSnapshot.logEndOffset;
+ else if (isolationType == FetchIsolation.HIGH_WATERMARK)
+ return offsetSnapshot.highWatermark;
+ else
+ return offsetSnapshot.lastStableOffset;
+
+ }
+
+ private Map<TopicIdPartition, LogReadResult>
readFromLog(Map<TopicIdPartition, FetchRequest.PartitionData>
topicPartitionData) {
Review Comment:
Since the ordering is important, let's use LinkedHashMap.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -207,13 +230,136 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
- 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;
+ private Map<TopicIdPartition, LogReadResult>
maybeReadFromLog(Map<TopicIdPartition, FetchRequest.PartitionData>
topicPartitionData) {
Review Comment:
Since the ordering is important, let's use LinkedHashMap.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -146,18 +149,44 @@ public void onComplete() {
*/
@Override
public boolean tryComplete() {
- topicPartitionDataFromTryComplete = acquirablePartitions();
-
- if (!topicPartitionDataFromTryComplete.isEmpty()) {
- boolean completedByMe = forceComplete();
- // If invocation of forceComplete is not successful, then that
means the request is already completed
- // hence release the acquired locks.
- if (!completedByMe) {
- releasePartitionLocks(shareFetchData.groupId(),
topicPartitionDataFromTryComplete.keySet());
+ Map<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData =
acquirablePartitions();
Review Comment:
Since the ordering is important, let's use LinkedHashMap.
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1576,6 +1582,32 @@ private Optional<Throwable> acknowledgeCompleteBatch(
return Optional.empty();
}
+ // The caller of this function is expected to hold lock.writeLock() when
calling this method.
+ protected void updateEndOffsetAndResetFetchOffsetMetadata(long
updatedEndOffset) {
+ endOffset = updatedEndOffset;
+ fetchOffsetMetadata = Optional.empty();
+ }
+
+ protected void updateFetchOffsetMetadata(Optional<LogOffsetMetadata>
fetchOffsetMetadata) {
+ lock.writeLock().lock();
+ try {
+ this.fetchOffsetMetadata = fetchOffsetMetadata;
+ } finally {
+ lock.writeLock().unlock();
+ }
+ }
+
+ protected Optional<LogOffsetMetadata> fetchOffsetMetadata() {
+ lock.readLock().lock();
+ try {
+ if (findNextFetchOffset.get())
+ return Optional.empty();
Review Comment:
This means `tryComplete` will never get non-empty `fetchOffsetMetadata` and
its calculation of minBytes will be off. We need to think through how to
address this.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]