junrao commented on code in PR #17539:
URL: https://github.com/apache/kafka/pull/17539#discussion_r1819587266
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1525,6 +1538,27 @@ private Optional<Throwable> acknowledgeCompleteBatch(
return Optional.empty();
}
+ protected void updateLatestFetchOffsetMetadata(LogOffsetMetadata
fetchOffsetMetadata) {
Review Comment:
Could we make fetchOffsetMetadata Optional instead of relying on `null`?
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -210,7 +225,169 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for one or more topic
partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private FetchOffsetMetadataUpdateResult
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ if (sharePartition.latestFetchOffsetMetadata().isEmpty()) {
+ if (missingFetchOffsetMetadataTopicPartitions == null)
+ missingFetchOffsetMetadataTopicPartitions = new
LinkedHashMap<>();
+
missingFetchOffsetMetadataTopicPartitions.put(topicIdPartition,
entry.getValue());
+ }
+ }
+ if (missingFetchOffsetMetadataTopicPartitions == null ||
missingFetchOffsetMetadataTopicPartitions.isEmpty()) {
+ return new FetchOffsetMetadataUpdateResult(false, null);
+ }
+ // We fetch data from replica manager corresponding to the topic
partitions that have missing fetch offset metadata.
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData =
readFromLog(missingFetchOffsetMetadataTopicPartitions);
+ return
updateFetchOffsetMetadataForMissingTopicPartitions(missingFetchOffsetMetadataTopicPartitions,
replicaManagerReadResponseData);
+ }
+
+ private FetchOffsetMetadataUpdateResult
updateFetchOffsetMetadataForMissingTopicPartitions(
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions,
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData) {
+ boolean isFetchOffsetMetadataUpdated = false;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
missingFetchOffsetMetadataTopicPartitions.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ FetchPartitionOffsetData fetchPartitionOffsetData =
replicaManagerReadResponseData.getOrDefault(topicIdPartition, null);
+ if (fetchPartitionOffsetData == null) {
+ log.debug("Replica manager read log result {} does not contain
topic partition {}",
+ replicaManagerReadResponseData, topicIdPartition);
+ continue;
+ }
+
sharePartition.updateLatestFetchOffsetMetadata(fetchPartitionOffsetData.logOffsetMetadata());
+ isFetchOffsetMetadataUpdated = true;
+ }
+ return new
FetchOffsetMetadataUpdateResult(isFetchOffsetMetadataUpdated,
replicaManagerReadResponseData);
+ }
+
+ private boolean isMinBytesSatisfied(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ long accumulatedSize = 0;
+ try {
+ 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 =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ return true;
+ }
+
+ Optional<LogOffsetMetadata> optionalFetchOffsetMetadata =
sharePartition.latestFetchOffsetMetadata();
+ 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;
+ } catch (Exception e) {
+ // Ideally we should complete the share fetch request's future
exceptionally in this case from tryComplete itself.
+ // A function that can be utilized is handleFetchException in an
in-flight PR https://github.com/apache/kafka/pull/16842.
+ // Perhaps, once the mentioned PR is merged, I'll change it to
better exception handling.
+ log.error("Error processing the minBytes criteria for share fetch
request", e);
+ return true;
+ }
+ }
+
+ 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 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, FetchPartitionOffsetData>
readFromLog(Map<TopicIdPartition, FetchRequest.PartitionData>
topicPartitionData) {
+ Seq<Tuple2<TopicIdPartition, LogReadResult>> responseLogResult =
replicaManager.readFromLog(
+ shareFetchData.fetchParams(),
+ CollectionConverters.asScala(
+ topicPartitionData.entrySet().stream().map(entry ->
+ new Tuple2<>(entry.getKey(),
entry.getValue())).collect(Collectors.toList())
+ ),
+ QuotaFactory.UnboundedQuota$.MODULE$,
+ true);
+
+ Map<TopicIdPartition, FetchPartitionOffsetData> responseData = new
HashMap<>();
Review Comment:
Do we need this wrapper class `FetchPartitionOffsetData`? It seems that it's
simpler for `readFromLog` to `return Map<TopicIdPartition, LogReadResult>`. We
can then convert `LogReadResult` to `Map<TopicIdPartition,
ShareFetchResponseData.PartitionData>` in `onComplete`.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -149,16 +148,32 @@ 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();
+
+ if (!topicPartitionData.isEmpty()) {
+ FetchOffsetMetadataUpdateResult fetchOffsetMetadataUpdateResult =
maybeUpdateFetchOffsetMetadataForTopicPartitions(topicPartitionData);
+ if (isMinBytesSatisfied(topicPartitionData)) {
+ topicPartitionDataFromTryComplete = topicPartitionData;
+ if
(fetchOffsetMetadataUpdateResult.isFetchOffsetMetadataUpdated
+ &&
fetchOffsetMetadataUpdateResult.replicaManagerReadResponse != null
+ &&
!fetchOffsetMetadataUpdateResult.replicaManagerReadResponse.isEmpty())
+ logReadResponse =
fetchOffsetMetadataUpdateResult.replicaManagerReadResponse;
+ 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());
+ }
+ return completedByMe;
}
- return completedByMe;
+ log.debug("minBytes is not satisfied for the share fetch request
for group {}, member {}, " +
Review Comment:
It's probably clearer to put those in an `else` clause?
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -149,16 +148,32 @@ 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();
+
+ if (!topicPartitionData.isEmpty()) {
+ FetchOffsetMetadataUpdateResult fetchOffsetMetadataUpdateResult =
maybeUpdateFetchOffsetMetadataForTopicPartitions(topicPartitionData);
+ if (isMinBytesSatisfied(topicPartitionData)) {
+ topicPartitionDataFromTryComplete = topicPartitionData;
+ if
(fetchOffsetMetadataUpdateResult.isFetchOffsetMetadataUpdated
Review Comment:
This condition seems unnecessary. We need to set `logReadResponse` as long
as `fetchOffsetMetadataUpdateResult.replicaManagerReadResponse` is not empty,
right?
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -210,7 +225,169 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for one or more topic
partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private FetchOffsetMetadataUpdateResult
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ if (sharePartition.latestFetchOffsetMetadata().isEmpty()) {
+ if (missingFetchOffsetMetadataTopicPartitions == null)
+ missingFetchOffsetMetadataTopicPartitions = new
LinkedHashMap<>();
+
missingFetchOffsetMetadataTopicPartitions.put(topicIdPartition,
entry.getValue());
+ }
+ }
+ if (missingFetchOffsetMetadataTopicPartitions == null ||
missingFetchOffsetMetadataTopicPartitions.isEmpty()) {
+ return new FetchOffsetMetadataUpdateResult(false, null);
+ }
+ // We fetch data from replica manager corresponding to the topic
partitions that have missing fetch offset metadata.
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData =
readFromLog(missingFetchOffsetMetadataTopicPartitions);
+ return
updateFetchOffsetMetadataForMissingTopicPartitions(missingFetchOffsetMetadataTopicPartitions,
replicaManagerReadResponseData);
Review Comment:
Every passed in partition to `readFromLog` will be included in the response.
So, there is no need to pass in both
`missingFetchOffsetMetadataTopicPartitions` and
`replicaManagerReadResponseData`. We do want to check the error code for each
partition. In regular fetch, if any partition has an error code, we send a
response immediately. We can just do the same here.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -210,7 +225,169 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for one or more topic
partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private FetchOffsetMetadataUpdateResult
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ if (sharePartition.latestFetchOffsetMetadata().isEmpty()) {
+ if (missingFetchOffsetMetadataTopicPartitions == null)
+ missingFetchOffsetMetadataTopicPartitions = new
LinkedHashMap<>();
+
missingFetchOffsetMetadataTopicPartitions.put(topicIdPartition,
entry.getValue());
+ }
+ }
+ if (missingFetchOffsetMetadataTopicPartitions == null ||
missingFetchOffsetMetadataTopicPartitions.isEmpty()) {
+ return new FetchOffsetMetadataUpdateResult(false, null);
+ }
+ // We fetch data from replica manager corresponding to the topic
partitions that have missing fetch offset metadata.
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData =
readFromLog(missingFetchOffsetMetadataTopicPartitions);
+ return
updateFetchOffsetMetadataForMissingTopicPartitions(missingFetchOffsetMetadataTopicPartitions,
replicaManagerReadResponseData);
+ }
+
+ private FetchOffsetMetadataUpdateResult
updateFetchOffsetMetadataForMissingTopicPartitions(
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions,
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData) {
+ boolean isFetchOffsetMetadataUpdated = false;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
missingFetchOffsetMetadataTopicPartitions.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
Review Comment:
Could we just make a single `sharePartitionManager.sharePartition` call per
`tryComplete` to avoid having to check null repeatedly?
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -210,7 +225,169 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for one or more topic
partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private FetchOffsetMetadataUpdateResult
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ if (sharePartition.latestFetchOffsetMetadata().isEmpty()) {
+ if (missingFetchOffsetMetadataTopicPartitions == null)
+ missingFetchOffsetMetadataTopicPartitions = new
LinkedHashMap<>();
Review Comment:
We could just initialize `missingFetchOffsetMetadataTopicPartitions` with
`new LinkedHashMap<>()`. Ditto in `combineLogReadResponse`.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -103,27 +109,20 @@ public void onComplete() {
topicPartitionData, shareFetchData.groupId(),
shareFetchData.fetchParams());
try {
- Seq<Tuple2<TopicIdPartition, LogReadResult>> responseLogResult =
replicaManager.readFromLog(
- shareFetchData.fetchParams(),
- CollectionConverters.asScala(
- topicPartitionData.entrySet().stream().map(entry ->
- new Tuple2<>(entry.getKey(),
entry.getValue())).collect(Collectors.toList())
- ),
- QuotaFactory.UnboundedQuota$.MODULE$,
- true);
-
- Map<TopicIdPartition, FetchPartitionData> responseData = new
HashMap<>();
- responseLogResult.foreach(tpLogResult -> {
- TopicIdPartition topicIdPartition = tpLogResult._1();
- LogReadResult logResult = tpLogResult._2();
- FetchPartitionData fetchPartitionData =
logResult.toFetchPartitionData(false);
- responseData.put(topicIdPartition, fetchPartitionData);
- return BoxedUnit.UNIT;
- });
+ Map<TopicIdPartition, FetchPartitionOffsetData> responseData;
+ if (logReadResponse == null)
Review Comment:
Hmm, it would be better for `logReadResponse` to only be empty, but never
`null`.
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -261,6 +263,11 @@ public static RecordState forId(byte id) {
*/
private long endOffset;
+ /**
+ * We maintain the latest fetch offset metadata to estimate the minBytes
requirement more efficiently.
+ */
+ private Optional<LogOffsetMetadata> latestFetchOffsetMetadata;
Review Comment:
latestFetchOffsetMetadata => fetchOffsetMetadata?
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -783,6 +790,12 @@ &&
checkForStartOffsetWithinBatch(inFlightBatch.firstOffset(), inFlightBatch.las
} finally {
lock.writeLock().unlock();
}
+
+ if (!stateBatches.isEmpty()) {
+ // The next fetch offset will change on release of acquired
records on session close, hence we update latestFetchOffsetMetadata
+ // for the share partition.
+ updateLatestFetchOffsetMetadata(null);
Review Comment:
This is getting a bit hard to track since we need to make this call in all
places where endOffset changes. Could we have a method for updating both
`endOffset` and `latestFetchOffsetMetadata`? Then we can replace all code that
changes `endOffset` with this method.
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -204,7 +202,117 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for any topic partition in
the list of topic partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private void
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.error("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
Review Comment:
I thought we agreed that we want to send a response immediately if a
sharePartition can't be found. Is that handled?
##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -210,7 +225,169 @@ Map<TopicIdPartition, FetchRequest.PartitionData>
acquirablePartitions() {
return topicPartitionData;
}
- private void releasePartitionLocks(String groupId, Set<TopicIdPartition>
topicIdPartitions) {
+ // In case, fetch offset metadata doesn't exist for one or more topic
partitions, we do a
+ // replicaManager.readFromLog to populate the offset metadata.
+ private FetchOffsetMetadataUpdateResult
maybeUpdateFetchOffsetMetadataForTopicPartitions(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions = null;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
topicPartitionData.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ if (sharePartition.latestFetchOffsetMetadata().isEmpty()) {
+ if (missingFetchOffsetMetadataTopicPartitions == null)
+ missingFetchOffsetMetadataTopicPartitions = new
LinkedHashMap<>();
+
missingFetchOffsetMetadataTopicPartitions.put(topicIdPartition,
entry.getValue());
+ }
+ }
+ if (missingFetchOffsetMetadataTopicPartitions == null ||
missingFetchOffsetMetadataTopicPartitions.isEmpty()) {
+ return new FetchOffsetMetadataUpdateResult(false, null);
+ }
+ // We fetch data from replica manager corresponding to the topic
partitions that have missing fetch offset metadata.
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData =
readFromLog(missingFetchOffsetMetadataTopicPartitions);
+ return
updateFetchOffsetMetadataForMissingTopicPartitions(missingFetchOffsetMetadataTopicPartitions,
replicaManagerReadResponseData);
+ }
+
+ private FetchOffsetMetadataUpdateResult
updateFetchOffsetMetadataForMissingTopicPartitions(
+ Map<TopicIdPartition, FetchRequest.PartitionData>
missingFetchOffsetMetadataTopicPartitions,
+ Map<TopicIdPartition, FetchPartitionOffsetData>
replicaManagerReadResponseData) {
+ boolean isFetchOffsetMetadataUpdated = false;
+ for (Map.Entry<TopicIdPartition, FetchRequest.PartitionData> entry :
missingFetchOffsetMetadataTopicPartitions.entrySet()) {
+ TopicIdPartition topicIdPartition = entry.getKey();
+ SharePartition sharePartition =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ log.debug("Encountered null share partition for groupId={},
topicIdPartition={}. Skipping it.", shareFetchData.groupId(), topicIdPartition);
+ continue;
+ }
+ FetchPartitionOffsetData fetchPartitionOffsetData =
replicaManagerReadResponseData.getOrDefault(topicIdPartition, null);
+ if (fetchPartitionOffsetData == null) {
+ log.debug("Replica manager read log result {} does not contain
topic partition {}",
+ replicaManagerReadResponseData, topicIdPartition);
+ continue;
+ }
+
sharePartition.updateLatestFetchOffsetMetadata(fetchPartitionOffsetData.logOffsetMetadata());
+ isFetchOffsetMetadataUpdated = true;
+ }
+ return new
FetchOffsetMetadataUpdateResult(isFetchOffsetMetadataUpdated,
replicaManagerReadResponseData);
+ }
+
+ private boolean isMinBytesSatisfied(Map<TopicIdPartition,
FetchRequest.PartitionData> topicPartitionData) {
+ long accumulatedSize = 0;
+ try {
+ 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 =
sharePartitionManager.sharePartition(shareFetchData.groupId(),
topicIdPartition);
+ if (sharePartition == null) {
+ return true;
+ }
+
+ Optional<LogOffsetMetadata> optionalFetchOffsetMetadata =
sharePartition.latestFetchOffsetMetadata();
+ 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;
+ } catch (Exception e) {
+ // Ideally we should complete the share fetch request's future
exceptionally in this case from tryComplete itself.
+ // A function that can be utilized is handleFetchException in an
in-flight PR https://github.com/apache/kafka/pull/16842.
+ // Perhaps, once the mentioned PR is merged, I'll change it to
better exception handling.
+ log.error("Error processing the minBytes criteria for share fetch
request", e);
+ return true;
+ }
+ }
+
+ private LogOffsetMetadata
endOffsetMetadataForTopicPartition(TopicIdPartition topicIdPartition) {
+ Partition partition =
replicaManager.getPartitionOrException(topicIdPartition.topicPartition());
Review Comment:
Ideally, we need to handle the exception at the partition level in the
caller.
--
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]