apoorvmittal10 commented on code in PR #20246:
URL: https://github.com/apache/kafka/pull/20246#discussion_r2364439499


##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -668,6 +668,7 @@ public long nextFetchOffset() {
     @SuppressWarnings({"cyclomaticcomplexity", "methodlength"}) // Consider 
refactoring to avoid suppression
     public ShareAcquiredRecords acquire(
         String memberId,
+        String acquireMode,

Review Comment:
   As we are not certain about client side changes for now because that might 
have higher egress for clients given we have to anyways send the complete batch 
but with partial acquired records hence we should not have the `acquireMode` in 
the `acquire` API for now. Rather we can decide dynamically regarding how the 
behaviour should be for any fetch.



##########
core/src/main/java/kafka/server/share/ShareAcquireMode.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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 java.util.Locale;
+
+public enum ShareAcquireMode {
+    STRICT("STRICT"), 
+    LOOSE("LOOSE");

Review Comment:
   May be the better word would be `BATCH_OPTIMIZED`, @AndrewJSchofield wdyt?



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -746,7 +747,7 @@ public ShareAcquiredRecords acquire(
                     groupId, topicIdPartition);
                 // Do not send the lastOffsetToAcquire as when the subMap is 
empty, it means that
                 // there isn't any overlap itself.
-                ShareAcquiredRecords shareAcquiredRecords = 
acquireNewBatchRecords(memberId, fetchPartitionData.records.batches(),
+                ShareAcquiredRecords shareAcquiredRecords = 
acquireNewBatchRecords(memberId, acquireMode, 
fetchPartitionData.records.batches(),

Review Comment:
   We should default to `BATCH_OPTIMIZED/LOOSE` acquire mode in this method and 
can solve dynamic toggle in the ticket: 
https://issues.apache.org/jira/browse/KAFKA-17541 



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2311,6 +2316,24 @@ long findLastOffsetAcknowledged() {
         return lastOffsetAcknowledged;
     }
 
+    /**
+     * Find the maximum offset from the first batch within the request offset 
for {@link ShareAcquireMode.STRICT} mode.
+     *
+     * @param batches The batches to search for the request offset.
+     * @param offset The request offset to find.
+     * @return The minimum value between the first batch's last offset and the 
request offset.
+     *         If no batches are available, returns the original request 
offset.
+     */
+    private long maxOffsetFromFirstBatch(
+        Iterable<? extends RecordBatch> batches,
+        long offset
+    ) {
+        for (RecordBatch batch : batches) {
+            return Math.min(batch.lastOffset(), offset);
+        }
+        return offset;
+    }

Review Comment:
   Can you please help clarify regarding what we want to achieve here? Seems 
there is a `for` loop but the comparison will happen only for first batch, then 
why do we need a `for` loop? Also say the batches are [0-2],[3-5] i.e. 2 
batches with 3 records each. And in strict mode the maxFetchRecords are 4 then 
what batches will be acquired? Do we have a test case for this?



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1553,12 +1555,15 @@ private ShareAcquiredRecords acquireNewBatchRecords(
 
             // Check how many records can be acquired from the batch.
             long lastAcquiredOffset = lastOffset;
+            long maxOffset = firstAcquiredOffset + maxFetchRecords - 1;
             if (maxFetchRecords < lastAcquiredOffset - firstAcquiredOffset + 
1) {
-                // The max records to acquire is less than the complete 
available batches hence
-                // limit the acquired records. The last offset shall be the 
batches last offset
-                // which falls under the max records limit. As the max fetch 
records is the soft
-                // limit, the last offset can be higher than the max records.

Review Comment:
   Rather removing the comment, enhance it with the changes.



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -1553,12 +1555,15 @@ private ShareAcquiredRecords acquireNewBatchRecords(
 
             // Check how many records can be acquired from the batch.
             long lastAcquiredOffset = lastOffset;
+            long maxOffset = firstAcquiredOffset + maxFetchRecords - 1;
             if (maxFetchRecords < lastAcquiredOffset - firstAcquiredOffset + 
1) {
-                // The max records to acquire is less than the complete 
available batches hence
-                // limit the acquired records. The last offset shall be the 
batches last offset
-                // which falls under the max records limit. As the max fetch 
records is the soft
-                // limit, the last offset can be higher than the max records.
-                lastAcquiredOffset = 
lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + 
maxFetchRecords - 1);
+                ShareAcquireMode mode = acquireMode(acquireMode);
+                lastAcquiredOffset = switch (mode) {
+                    case STRICT ->
+                        maxOffsetFromFirstBatch(batches, maxOffset);

Review Comment:
   This seems to be incorrect as we ll now create partial batches in 
SharePartition, not aligned on batch boundaries. We should still create batches 
on batch boundaries but should acquire only subset.



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

Reply via email to