apoorvmittal10 commented on code in PR #17870: URL: https://github.com/apache/kafka/pull/17870#discussion_r1912148379
########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -390,18 +407,27 @@ private void handleFetchException( } // Visible for testing. - LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData, + LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, Long> topicPartitionData, LinkedHashMap<TopicIdPartition, LogReadResult> existingFetchedData) { - LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> missingLogReadTopicPartitions = new LinkedHashMap<>(); - topicPartitionData.forEach((topicIdPartition, partitionData) -> { + LinkedHashMap<TopicIdPartition, Long> missingLogReadTopicPartitions = new LinkedHashMap<>(); + topicPartitionData.forEach((topicIdPartition, fetchOffset) -> { if (!existingFetchedData.containsKey(topicIdPartition)) { - missingLogReadTopicPartitions.put(topicIdPartition, partitionData); + missingLogReadTopicPartitions.put(topicIdPartition, fetchOffset); } }); if (missingLogReadTopicPartitions.isEmpty()) { return existingFetchedData; } - LinkedHashMap<TopicIdPartition, LogReadResult> missingTopicPartitionsLogReadResponse = readFromLog(missingLogReadTopicPartitions); + + // Computing the total bytes that has already been fetched for the existing fetched data. + int totalPartitionMaxBytesUsed = 0; + for (LogReadResult logReadResult : existingFetchedData.values()) { + totalPartitionMaxBytesUsed += logReadResult.info().records.sizeInBytes(); + } + Review Comment: Why do we need to recompute the bytes to be fetched? Shouldn't they be aligned as per the acquired topicIdPartition size i.e. by doing this we are doing dynamic re-allocation of the partition bytes to next set of replica read. ########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -390,18 +407,27 @@ private void handleFetchException( } // Visible for testing. - LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData, + LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, Long> topicPartitionData, LinkedHashMap<TopicIdPartition, LogReadResult> existingFetchedData) { Review Comment: Moreover `private LinkedHashMap<TopicIdPartition, LogReadResult> readFromLog(` has a different params indentation that others. Can we please align all same. ########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -335,13 +336,29 @@ else if (isolationType == FetchIsolation.HIGH_WATERMARK) } - private LinkedHashMap<TopicIdPartition, LogReadResult> readFromLog(LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData) { + private LinkedHashMap<TopicIdPartition, LogReadResult> readFromLog( + LinkedHashMap<TopicIdPartition, Long> topicPartitionFetchOffsets, + LinkedHashMap<TopicIdPartition, Integer> partitionMaxBytes) { // Filter if there already exists any erroneous topic partition. - Set<TopicIdPartition> partitionsToFetch = shareFetch.filterErroneousTopicPartitions(topicPartitionData.keySet()); + Set<TopicIdPartition> partitionsToFetch = shareFetch.filterErroneousTopicPartitions(topicPartitionFetchOffsets.keySet()); if (partitionsToFetch.isEmpty()) { return new LinkedHashMap<>(); } + LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData = new LinkedHashMap<>(); + + for (Map.Entry<TopicIdPartition, Long> entry : topicPartitionFetchOffsets.entrySet()) { + TopicIdPartition topicIdPartition = entry.getKey(); + long fetchOffset = entry.getValue(); + topicPartitionData.put(topicIdPartition, new FetchRequest.PartitionData( + topicIdPartition.topicId(), + fetchOffset, + 0, + partitionMaxBytes.get(topicIdPartition), + Optional.empty() + )); + } Review Comment: nit: just streamlines the code to not fetch key and value from entry: ``` topicPartitionFetchOffsets.forEach((topicIdPartition, fetchOffset) -> { topicPartitionData.put(topicIdPartition, new FetchRequest.PartitionData( topicIdPartition.topicId(), fetchOffset, 0, partitionMaxBytes.get(topicIdPartition), Optional.empty() )); }); ########## core/src/test/java/kafka/test/api/ShareConsumerTest.java: ########## @@ -902,7 +902,7 @@ public void testFetchRecordLargerThanMaxPartitionFetchBytes(String persister) th shareConsumer.subscribe(Collections.singleton(tp.topic())); ConsumerRecords<byte[], byte[]> records = shareConsumer.poll(Duration.ofMillis(5000)); - assertEquals(1, records.count()); + assertEquals(2, records.count()); Review Comment: Why did we require to change this test? ########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -335,13 +336,29 @@ else if (isolationType == FetchIsolation.HIGH_WATERMARK) } - private LinkedHashMap<TopicIdPartition, LogReadResult> readFromLog(LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData) { + private LinkedHashMap<TopicIdPartition, LogReadResult> readFromLog( + LinkedHashMap<TopicIdPartition, Long> topicPartitionFetchOffsets, + LinkedHashMap<TopicIdPartition, Integer> partitionMaxBytes) { Review Comment: nit: just better readable: ```suggestion LinkedHashMap<TopicIdPartition, Integer> partitionMaxBytes ) { ``` ########## share/src/test/java/org/apache/kafka/server/share/fetch/PartitionMaxBytesStrategyTest.java: ########## @@ -0,0 +1,87 @@ +/* + * 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 org.apache.kafka.server.share.fetch; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.server.share.fetch.PartitionMaxBytesStrategy.StrategyType; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class PartitionMaxBytesStrategyTest { + + @Test + public void testConstructor() { + assertThrows(IllegalArgumentException.class, () -> PartitionMaxBytesStrategy.type(null)); + assertDoesNotThrow(() -> PartitionMaxBytesStrategy.type(StrategyType.UNIFORM)); + } + + @Test + public void testCheckValidArguments() { + TopicIdPartition topicIdPartition1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic1", 0)); + TopicIdPartition topicIdPartition2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic1", 1)); + TopicIdPartition topicIdPartition3 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic2", 0)); + Set<TopicIdPartition> partitions = new LinkedHashSet<>(); + partitions.add(topicIdPartition1); + partitions.add(topicIdPartition2); + partitions.add(topicIdPartition3); + + // acquired partitions size is 0. + assertThrows(IllegalArgumentException.class, () -> PartitionMaxBytesStrategy.checkValidArguments( + 100, partitions, 0)); + // empty partitions set. Review Comment: Can you add another assert throws for null topicIdPartition as argument to method? ########## core/src/test/java/kafka/server/share/DelayedShareFetchTest.java: ########## @@ -559,13 +561,18 @@ public void testCombineLogReadResponse() { .withSharePartitions(sharePartitions) .build(); - LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData = new LinkedHashMap<>(); - topicPartitionData.put(tp0, mock(FetchRequest.PartitionData.class)); - topicPartitionData.put(tp1, mock(FetchRequest.PartitionData.class)); + LinkedHashMap<TopicIdPartition, Long> topicPartitionData = new LinkedHashMap<>(); + topicPartitionData.put(tp0, 0L); + topicPartitionData.put(tp1, 0L); // Case 1 - logReadResponse contains tp0. LinkedHashMap<TopicIdPartition, LogReadResult> logReadResponse = new LinkedHashMap<>(); - logReadResponse.put(tp0, mock(LogReadResult.class)); + LogReadResult logReadResult = mock(LogReadResult.class); + Records records = mock(Records.class); + when(records.sizeInBytes()).thenReturn(2); + FetchDataInfo fetchDataInfo = new FetchDataInfo(mock(LogOffsetMetadata.class), records); + when(logReadResult.info()).thenReturn(fetchDataInfo); + logReadResponse.put(tp0, logReadResult); Review Comment: Should we add a test when partitionMaxBytesStrategy.maxBytes throws an exception? Also do we have a test which validates fetch bytes are in accordance with the share fetch max bytes? ########## core/src/main/java/kafka/server/share/DelayedShareFetch.java: ########## @@ -390,18 +407,27 @@ private void handleFetchException( } // Visible for testing. - LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, FetchRequest.PartitionData> topicPartitionData, + LinkedHashMap<TopicIdPartition, LogReadResult> combineLogReadResponse(LinkedHashMap<TopicIdPartition, Long> topicPartitionData, LinkedHashMap<TopicIdPartition, LogReadResult> existingFetchedData) { Review Comment: nit: can we please fix the indentation -- 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