junrao commented on code in PR #16969: URL: https://github.com/apache/kafka/pull/16969#discussion_r1734972408
########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -0,0 +1,271 @@ +/* + * 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.DelayedOperation; +import kafka.server.QuotaFactory; +import kafka.server.ReplicaManager; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.ListOffsetsRequest; +import org.apache.kafka.storage.internals.log.FetchPartitionData; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import scala.Option; +import scala.Tuple2; +import scala.jdk.javaapi.CollectionConverters; +import scala.runtime.BoxedUnit; + +/** + * A delayed share fetch operation has been introduced in case there is no share partition for which we can acquire records. We will try to wait + * for MaxWaitMs for records to be released else complete the share fetch request. + */ +public class DelayedShareFetch extends DelayedOperation { + private final SharePartitionManager.ShareFetchPartitionData shareFetchPartitionData; + private final ReplicaManager replicaManager; + private final Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap; + + private static final Logger log = LoggerFactory.getLogger(DelayedShareFetch.class); + + DelayedShareFetch( + SharePartitionManager.ShareFetchPartitionData shareFetchPartitionData, + ReplicaManager replicaManager, + Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap) { + super(shareFetchPartitionData.fetchParams().maxWaitMs, Option.empty()); + this.shareFetchPartitionData = shareFetchPartitionData; + this.replicaManager = replicaManager; + this.partitionCacheMap = partitionCacheMap; + } + + @Override + public void onExpiration() { + } + + /** + * Complete the share fetch operation by fetching records for all partitions in the share fetch request irrespective + * of whether they have any acquired records. This is called when the fetch operation is forced to complete either + * because records can be acquired for some partitions or due to MaxWaitMs timeout. + */ + @Override + public void onComplete() { + log.trace("onCompletion of delayed share fetch request for group {}, member {}, " + + "topic partitions {}", shareFetchPartitionData.groupId(), + shareFetchPartitionData.memberId(), shareFetchPartitionData.partitionMaxBytes().keySet()); + + Map<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData = topicPartitionDataForAcquirablePartitions(); + try { + if (topicPartitionData.isEmpty()) { + // No locks for share partitions could be acquired, so we complete the request with an empty response. + shareFetchPartitionData.future().complete(Collections.emptyMap()); + return; + } + log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", + topicPartitionData, shareFetchPartitionData.groupId(), shareFetchPartitionData.fetchParams()); + + replicaManager.fetchMessages( + shareFetchPartitionData.fetchParams(), + CollectionConverters.asScala( + topicPartitionData.entrySet().stream().map(entry -> + new Tuple2<>(entry.getKey(), entry.getValue())).collect(Collectors.toList()) + ), + QuotaFactory.UnboundedQuota$.MODULE$, + responsePartitionData -> { + log.trace("Data successfully retrieved by replica manager: {}", responsePartitionData); + List<Tuple2<TopicIdPartition, FetchPartitionData>> responseData = CollectionConverters.asJava( + responsePartitionData); + processFetchResponse(shareFetchPartitionData, responseData).whenComplete( + (result, throwable) -> { + if (throwable != null) { + log.error("Error processing fetch response for share partitions", throwable); + shareFetchPartitionData.future().completeExceptionally(throwable); + } else { + shareFetchPartitionData.future().complete(result); + } + // Releasing the lock to move ahead with the next request in queue. + releasePartitionsLock(shareFetchPartitionData.groupId(), topicPartitionData.keySet()); + }); + return BoxedUnit.UNIT; + }); + } catch (Exception e) { + // Release the locks acquired for the partitions in the share fetch request in case there is an exception + log.error("Error processing delayed share fetch request", e); + shareFetchPartitionData.future().completeExceptionally(e); + releasePartitionsLock(shareFetchPartitionData.groupId(), topicPartitionData.keySet()); + } + } + + /** + * Try to complete the fetch operation if we can acquire records for any partition in the share fetch request. + */ + @Override + public boolean tryComplete() { + log.trace("onTry of delayed share fetch request for group {}, member {}, topic partitions {}", + shareFetchPartitionData.groupId(), shareFetchPartitionData.memberId(), + shareFetchPartitionData.partitionMaxBytes().keySet()); + + boolean canAnyPartitionBeAcquired = false; + for (TopicIdPartition topicIdPartition: shareFetchPartitionData.partitionMaxBytes().keySet()) { + SharePartition sharePartition = partitionCacheMap.get(new SharePartitionManager.SharePartitionKey( + shareFetchPartitionData.groupId(), topicIdPartition)); + if (sharePartition.maybeAcquireFetchLock()) { + if (sharePartition.canAcquireRecords()) { + canAnyPartitionBeAcquired = true; + } + sharePartition.releaseFetchLock(); + if (canAnyPartitionBeAcquired) Review Comment: > I am not sure if it is a problem if request 2 gets the data and request 1 contains an empty response (since now we're using replicaManager.readFromLog instead of replicaManager.fetchMessages). If we return request 1 with an empty response before maxWait, it means that we are returning the response too early, right? The response is supposed to be sent when either minByte or maxWait is satisfied and minByte is always > 0. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org