adixitconfluent commented on code in PR #17539:
URL: https://github.com/apache/kafka/pull/17539#discussion_r1806716182


##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -204,6 +187,58 @@ Map<TopicIdPartition, FetchRequest.PartitionData> 
acquirablePartitions() {
         return topicPartitionData;
     }
 
+    /**
+     * Prepare partitions fetch data structure for acquirable partitions in 
the share fetch request satisfying minBytes criteria.
+     */
+    Map<TopicIdPartition, FetchPartitionData> 
replicaManagerFetchData(Map<TopicIdPartition, FetchRequest.PartitionData> 
topicPartitionData,
+                                                                      boolean 
hasRequestTimedOut) {
+        log.trace("Fetchable share partitions data: {} with groupId: {} fetch 
params: {}", topicPartitionData,
+            shareFetchData.groupId(), shareFetchData.fetchParams());
+        Map<TopicIdPartition, FetchPartitionData> 
replicaManagerFetchSatisfyingMinBytes = new HashMap<>();
+        Map<TopicIdPartition, FetchPartitionData> responseData = new 
HashMap<>();
+        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);
+
+            AtomicInteger accumulatedBytes = new AtomicInteger(0);
+
+            responseLogResult.foreach(tpLogResult -> {
+                TopicIdPartition topicIdPartition = tpLogResult._1();
+                LogReadResult logResult = tpLogResult._2();
+                FetchPartitionData fetchPartitionData = 
logResult.toFetchPartitionData(false);
+                responseData.put(topicIdPartition, fetchPartitionData);
+                
accumulatedBytes.addAndGet(logResult.info().records.sizeInBytes());
+                return BoxedUnit.UNIT;
+            });
+            log.trace("Data successfully retrieved by replica manager: {}", 
responseData);
+
+            if (accumulatedBytes.get() >= 
shareFetchData.fetchParams().minBytes)
+                replicaManagerFetchSatisfyingMinBytes = responseData;
+        } catch (Exception e) {
+            log.error("Error processing delayed share fetch request", e);
+        } finally {
+            // The case when we cannot satisfy the share fetch requests 
because the response has lesser data than minBytes
+            // and the call is coming from tryComplete, hence we want to 
release partitions lock so that the next
+            // tryComplete/onComplete call complete successfully.
+            if (replicaManagerFetchSatisfyingMinBytes.isEmpty() && 
!hasRequestTimedOut) {
+                // Releasing the lock to move ahead with the next request in 
queue.
+                releasePartitionLocks(shareFetchData.groupId(), 
topicPartitionData.keySet());
+            }

Review Comment:
   in this case, we want to call 
`ShareFetchUtils.processFetchResponse(shareFetchData, fetchResponseData, 
sharePartitionManager, replicaManager)` before we want to release the locks. 
This part is in `onComplete`, hence we don't release the lock



##########
core/src/main/java/kafka/server/share/DelayedShareFetch.java:
##########
@@ -204,6 +187,58 @@ Map<TopicIdPartition, FetchRequest.PartitionData> 
acquirablePartitions() {
         return topicPartitionData;
     }
 
+    /**
+     * Prepare partitions fetch data structure for acquirable partitions in 
the share fetch request satisfying minBytes criteria.
+     */
+    Map<TopicIdPartition, FetchPartitionData> 
replicaManagerFetchData(Map<TopicIdPartition, FetchRequest.PartitionData> 
topicPartitionData,
+                                                                      boolean 
hasRequestTimedOut) {
+        log.trace("Fetchable share partitions data: {} with groupId: {} fetch 
params: {}", topicPartitionData,
+            shareFetchData.groupId(), shareFetchData.fetchParams());
+        Map<TopicIdPartition, FetchPartitionData> 
replicaManagerFetchSatisfyingMinBytes = new HashMap<>();
+        Map<TopicIdPartition, FetchPartitionData> responseData = new 
HashMap<>();
+        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);
+
+            AtomicInteger accumulatedBytes = new AtomicInteger(0);
+
+            responseLogResult.foreach(tpLogResult -> {
+                TopicIdPartition topicIdPartition = tpLogResult._1();
+                LogReadResult logResult = tpLogResult._2();
+                FetchPartitionData fetchPartitionData = 
logResult.toFetchPartitionData(false);
+                responseData.put(topicIdPartition, fetchPartitionData);
+                
accumulatedBytes.addAndGet(logResult.info().records.sizeInBytes());
+                return BoxedUnit.UNIT;
+            });
+            log.trace("Data successfully retrieved by replica manager: {}", 
responseData);
+
+            if (accumulatedBytes.get() >= 
shareFetchData.fetchParams().minBytes)
+                replicaManagerFetchSatisfyingMinBytes = responseData;
+        } catch (Exception e) {
+            log.error("Error processing delayed share fetch request", e);
+        } finally {
+            // The case when we cannot satisfy the share fetch requests 
because the response has lesser data than minBytes
+            // and the call is coming from tryComplete, hence we want to 
release partitions lock so that the next
+            // tryComplete/onComplete call complete successfully.
+            if (replicaManagerFetchSatisfyingMinBytes.isEmpty() && 
!hasRequestTimedOut) {
+                // Releasing the lock to move ahead with the next request in 
queue.
+                releasePartitionLocks(shareFetchData.groupId(), 
topicPartitionData.keySet());
+            }

Review Comment:
   in this case, we want to call 
`ShareFetchUtils.processFetchResponse(shareFetchData, fetchResponseData, 
sharePartitionManager, replicaManager)` before we want to release the locks. 
That part is in `onComplete`, hence we don't release the lock



-- 
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]

Reply via email to