AndrewJSchofield commented on code in PR #18459: URL: https://github.com/apache/kafka/pull/18459#discussion_r1914678819
########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, + int batchSize + ) { + lock.writeLock().lock(); + try { + List<AcquiredRecords> result = new ArrayList<>(); + if (lastAcquiredOffset - firstAcquiredOffset + 1 <= batchSize) { + // No split of batches is required as the batch size is greater than records which + // can be acquired. + result.add(new AcquiredRecords() + .setFirstOffset(firstAcquiredOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } else { + // The batch is split into multiple batches considering batch size. + long currentFirstOffset = firstAcquiredOffset; + RecordBatch lastBatch = null; + // Try reading only the baseOffset of the batch and avoid reading the lastOffset as + // lastOffset call of RecordBatch is expensive (loads headers). + for (RecordBatch batch : batches) { + long batchBaseOffset = batch.baseOffset(); + if (batchBaseOffset < firstAcquiredOffset) { + continue; + } + + // Track last batch for filling the final batch, if required. + lastBatch = batch; Review Comment: nit: I'd add a blank line here for legibility. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, + int batchSize + ) { + lock.writeLock().lock(); + try { + List<AcquiredRecords> result = new ArrayList<>(); + if (lastAcquiredOffset - firstAcquiredOffset + 1 <= batchSize) { + // No split of batches is required as the batch size is greater than records which + // can be acquired. + result.add(new AcquiredRecords() + .setFirstOffset(firstAcquiredOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } else { + // The batch is split into multiple batches considering batch size. + long currentFirstOffset = firstAcquiredOffset; + RecordBatch lastBatch = null; + // Try reading only the baseOffset of the batch and avoid reading the lastOffset as + // lastOffset call of RecordBatch is expensive (loads headers). + for (RecordBatch batch : batches) { + long batchBaseOffset = batch.baseOffset(); + if (batchBaseOffset < firstAcquiredOffset) { + continue; + } + + // Track last batch for filling the final batch, if required. + lastBatch = batch; + // Check if the batch is already past the last acquired offset then break. + if (batchBaseOffset > lastAcquiredOffset) { + result.add(new AcquiredRecords() + .setFirstOffset(currentFirstOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + lastBatch = null; + break; + } + + // Create new batch once the batch size is reached. + if (batchBaseOffset - currentFirstOffset >= batchSize) { + result.add(new AcquiredRecords() + .setFirstOffset(currentFirstOffset) + .setLastOffset(batchBaseOffset - 1) + .setDeliveryCount((short) 1)); + currentFirstOffset = batchBaseOffset; + } + } + // Add the last batch if required. + if (lastBatch != null) { + result.add(new AcquiredRecords() + .setFirstOffset(currentFirstOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } + } + + result.forEach(record -> { Review Comment: The variable `record` is really an instance of `AcquiredRecords`. I suggest renaming it to `acquiredRecords` because `record.firstOffset()` and `record.lastOffset()` doesn't make sense for something is currently named as if it was a single record. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, Review Comment: nit: Almost always, `memberId` is the first argument. I suggest making it the first argument here too. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, + int batchSize + ) { + lock.writeLock().lock(); + try { + List<AcquiredRecords> result = new ArrayList<>(); + if (lastAcquiredOffset - firstAcquiredOffset + 1 <= batchSize) { + // No split of batches is required as the batch size is greater than records which + // can be acquired. + result.add(new AcquiredRecords() + .setFirstOffset(firstAcquiredOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } else { + // The batch is split into multiple batches considering batch size. + long currentFirstOffset = firstAcquiredOffset; Review Comment: I don't think that's an improvement, I'm afraid. I find the code easier to follow with the current name. ########## core/src/main/java/kafka/server/share/SharePartition.java: ########## @@ -1204,25 +1205,91 @@ private AcquiredRecords acquireNewBatchRecords( lastAcquiredOffset = lastOffsetFromBatchWithRequestOffset(batches, firstAcquiredOffset + maxFetchRecords - 1); } - // Schedule acquisition lock timeout for the batch. - AcquisitionLockTimerTask timerTask = scheduleAcquisitionLockTimeout(memberId, firstAcquiredOffset, lastAcquiredOffset); - // Add the new batch to the in-flight records along with the acquisition lock timeout task for the batch. - cachedState.put(firstAcquiredOffset, new InFlightBatch( - memberId, - firstAcquiredOffset, - lastAcquiredOffset, - RecordState.ACQUIRED, - 1, - timerTask)); + // Create batches of acquired records. + List<AcquiredRecords> acquiredRecords = createBatches(batches, firstAcquiredOffset, lastAcquiredOffset, memberId, batchSize); // if the cachedState was empty before acquiring the new batches then startOffset needs to be updated if (cachedState.firstKey() == firstAcquiredOffset) { startOffset = firstAcquiredOffset; } endOffset = lastAcquiredOffset; - return new AcquiredRecords() - .setFirstOffset(firstAcquiredOffset) - .setLastOffset(lastAcquiredOffset) - .setDeliveryCount((short) 1); + return new ShareAcquiredRecords(acquiredRecords, (int) (lastAcquiredOffset - firstAcquiredOffset + 1)); + } finally { + lock.writeLock().unlock(); + } + } + + private List<AcquiredRecords> createBatches( + Iterable<? extends RecordBatch> batches, + long firstAcquiredOffset, + long lastAcquiredOffset, + String memberId, + int batchSize + ) { + lock.writeLock().lock(); + try { + List<AcquiredRecords> result = new ArrayList<>(); + if (lastAcquiredOffset - firstAcquiredOffset + 1 <= batchSize) { + // No split of batches is required as the batch size is greater than records which + // can be acquired. + result.add(new AcquiredRecords() + .setFirstOffset(firstAcquiredOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); + } else { + // The batch is split into multiple batches considering batch size. + long currentFirstOffset = firstAcquiredOffset; + RecordBatch lastBatch = null; + // Try reading only the baseOffset of the batch and avoid reading the lastOffset as + // lastOffset call of RecordBatch is expensive (loads headers). + for (RecordBatch batch : batches) { + long batchBaseOffset = batch.baseOffset(); + if (batchBaseOffset < firstAcquiredOffset) { + continue; + } + + // Track last batch for filling the final batch, if required. + lastBatch = batch; + // Check if the batch is already past the last acquired offset then break. + if (batchBaseOffset > lastAcquiredOffset) { + result.add(new AcquiredRecords() + .setFirstOffset(currentFirstOffset) + .setLastOffset(lastAcquiredOffset) + .setDeliveryCount((short) 1)); Review Comment: Yes. Agreed. -- 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