diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java index b7fefbcaadb..e88318217ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java @@ -179,7 +179,7 @@ public class ShareSessionHandler { return ShareFetchRequest.Builder.forConsumer( groupId, nextMetadata, fetchConfig.maxWaitMs, - fetchConfig.minBytes, fetchConfig.maxBytes, fetchConfig.fetchSize, fetchConfig.maxPollRecords, + fetchConfig.minBytes, fetchConfig.maxBytes, fetchConfig.maxPollRecords, added, removed, acknowledgementBatches); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java index f1a5753fef1..ff395ea7eb3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java @@ -28,10 +28,8 @@ import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; public class ShareFetchRequest extends AbstractRequest { @@ -49,19 +47,17 @@ public class ShareFetchRequest extends AbstractRequest { } public static Builder forConsumer(String groupId, ShareRequestMetadata metadata, - int maxWait, int minBytes, int maxBytes, int fetchSize, int batchSize, + int maxWait, int minBytes, int maxBytes, int batchSize, List send, List forget, Map> acknowledgementsMap) { ShareFetchRequestData data = new ShareFetchRequestData(); data.setGroupId(groupId); - int ackOnlyPartitionMaxBytes = fetchSize; boolean isClosingShareSession = false; if (metadata != null) { data.setMemberId(metadata.memberId().toString()); data.setShareSessionEpoch(metadata.epoch()); if (metadata.isFinalEpoch()) { isClosingShareSession = true; - ackOnlyPartitionMaxBytes = 0; } } data.setMaxWaitMs(maxWait); @@ -77,8 +73,7 @@ public class ShareFetchRequest extends AbstractRequest { for (TopicIdPartition tip : send) { Map partMap = fetchMap.computeIfAbsent(tip.topicId(), k -> new HashMap<>()); ShareFetchRequestData.FetchPartition fetchPartition = new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(tip.partition()) - .setPartitionMaxBytes(fetchSize); + .setPartitionIndex(tip.partition()); partMap.put(tip.partition(), fetchPartition); } } @@ -91,8 +86,7 @@ public class ShareFetchRequest extends AbstractRequest { ShareFetchRequestData.FetchPartition fetchPartition = partMap.get(tip.partition()); if (fetchPartition == null) { fetchPartition = new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(tip.partition()) - .setPartitionMaxBytes(ackOnlyPartitionMaxBytes); + .setPartitionIndex(tip.partition()); partMap.put(tip.partition(), fetchPartition); } fetchPartition.setAcknowledgementBatches(acknowledgeEntry.getValue()); @@ -151,7 +145,7 @@ public class ShareFetchRequest extends AbstractRequest { } private final ShareFetchRequestData data; - private volatile LinkedHashMap shareFetchData = null; + private volatile List shareFetchData = null; private volatile List toForget = null; public ShareFetchRequest(ShareFetchRequestData data, short version) { @@ -179,41 +173,6 @@ public class ShareFetchRequest extends AbstractRequest { ); } - public static final class SharePartitionData { - public final Uuid topicId; - public final int maxBytes; - - public SharePartitionData( - Uuid topicId, - int maxBytes - ) { - this.topicId = topicId; - this.maxBytes = maxBytes; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ShareFetchRequest.SharePartitionData that = (ShareFetchRequest.SharePartitionData) o; - return Objects.equals(topicId, that.topicId) && - maxBytes == that.maxBytes; - } - - @Override - public int hashCode() { - return Objects.hash(topicId, maxBytes); - } - - @Override - public String toString() { - return "SharePartitionData(" + - "topicId=" + topicId + - ", maxBytes=" + maxBytes + - ')'; - } - } - public int minBytes() { return data.minBytes(); } @@ -226,23 +185,18 @@ public class ShareFetchRequest extends AbstractRequest { return data.maxWaitMs(); } - public Map shareFetchData(Map topicNames) { + public List shareFetchData(Map topicNames) { if (shareFetchData == null) { synchronized (this) { if (shareFetchData == null) { // Assigning the lazy-initialized `shareFetchData` in the last step // to avoid other threads accessing a half-initialized object. - final LinkedHashMap shareFetchDataTmp = new LinkedHashMap<>(); + final List shareFetchDataTmp = new ArrayList<>(); data.topics().forEach(shareFetchTopic -> { String name = topicNames.get(shareFetchTopic.topicId()); shareFetchTopic.partitions().forEach(shareFetchPartition -> { // Topic name may be null here if the topic name was unable to be resolved using the topicNames map. - shareFetchDataTmp.put(new TopicIdPartition(shareFetchTopic.topicId(), new TopicPartition(name, shareFetchPartition.partitionIndex())), - new ShareFetchRequest.SharePartitionData( - shareFetchTopic.topicId(), - shareFetchPartition.partitionMaxBytes() - ) - ); + shareFetchDataTmp.add(new TopicIdPartition(shareFetchTopic.topicId(), shareFetchPartition.partitionIndex(), name)); }); }); shareFetchData = shareFetchDataTmp; diff --git a/clients/src/main/resources/common/message/ShareFetchRequest.json b/clients/src/main/resources/common/message/ShareFetchRequest.json index b0b91b82228..e85fc095861 100644 --- a/clients/src/main/resources/common/message/ShareFetchRequest.json +++ b/clients/src/main/resources/common/message/ShareFetchRequest.json @@ -46,8 +46,6 @@ "about": "The partitions to fetch.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, - { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", - "about": "TO BE REMOVED. The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." }, { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", "about": "Record batches to acknowledge.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index 18797f3cc0c..d68ed06d307 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -181,7 +181,7 @@ public class DelayedShareFetch extends DelayedOperation { return; } else { // Update metric to record acquired to requested partitions. - double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.partitionMaxBytes().size(); + double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.topicIdPartitions().size(); shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (requestTopicToAcquired * 100)); } log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index f080b08e8c8..4a8c373fa69 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.message.ShareAcknowledgeResponseData; import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.ShareFetchRequest; import org.apache.kafka.common.requests.ShareRequestMetadata; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.common.utils.Time; @@ -248,7 +247,7 @@ public class SharePartitionManager implements AutoCloseable { * @param memberId The member id, generated by the group-coordinator, this is used to identify the client. * @param fetchParams The fetch parameters from the share fetch request. * @param batchSize The number of records per acquired records batch. - * @param partitionMaxBytes The maximum number of bytes to fetch for each partition. + * @param topicIdPartitions The topic partitions to fetch for. * * @return A future that will be completed with the fetched messages. */ @@ -258,17 +257,17 @@ public class SharePartitionManager implements AutoCloseable { FetchParams fetchParams, int sessionEpoch, int batchSize, - LinkedHashMap partitionMaxBytes + List topicIdPartitions ) { log.trace("Fetch request for topicIdPartitions: {} with groupId: {} fetch params: {}", - partitionMaxBytes.keySet(), groupId, fetchParams); + topicIdPartitions, groupId, fetchParams); - LinkedHashMap topicIdPartitions = PartitionRotateStrategy + List rotatedTopicIdPartitions = PartitionRotateStrategy .type(PartitionRotateStrategy.StrategyType.ROUND_ROBIN) - .rotate(partitionMaxBytes, new PartitionRotateMetadata(sessionEpoch)); + .rotate(topicIdPartitions, new PartitionRotateMetadata(sessionEpoch)); CompletableFuture> future = new CompletableFuture<>(); - processShareFetch(new ShareFetch(fetchParams, groupId, memberId, future, topicIdPartitions, batchSize, maxFetchRecords, brokerTopicStats)); + processShareFetch(new ShareFetch(fetchParams, groupId, memberId, future, rotatedTopicIdPartitions, batchSize, maxFetchRecords, brokerTopicStats)); return future; } @@ -427,29 +426,20 @@ public class SharePartitionManager implements AutoCloseable { /** * The newContext method is used to create a new share fetch context for every share fetch request. * @param groupId The group id in the share fetch request. - * @param shareFetchData The topic-partitions and their corresponding maxBytes data in the share fetch request. + * @param shareFetchData The topic-partitions in the share fetch request. * @param toForget The topic-partitions to forget present in the share fetch request. * @param reqMetadata The metadata in the share fetch request. * @param isAcknowledgeDataPresent This tells whether the fetch request received includes piggybacked acknowledgements or not * @return The new share fetch context object */ - public ShareFetchContext newContext(String groupId, Map shareFetchData, + public ShareFetchContext newContext(String groupId, List shareFetchData, List toForget, ShareRequestMetadata reqMetadata, Boolean isAcknowledgeDataPresent) { ShareFetchContext context; - // TopicPartition with maxBytes as 0 should not be added in the cachedPartitions - Map shareFetchDataWithMaxBytes = new HashMap<>(); - shareFetchData.forEach((tp, sharePartitionData) -> { - if (sharePartitionData.maxBytes > 0) shareFetchDataWithMaxBytes.put(tp, sharePartitionData); - }); // If the request's epoch is FINAL_EPOCH or INITIAL_EPOCH, we should remove the existing sessions. Also, start a // new session in case it is INITIAL_EPOCH. Hence, we need to treat them as special cases. if (reqMetadata.isFull()) { ShareSessionKey key = shareSessionKey(groupId, reqMetadata.memberId()); if (reqMetadata.epoch() == ShareRequestMetadata.FINAL_EPOCH) { - // If the epoch is FINAL_EPOCH, don't try to create a new session. - if (!shareFetchDataWithMaxBytes.isEmpty()) { - throw Errors.INVALID_REQUEST.exception(); - } if (cache.get(key) == null) { log.error("Share session error for {}: no such share session found", key); throw Errors.SHARE_SESSION_NOT_FOUND.exception(); @@ -464,9 +454,9 @@ public class SharePartitionManager implements AutoCloseable { log.debug("Removed share session with key {}", key); } ImplicitLinkedHashCollection cachedSharePartitions = new - ImplicitLinkedHashCollection<>(shareFetchDataWithMaxBytes.size()); - shareFetchDataWithMaxBytes.forEach((topicIdPartition, reqData) -> - cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, reqData, false))); + ImplicitLinkedHashCollection<>(shareFetchData.size()); + shareFetchData.forEach(topicIdPartition -> + cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, false))); ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), time.milliseconds(), cachedSharePartitions); if (responseShareSessionKey == null) { @@ -474,10 +464,10 @@ public class SharePartitionManager implements AutoCloseable { throw Errors.SHARE_SESSION_NOT_FOUND.exception(); } - context = new ShareSessionContext(reqMetadata, shareFetchDataWithMaxBytes); + context = new ShareSessionContext(reqMetadata, shareFetchData); log.debug("Created a new ShareSessionContext with key {} isSubsequent {} returning {}. A new share " + "session will be started.", responseShareSessionKey, false, - partitionsToLogString(shareFetchDataWithMaxBytes.keySet())); + partitionsToLogString(shareFetchData)); } } else { // We update the already existing share session. @@ -494,7 +484,7 @@ public class SharePartitionManager implements AutoCloseable { throw Errors.INVALID_SHARE_SESSION_EPOCH.exception(); } Map> modifiedTopicIdPartitions = shareSession.update( - shareFetchDataWithMaxBytes, toForget); + shareFetchData, toForget); cache.touch(shareSession, time.milliseconds()); shareSession.epoch = ShareRequestMetadata.nextEpoch(shareSession.epoch); log.debug("Created a new ShareSessionContext for session key {}, epoch {}: " + @@ -586,7 +576,7 @@ public class SharePartitionManager implements AutoCloseable { // Visible for testing. void processShareFetch(ShareFetch shareFetch) { - if (shareFetch.partitionMaxBytes().isEmpty()) { + if (shareFetch.topicIdPartitions().isEmpty()) { // If there are no partitions to fetch then complete the future with an empty map. shareFetch.maybeComplete(Collections.emptyMap()); return; @@ -596,7 +586,7 @@ public class SharePartitionManager implements AutoCloseable { LinkedHashMap sharePartitions = new LinkedHashMap<>(); // Track the topics for which we have received a share fetch request for metrics. Set topics = new HashSet<>(); - for (TopicIdPartition topicIdPartition : shareFetch.partitionMaxBytes().keySet()) { + for (TopicIdPartition topicIdPartition : shareFetch.topicIdPartitions()) { topics.add(topicIdPartition.topic()); SharePartitionKey sharePartitionKey = sharePartitionKey( shareFetch.groupId(), diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 02c1cb5b73b..e379d1031c2 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2996,12 +2996,8 @@ class KafkaApis(val requestChannel: RequestChannel, erroneousAndValidPartitionData.erroneous.forEach { case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) topicIdPartitionSeq += tp } - erroneousAndValidPartitionData.validTopicIdPartitions.forEach { - case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) topicIdPartitionSeq += tp - } - shareFetchData.forEach { - case(tp, _) => if (!topicIdPartitionSeq.contains(tp)) topicIdPartitionSeq += tp - } + erroneousAndValidPartitionData.validTopicIdPartitions.forEach(tp => if (!topicIdPartitionSeq.contains(tp)) topicIdPartitionSeq += tp) + shareFetchData.forEach { tp => if (!topicIdPartitionSeq.contains(tp)) topicIdPartitionSeq += tp} // Kafka share consumers need READ permission on each topic they are fetching. val authorizedTopics = authHelper.filterByAuthorized( @@ -3138,15 +3134,15 @@ class KafkaApis(val requestChannel: RequestChannel, val erroneous = mutable.Map.empty[TopicIdPartition, ShareFetchResponseData.PartitionData] erroneousAndValidPartitionData.erroneous.forEach { (topicIdPartition, partitionData) => erroneous.put(topicIdPartition, partitionData) } - val interestedWithMaxBytes = new util.LinkedHashMap[TopicIdPartition, Integer] + val interestedTopicPartitions = new util.ArrayList[TopicIdPartition] - erroneousAndValidPartitionData.validTopicIdPartitions.forEach { case (topicIdPartition, sharePartitionData) => + erroneousAndValidPartitionData.validTopicIdPartitions.forEach { case topicIdPartition => if (!authorizedTopics.contains(topicIdPartition.topicPartition.topic)) erroneous += topicIdPartition -> ShareFetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED) else if (!metadataCache.contains(topicIdPartition.topicPartition)) erroneous += topicIdPartition -> ShareFetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) else - interestedWithMaxBytes.put(topicIdPartition, sharePartitionData.maxBytes) + interestedTopicPartitions.add(topicIdPartition) } val shareFetchRequest = request.body[ShareFetchRequest] @@ -3154,7 +3150,7 @@ class KafkaApis(val requestChannel: RequestChannel, val clientId = request.header.clientId val groupId = shareFetchRequest.data.groupId - if (interestedWithMaxBytes.isEmpty) { + if (interestedTopicPartitions.isEmpty) { CompletableFuture.completedFuture(erroneous) } else { // for share fetch from consumer, cap fetchMaxBytes to the maximum bytes that could be fetched without being @@ -3191,7 +3187,7 @@ class KafkaApis(val requestChannel: RequestChannel, params, shareSessionEpoch, shareFetchRequest.data.batchSize, - interestedWithMaxBytes + interestedTopicPartitions ).thenApply{ result => val combinedResult = mutable.Map.empty[TopicIdPartition, ShareFetchResponseData.PartitionData] result.asScala.foreach { case (tp, data) => diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index aa0e855d931..6b3165bae15 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -69,11 +69,9 @@ import scala.Tuple2; import scala.jdk.javaapi.CollectionConverters; import static kafka.server.share.SharePartitionManagerTest.DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL; -import static kafka.server.share.SharePartitionManagerTest.PARTITION_MAX_BYTES; import static kafka.server.share.SharePartitionManagerTest.buildLogReadResult; import static kafka.server.share.SharePartitionManagerTest.mockReplicaManagerDelayedShareFetch; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.orderedMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -120,7 +118,6 @@ public class DelayedShareFetchTest { Uuid topicId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -133,7 +130,7 @@ public class DelayedShareFetchTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(false); @@ -166,7 +163,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -181,7 +177,7 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 2, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(true); @@ -198,7 +194,7 @@ public class DelayedShareFetchTest { LogOffsetMetadata hwmOffsetMetadata = new LogOffsetMetadata(1, 1, 1); mockTopicIdPartitionFetchBytes(replicaManager, tp0, hwmOffsetMetadata); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); BiConsumer exceptionHandler = mockExceptionHandler(); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp0)); @@ -239,7 +235,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -254,7 +249,7 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 2, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(true); @@ -294,7 +289,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -307,14 +301,14 @@ public class DelayedShareFetchTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); when(sp0.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(FetchPartitionData.class))).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp0, 1); @@ -354,7 +348,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -368,7 +361,7 @@ public class DelayedShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - future, partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(false); when(sp1.canAcquireRecords()).thenReturn(false); @@ -409,7 +402,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -422,14 +414,14 @@ public class DelayedShareFetchTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); when(sp0.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(FetchPartitionData.class))).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp0)); @@ -470,7 +462,6 @@ public class DelayedShareFetchTest { Uuid topicId = Uuid.randomUuid(); ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); SharePartition sp0 = mock(SharePartition.class); @@ -479,7 +470,7 @@ public class DelayedShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - future, partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); + future, List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(false); @@ -525,7 +516,7 @@ public class DelayedShareFetchTest { TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(topicId, new TopicPartition("foo", 1)); TopicIdPartition tp2 = new TopicIdPartition(topicId, new TopicPartition("foo", 2)); - LinkedHashMap partitionMaxBytes1 = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); + List topicIdPartitions1 = List.of(tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -541,7 +532,7 @@ public class DelayedShareFetchTest { sharePartitions1.put(tp2, sp2); ShareFetch shareFetch1 = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes1, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), topicIdPartitions1, BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); DelayedOperationPurgatory delayedShareFetchPurgatory = new DelayedOperationPurgatory<>( @@ -550,7 +541,7 @@ public class DelayedShareFetchTest { mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory); List delayedShareFetchWatchKeys = new ArrayList<>(); - partitionMaxBytes1.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + topicIdPartitions1.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); DelayedShareFetch delayedShareFetch1 = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch1) @@ -567,12 +558,11 @@ public class DelayedShareFetchTest { assertTrue(delayedShareFetch1.lock().tryLock()); delayedShareFetch1.lock().unlock(); - LinkedHashMap partitionMaxBytes2 = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); ShareFetch shareFetch2 = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes2, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp1)); @@ -615,7 +605,6 @@ public class DelayedShareFetchTest { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -628,7 +617,7 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - future, partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp1)); @@ -652,7 +641,7 @@ public class DelayedShareFetchTest { when(logReadResult.info()).thenReturn(fetchDataInfo); logReadResponse.put(tp0, logReadResult); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); LinkedHashMap combinedLogReadResponse = delayedShareFetch.combineLogReadResponse(topicPartitionData, logReadResponse); assertEquals(topicPartitionData.keySet(), combinedLogReadResponse.keySet()); assertEquals(combinedLogReadResponse.get(tp0), logReadResponse.get(tp0)); @@ -673,7 +662,6 @@ public class DelayedShareFetchTest { Uuid topicId = Uuid.randomUuid(); ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(topicId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); SharePartition sp0 = mock(SharePartition.class); @@ -685,13 +673,13 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.canAcquireRecords()).thenReturn(true); when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any())).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); // Mocking partition object to throw an exception during min bytes calculation while calling fetchOffsetSnapshot Partition partition = mock(Partition.class); @@ -752,7 +740,6 @@ public class DelayedShareFetchTest { public void testTryCompleteLocksReleasedOnCompleteException() { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); SharePartition sp0 = mock(SharePartition.class); when(sp0.maybeAcquireFetchLock()).thenReturn(true); @@ -763,10 +750,10 @@ public class DelayedShareFetchTest { sharePartitions.put(tp0, sp0); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp0, 1); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp0)); @@ -799,11 +786,11 @@ public class DelayedShareFetchTest { sharePartitions1.put(tp0, sp0); ReplicaManager replicaManager = mock(ReplicaManager.class); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp0, 1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), orderedMap(PARTITION_MAX_BYTES, tp0), BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Collections.singleton(tp0)); @@ -837,7 +824,7 @@ public class DelayedShareFetchTest { sharePartitions.put(tp0, sp0); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), orderedMap(PARTITION_MAX_BYTES, tp0), BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() @@ -856,7 +843,6 @@ public class DelayedShareFetchTest { String groupId = "grp"; TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); SharePartition sp0 = mock(SharePartition.class); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); @@ -867,7 +853,7 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 2, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - future, partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); + future, List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); // partitionMaxBytesStrategy.maxBytes() function throws an exception PartitionMaxBytesStrategy partitionMaxBytesStrategy = mock(PartitionMaxBytesStrategy.class); @@ -912,8 +898,6 @@ public class DelayedShareFetchTest { SharePartition sp3 = mock(SharePartition.class); SharePartition sp4 = mock(SharePartition.class); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2, tp3, tp4); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp1.maybeAcquireFetchLock()).thenReturn(true); when(sp2.maybeAcquireFetchLock()).thenReturn(true); @@ -933,7 +917,7 @@ public class DelayedShareFetchTest { sharePartitions.put(tp4, sp4); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1, tp2, tp3, tp4), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(FetchPartitionData.class))).thenReturn( @@ -948,7 +932,7 @@ public class DelayedShareFetchTest { createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); // All 5 partitions are acquirable. - doAnswer(invocation -> buildLogReadResult(sharePartitions.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(sharePartitions.keySet().stream().toList())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); @@ -1010,8 +994,6 @@ public class DelayedShareFetchTest { SharePartition sp3 = mock(SharePartition.class); SharePartition sp4 = mock(SharePartition.class); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2, tp3, tp4); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp1.maybeAcquireFetchLock()).thenReturn(true); when(sp2.maybeAcquireFetchLock()).thenReturn(false); @@ -1031,7 +1013,7 @@ public class DelayedShareFetchTest { sharePartitions.put(tp4, sp4); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1, tp2, tp3, tp4), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(FetchPartitionData.class))).thenReturn( @@ -1043,7 +1025,7 @@ public class DelayedShareFetchTest { Set acquirableTopicPartitions = new LinkedHashSet<>(); acquirableTopicPartitions.add(tp0); acquirableTopicPartitions.add(tp1); - doAnswer(invocation -> buildLogReadResult(acquirableTopicPartitions)).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(acquirableTopicPartitions.stream().toList())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); @@ -1090,7 +1072,6 @@ public class DelayedShareFetchTest { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -1104,7 +1085,7 @@ public class DelayedShareFetchTest { ShareFetch shareFetch = new ShareFetch( new FetchParams(FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); DelayedShareFetch delayedShareFetch = DelayedShareFetchBuilder.builder() @@ -1132,7 +1113,7 @@ public class DelayedShareFetchTest { fetchableTopicPartitions.add(tp1); fetchableTopicPartitions.add(tp2); // We will be doing replica manager fetch only for tp1 and tp2. - doAnswer(invocation -> buildLogReadResult(fetchableTopicPartitions)).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(fetchableTopicPartitions.stream().toList())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); LinkedHashMap combinedLogReadResponse = delayedShareFetch.combineLogReadResponse(topicPartitionData, logReadResponse); assertEquals(topicPartitionData.keySet(), combinedLogReadResponse.keySet()); @@ -1162,7 +1143,7 @@ public class DelayedShareFetchTest { public void testOnCompleteExecutionOnTimeout() { ShareFetch shareFetch = new ShareFetch( FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), - new CompletableFuture<>(), new LinkedHashMap<>(), BATCH_SIZE, MAX_FETCH_RECORDS, + new CompletableFuture<>(), List.of(), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); DelayedShareFetch delayedShareFetch = DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) diff --git a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java index f4bd75971ee..3ccd2ad7dab 100644 --- a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java +++ b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java @@ -58,11 +58,9 @@ import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; -import static kafka.server.share.SharePartitionManagerTest.PARTITION_MAX_BYTES; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createFileRecords; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.memoryRecordsBuilder; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.orderedMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -94,7 +92,6 @@ public class ShareFetchUtilsTest { String memberId = Uuid.randomUuid().toString(); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -114,7 +111,7 @@ public class ShareFetchUtilsTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, memberId, - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, 100, BROKER_TOPIC_STATS); + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, 100, BROKER_TOPIC_STATS); MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("0".getBytes(), "v".getBytes()), @@ -160,7 +157,6 @@ public class ShareFetchUtilsTest { String memberId = Uuid.randomUuid().toString(); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -176,7 +172,7 @@ public class ShareFetchUtilsTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, memberId, - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, 100, BROKER_TOPIC_STATS); + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, 100, BROKER_TOPIC_STATS); List responseData = List.of( new ShareFetchPartitionData(tp0, 0, new FetchPartitionData(Errors.NONE, 0L, 0L, @@ -207,8 +203,6 @@ public class ShareFetchUtilsTest { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); - SharePartition sp0 = Mockito.mock(SharePartition.class); SharePartition sp1 = Mockito.mock(SharePartition.class); @@ -217,7 +211,7 @@ public class ShareFetchUtilsTest { sharePartitions.put(tp1, sp1); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, 100, BROKER_TOPIC_STATS); + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, 100, BROKER_TOPIC_STATS); ReplicaManager replicaManager = mock(ReplicaManager.class); @@ -303,14 +297,13 @@ public class ShareFetchUtilsTest { String groupId = "grp"; TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); SharePartition sp0 = Mockito.mock(SharePartition.class); LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, 100, BROKER_TOPIC_STATS); + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, 100, BROKER_TOPIC_STATS); ReplicaManager replicaManager = mock(ReplicaManager.class); @@ -365,8 +358,6 @@ public class ShareFetchUtilsTest { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); - SharePartition sp0 = Mockito.mock(SharePartition.class); SharePartition sp1 = Mockito.mock(SharePartition.class); @@ -380,7 +371,7 @@ public class ShareFetchUtilsTest { Uuid memberId = Uuid.randomUuid(); // Set max fetch records to 10 ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, memberId.toString(), - new CompletableFuture<>(), partitionMaxBytes, BATCH_SIZE, 10, BROKER_TOPIC_STATS); + new CompletableFuture<>(), List.of(tp0, tp1), BATCH_SIZE, 10, BROKER_TOPIC_STATS); LinkedHashMap recordsPerOffset = new LinkedHashMap<>(); recordsPerOffset.put(0L, 1); diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 832303a58ef..c00897eee25 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -44,7 +44,6 @@ import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchRequest; -import org.apache.kafka.common.requests.ShareFetchRequest; import org.apache.kafka.common.requests.ShareFetchResponse; import org.apache.kafka.common.requests.ShareRequestMetadata; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; @@ -116,8 +115,7 @@ import scala.collection.Seq; import scala.jdk.javaapi.CollectionConverters; import static kafka.server.share.DelayedShareFetchTest.mockTopicIdPartitionToReturnDataEqualToMinBytes; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.orderedMap; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.validateRotatedMapEquals; +import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.validateRotatedListEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -157,7 +155,6 @@ public class SharePartitionManagerTest { 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true); private static final String TIMER_NAME_PREFIX = "share-partition-manager"; - static final int PARTITION_MAX_BYTES = 40000; static final int DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL = 1000; private Time time; @@ -201,9 +198,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); // Create a new share session with an initial share fetch request - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), PARTITION_MAX_BYTES)); - reqData1.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), PARTITION_MAX_BYTES)); + List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); @@ -211,7 +206,7 @@ public class SharePartitionManagerTest { assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), Collections.emptyList(), reqMetadata2, true); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(), reqMetadata2, true); assertEquals(FinalContext.class, context2.getClass()); } @@ -231,9 +226,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); // Create a new share session with an initial share fetch request - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), PARTITION_MAX_BYTES)); - reqData1.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), PARTITION_MAX_BYTES)); + List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); @@ -244,14 +237,13 @@ public class SharePartitionManagerTest { // shareFetch is not empty, but the maxBytes of topic partition is 0, which means this is added only for acknowledgements. // New context should be created successfully - Map reqData3 = Collections.singletonMap(new TopicIdPartition(tpId1, new TopicPartition("foo", 0)), - new ShareFetchRequest.SharePartitionData(tpId1, 0)); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData3, Collections.emptyList(), reqMetadata2, true); + List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true); assertEquals(FinalContext.class, context2.getClass()); } @Test - public void testNewContextReturnsFinalContextError() { + public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequestData() { ShareSessionCache cache = new ShareSessionCache(10, 1000); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) @@ -266,9 +258,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); // Create a new share session with an initial share fetch request - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), PARTITION_MAX_BYTES)); - reqData1.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), PARTITION_MAX_BYTES)); + List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); @@ -277,12 +267,10 @@ public class SharePartitionManagerTest { ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - // shareFetch is not empty and the maxBytes of topic partition is not 0, which means this is trying to fetch on a Final request. - // New context should throw an error - Map reqData3 = Collections.singletonMap(new TopicIdPartition(tpId1, new TopicPartition("foo", 0)), - new ShareFetchRequest.SharePartitionData(tpId1, PARTITION_MAX_BYTES)); - assertThrows(InvalidRequestException.class, - () -> sharePartitionManager.newContext(groupId, reqData3, Collections.emptyList(), reqMetadata2, true)); + // shareFetch is not empty, and it contains tpId1, which should return FinalContext instance since it is FINAL_EPOCH + List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); + assertInstanceOf(FinalContext.class, + sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true)); } @Test @@ -305,20 +293,14 @@ public class SharePartitionManagerTest { String groupId = "grp"; // Create a new share session with an initial share fetch request - Map reqData2 = new LinkedHashMap<>(); - reqData2.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), 100)); - reqData2.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), 100)); - + List reqData2 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); - ((ShareSessionContext) context2).shareFetchData().forEach((topicIdPartition, sharePartitionData) -> { - assertTrue(reqData2.containsKey(topicIdPartition)); - assertEquals(reqData2.get(topicIdPartition), sharePartitionData); - }); + ((ShareSessionContext) context2).shareFetchData().forEach(topicIdPartition -> assertTrue(reqData2.contains(topicIdPartition))); LinkedHashMap respData2 = new LinkedHashMap<>(); respData2.put(tp0, new ShareFetchResponseData.PartitionData().setPartitionIndex(0)); @@ -341,7 +323,7 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(memberId4, 1), true)); // Continue the first share session we created. - ShareFetchContext context5 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -351,9 +333,7 @@ public class SharePartitionManagerTest { shareSessionContext5.session().partitionMap().forEach(cachedSharePartition -> { TopicIdPartition topicIdPartition = new TopicIdPartition(cachedSharePartition.topicId(), new TopicPartition(cachedSharePartition.topic(), cachedSharePartition.partition())); - ShareFetchRequest.SharePartitionData data = cachedSharePartition.reqData(); - assertTrue(reqData2.containsKey(topicIdPartition)); - assertEquals(reqData2.get(topicIdPartition), data); + assertTrue(reqData2.contains(topicIdPartition)); }); } ShareFetchResponse resp5 = context5.updateAndGenerateResponseData(groupId, reqMetadata2.memberId(), respData2); @@ -365,14 +345,14 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session - ShareFetchContext context7 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); ShareFetchResponse resp7 = context7.throttleResponse(100); assertEquals(Errors.NONE, resp7.error()); assertEquals(100, resp7.throttleTimeMs()); // Get the final share session. - ShareFetchContext context8 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -407,9 +387,7 @@ public class SharePartitionManagerTest { TopicIdPartition foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1)); // Create a new share session, session 1 - Map session1req = new LinkedHashMap<>(); - session1req.put(foo0, new ShareFetchRequest.SharePartitionData(foo0.topicId(), 100)); - session1req.put(foo1, new ShareFetchRequest.SharePartitionData(foo1.topicId(), 100)); + List session1req = List.of(foo0, foo1); String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); @@ -432,9 +410,7 @@ public class SharePartitionManagerTest { time.sleep(500); // Create a second new share session - Map session2req = new LinkedHashMap<>(); - session2req.put(foo0, new ShareFetchRequest.SharePartitionData(foo0.topicId(), 100)); - session2req.put(foo1, new ShareFetchRequest.SharePartitionData(foo1.topicId(), 100)); + List session2req = List.of(foo0, foo1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); @@ -458,7 +434,7 @@ public class SharePartitionManagerTest { time.sleep(500); // Create a subsequent share fetch context for session 1 - ShareFetchContext session1context2 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext session1context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, session1context2); @@ -468,9 +444,7 @@ public class SharePartitionManagerTest { // create one final share session to test that the least recently used entry is evicted // the second share session should be evicted because the first share session was incrementally fetched // more recently than the second session was created - Map session3req = new LinkedHashMap<>(); - session3req.put(foo0, new ShareFetchRequest.SharePartitionData(foo0.topicId(), 100)); - session3req.put(foo1, new ShareFetchRequest.SharePartitionData(foo1.topicId(), 100)); + List session3req = List.of(foo0, foo1); ShareRequestMetadata reqMetadata3 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); @@ -505,9 +479,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp2 = new TopicIdPartition(barId, new TopicPartition("bar", 0)); // Create a new share session with foo-0 and foo-1 - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), 100)); - reqData1.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), 100)); + List reqData1 = List.of(tp0, tp1); String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); @@ -524,8 +496,7 @@ public class SharePartitionManagerTest { assertEquals(2, resp1.responseData(topicNames).size()); // Create a subsequent fetch request that removes foo-0 and adds bar-0 - Map reqData2 = Collections.singletonMap( - tp2, new ShareFetchRequest.SharePartitionData(tp2.topicId(), 100)); + List reqData2 = List.of(tp2); List removed2 = new ArrayList<>(); removed2.add(tp0); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, removed2, @@ -571,9 +542,7 @@ public class SharePartitionManagerTest { TopicIdPartition foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1)); // Create a new share session with foo-0 and foo-1 - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(foo0, new ShareFetchRequest.SharePartitionData(foo0.topicId(), 100)); - reqData1.put(foo1, new ShareFetchRequest.SharePartitionData(foo1.topicId(), 100)); + List reqData1 = List.of(foo0, foo1); String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); @@ -594,7 +563,7 @@ public class SharePartitionManagerTest { List removed2 = new ArrayList<>(); removed2.add(foo0); removed2.add(foo1); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), removed2, + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), removed2, new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context2); @@ -619,10 +588,7 @@ public class SharePartitionManagerTest { ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(foo, new ShareFetchRequest.SharePartitionData(foo.topicId(), 100)); - reqData1.put(bar, new ShareFetchRequest.SharePartitionData(bar.topicId(), 100)); - + List reqData1 = List.of(foo, bar); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); @@ -630,7 +596,7 @@ public class SharePartitionManagerTest { mockUpdateAndGenerateResponseData(context1, groupId, reqMetadata1.memberId()); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), Collections.singletonList(foo), + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), Collections.singletonList(foo), new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); // So foo is removed but not the others. @@ -638,9 +604,9 @@ public class SharePartitionManagerTest { mockUpdateAndGenerateResponseData(context2, groupId, reqMetadata1.memberId()); - ShareFetchContext context3 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), Collections.singletonList(bar), + ShareFetchContext context3 = sharePartitionManager.newContext(groupId, List.of(), Collections.singletonList(bar), new ShareRequestMetadata(reqMetadata1.memberId(), 2), true); - assertPartitionsPresent((ShareSessionContext) context3, Collections.emptyList()); + assertPartitionsPresent((ShareSessionContext) context3, List.of()); } // This test simulates a share session where the topic ID changes broker side (the one handling the request) in both the metadata cache and the log @@ -663,9 +629,7 @@ public class SharePartitionManagerTest { topicNames.put(barId, "bar"); // Create a new share session with foo-0 and bar-1 - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(foo, new ShareFetchRequest.SharePartitionData(foo.topicId(), 100)); - reqData1.put(bar, new ShareFetchRequest.SharePartitionData(bar.topicId(), 100)); + List reqData1 = List.of(foo, bar); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); @@ -683,7 +647,7 @@ public class SharePartitionManagerTest { assertEquals(2, resp1.responseData(topicNames).size()); // Create a subsequent share fetch request as though no topics changed. - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context2); @@ -715,11 +679,7 @@ public class SharePartitionManagerTest { String groupId = "grp"; // Create a new share session with an initial share fetch request - Map reqData2 = new LinkedHashMap<>(); - reqData2.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), 100)); - reqData2.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), 100)); - reqData2.put(tpNull1, new ShareFetchRequest.SharePartitionData(tpNull1.topicId(), 100)); - + List reqData2 = List.of(tp0, tp1, tpNull1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); @@ -751,7 +711,7 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(Uuid.randomUuid(), 1), true)); // Continue the first share session we created. - ShareFetchContext context5 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -766,8 +726,7 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session - Map reqData7 = Collections.singletonMap( - tpNull2, new ShareFetchRequest.SharePartitionData(tpNull2.topicId(), 100)); + List reqData7 = List.of(tpNull2); ShareFetchContext context7 = sharePartitionManager.newContext(groupId, reqData7, EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); // Check for throttled response @@ -778,12 +737,12 @@ public class SharePartitionManagerTest { assertErroneousAndValidTopicIdPartitions(context7.getErroneousAndValidTopicIdPartitions(), Arrays.asList(tpNull1, tpNull2), Arrays.asList(tp0, tp1)); // Get the final share session. - ShareFetchContext context8 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); - assertErroneousAndValidTopicIdPartitions(context8.getErroneousAndValidTopicIdPartitions(), Collections.emptyList(), Collections.emptyList()); + assertErroneousAndValidTopicIdPartitions(context8.getErroneousAndValidTopicIdPartitions(), List.of(), List.of()); // Check for throttled response ShareFetchResponse resp8 = context8.throttleResponse(100); assertEquals(Errors.NONE, resp8.error()); @@ -817,9 +776,7 @@ public class SharePartitionManagerTest { String groupId = "grp"; // Create a new share session with an initial share fetch request - Map reqData2 = new LinkedHashMap<>(); - reqData2.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), 100)); - reqData2.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), 100)); + List reqData2 = List.of(tp0, tp1); // For response size expected value calculation ObjectSerializationCache objectSerializationCache = new ObjectSerializationCache(); @@ -854,8 +811,7 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(memberId4, 1), true)); // Continue the first share session we created. - Map reqData5 = Collections.singletonMap( - tp2, new ShareFetchRequest.SharePartitionData(tp2.topicId(), 100)); + List reqData5 = List.of(tp2); ShareFetchContext context5 = sharePartitionManager.newContext(groupId, reqData5, EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); @@ -875,7 +831,7 @@ public class SharePartitionManagerTest { new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session - ShareFetchContext context7 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); int respSize7 = context7.responseSize(respData2, version); @@ -886,7 +842,7 @@ public class SharePartitionManagerTest { assertEquals(4 + new ShareFetchResponseData().size(objectSerializationCache, version), respSize7); // Get the final share session. - ShareFetchContext context8 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -930,9 +886,7 @@ public class SharePartitionManagerTest { Uuid memberId2 = Uuid.randomUuid(); // Create a new share session with an initial share fetch request. - Map reqData1 = new LinkedHashMap<>(); - reqData1.put(tp0, new ShareFetchRequest.SharePartitionData(tp0.topicId(), 100)); - reqData1.put(tp1, new ShareFetchRequest.SharePartitionData(tp1.topicId(), 100)); + List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); @@ -953,8 +907,7 @@ public class SharePartitionManagerTest { new HashSet<>(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId1))); // Create a new share session with an initial share fetch request. - Map reqData2 = Collections.singletonMap( - tp2, new ShareFetchRequest.SharePartitionData(tp2.topicId(), 100)); + List reqData2 = List.of(tp2); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); @@ -973,8 +926,7 @@ public class SharePartitionManagerTest { assertEquals(Collections.singletonList(tp2), sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId2)); // Continue the first share session we created. - Map reqData3 = Collections.singletonMap( - tp2, new ShareFetchRequest.SharePartitionData(tp2.topicId(), 100)); + List reqData3 = List.of(tp2); ShareFetchContext context3 = sharePartitionManager.newContext(groupId, reqData3, EMPTY_PART_LIST, new ShareRequestMetadata(shareSessionKey1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context3); @@ -989,8 +941,7 @@ public class SharePartitionManagerTest { new HashSet<>(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId1))); // Continue the second session we created. - Map reqData4 = Collections.singletonMap( - tp3, new ShareFetchRequest.SharePartitionData(tp3.topicId(), 100)); + List reqData4 = List.of(tp3); ShareFetchContext context4 = sharePartitionManager.newContext(groupId, reqData4, Collections.singletonList(tp2), new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context4); @@ -1004,7 +955,7 @@ public class SharePartitionManagerTest { assertEquals(Collections.singletonList(tp3), sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId2)); // Get the final share session. - ShareFetchContext context5 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), EMPTY_PART_LIST, + ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, new ShareRequestMetadata(reqMetadata1.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context5.getClass()); @@ -1019,7 +970,7 @@ public class SharePartitionManagerTest { assertTrue(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId1).isEmpty()); // Continue the second share session . - ShareFetchContext context6 = sharePartitionManager.newContext(groupId, Collections.emptyMap(), Collections.singletonList(tp3), + ShareFetchContext context6 = sharePartitionManager.newContext(groupId, List.of(), Collections.singletonList(tp3), new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); assertInstanceOf(ShareSessionContext.class, context6); assertTrue(((ShareSessionContext) context6).isSubsequent()); @@ -1028,7 +979,7 @@ public class SharePartitionManagerTest { ShareFetchResponse resp6 = context6.updateAndGenerateResponseData(groupId, reqMetadata2.memberId(), respData6); assertEquals(Errors.NONE, resp6.error()); - assertEquals(Collections.emptyList(), sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId2)); + assertEquals(List.of(), sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId2)); } @Test @@ -1067,7 +1018,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp4 = new TopicIdPartition(fooId, new TopicPartition("foo", 2)); TopicIdPartition tp5 = new TopicIdPartition(barId, new TopicPartition("bar", 2)); TopicIdPartition tp6 = new TopicIdPartition(fooId, new TopicPartition("foo", 3)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2, tp3, tp4, tp5, tp6); + List topicIdPartitions = List.of(tp0, tp1, tp2, tp3, tp4, tp5, tp6); mockFetchOffsetForTimestamp(mockReplicaManager); @@ -1090,22 +1041,22 @@ public class SharePartitionManagerTest { .withBrokerTopicStats(brokerTopicStats) .build(); - doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(topicIdPartitions)).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); CompletableFuture> future = sharePartitionManager.fetchMessages( - groupId, memberId1.toString(), FETCH_PARAMS, 1, BATCH_SIZE, partitionMaxBytes); + groupId, memberId1.toString(), FETCH_PARAMS, 1, BATCH_SIZE, topicIdPartitions); assertTrue(future.isDone()); Mockito.verify(mockReplicaManager, times(1)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); future = sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 3, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); assertTrue(future.isDone()); Mockito.verify(mockReplicaManager, times(2)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); future = sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 10, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); assertTrue(future.isDone()); Mockito.verify(mockReplicaManager, times(3)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); @@ -1129,7 +1080,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1)); TopicIdPartition tp2 = new TopicIdPartition(barId, new TopicPartition("bar", 0)); TopicIdPartition tp3 = new TopicIdPartition(barId, new TopicPartition("bar", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2, tp3); + List topicIdPartitions = List.of(tp0, tp1, tp2, tp3); mockFetchOffsetForTimestamp(mockReplicaManager); @@ -1165,31 +1116,31 @@ public class SharePartitionManagerTest { assertEquals(4, sp1.nextFetchOffset()); assertEquals(10, sp2.nextFetchOffset()); assertEquals(20, sp3.nextFetchOffset()); - return buildLogReadResult(partitionMaxBytes.keySet()); + return buildLogReadResult(topicIdPartitions); }).doAnswer(invocation -> { assertEquals(15, sp0.nextFetchOffset()); assertEquals(1, sp1.nextFetchOffset()); assertEquals(25, sp2.nextFetchOffset()); assertEquals(15, sp3.nextFetchOffset()); - return buildLogReadResult(partitionMaxBytes.keySet()); + return buildLogReadResult(topicIdPartitions); }).doAnswer(invocation -> { assertEquals(6, sp0.nextFetchOffset()); assertEquals(18, sp1.nextFetchOffset()); assertEquals(26, sp2.nextFetchOffset()); assertEquals(23, sp3.nextFetchOffset()); - return buildLogReadResult(partitionMaxBytes.keySet()); + return buildLogReadResult(topicIdPartitions); }).doAnswer(invocation -> { assertEquals(30, sp0.nextFetchOffset()); assertEquals(5, sp1.nextFetchOffset()); assertEquals(26, sp2.nextFetchOffset()); assertEquals(16, sp3.nextFetchOffset()); - return buildLogReadResult(partitionMaxBytes.keySet()); + return buildLogReadResult(topicIdPartitions); }).doAnswer(invocation -> { assertEquals(25, sp0.nextFetchOffset()); assertEquals(5, sp1.nextFetchOffset()); assertEquals(26, sp2.nextFetchOffset()); assertEquals(16, sp3.nextFetchOffset()); - return buildLogReadResult(partitionMaxBytes.keySet()); + return buildLogReadResult(topicIdPartitions); }).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); int threadCount = 100; @@ -1199,7 +1150,7 @@ public class SharePartitionManagerTest { for (int i = 0; i != threadCount; ++i) { executorService.submit(() -> { sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); }); // We are blocking the main thread at an interval of 10 threads so that the currently running executorService threads can complete. if (i % 10 == 0) @@ -1222,7 +1173,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); Uuid fooId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); when(sp0.maybeAcquireFetchLock()).thenReturn(true); @@ -1246,7 +1197,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); Mockito.verify(mockReplicaManager, times(0)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); Map result = future.join(); @@ -1265,7 +1216,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); Uuid fooId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); mockFetchOffsetForTimestamp(mockReplicaManager); @@ -1282,10 +1233,10 @@ public class SharePartitionManagerTest { .withBrokerTopicStats(brokerTopicStats) .build(); - doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(topicIdPartitions)).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); // Since the nextFetchOffset does not point to endOffset + 1, i.e. some of the records in the cachedState are AVAILABLE, // even though the maxInFlightMessages limit is exceeded, replicaManager.readFromLog should be called Mockito.verify(mockReplicaManager, times(1)).readFromLog( @@ -1757,7 +1708,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo1", 0)); TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo2", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp1, tp2); + List topicIdPartitions = List.of(tp1, tp2); SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); @@ -1780,7 +1731,7 @@ public class SharePartitionManagerTest { groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), - partitionMaxBytes, + topicIdPartitions, BATCH_SIZE, 100, brokerTopicStats); @@ -1802,7 +1753,7 @@ public class SharePartitionManagerTest { when(sp2.acquire(anyString(), anyInt(), anyInt(), anyLong(), any())).thenReturn(ShareAcquiredRecords.empty()); List delayedShareFetchWatchKeys = new ArrayList<>(); - partitionMaxBytes.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + topicIdPartitions.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); sharePartitionManager = SharePartitionManagerBuilder.builder() .withPartitionCacheMap(partitionCacheMap) @@ -1827,7 +1778,7 @@ public class SharePartitionManagerTest { // Since acquisition lock for sp1 and sp2 cannot be acquired, we should have 2 watched keys. assertEquals(2, delayedShareFetchPurgatory.watched()); - doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(topicIdPartitions)).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); Map> acknowledgeTopics = new HashMap<>(); acknowledgeTopics.put(tp1, Arrays.asList( @@ -1861,7 +1812,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo2", 0)); TopicIdPartition tp3 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo3", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp1, tp2); + List topicIdPartitions = List.of(tp1, tp2); SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); @@ -1891,7 +1842,7 @@ public class SharePartitionManagerTest { groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), - partitionMaxBytes, + topicIdPartitions, BATCH_SIZE, 100, brokerTopicStats); @@ -1911,7 +1862,7 @@ public class SharePartitionManagerTest { when(sp3.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); - partitionMaxBytes.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + topicIdPartitions.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); sharePartitionManager = SharePartitionManagerBuilder.builder() .withPartitionCacheMap(partitionCacheMap) @@ -1967,7 +1918,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo2", 0)); TopicIdPartition tp3 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo3", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp1, tp2); + List topicIdPartitions = List.of(tp1, tp2); SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); @@ -1995,7 +1946,7 @@ public class SharePartitionManagerTest { groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), - partitionMaxBytes, + topicIdPartitions, BATCH_SIZE, 100, brokerTopicStats); @@ -2015,7 +1966,7 @@ public class SharePartitionManagerTest { when(sp2.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); - partitionMaxBytes.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + topicIdPartitions.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); sharePartitionManager = spy(SharePartitionManagerBuilder.builder() .withPartitionCacheMap(partitionCacheMap) @@ -2043,8 +1994,8 @@ public class SharePartitionManagerTest { // The share session for this share group member returns tp1 and tp3, tp1 is common in both the delayed fetch request and the share session. when(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, Uuid.fromString(memberId))).thenReturn(Arrays.asList(tp1, tp3)); - doAnswer(invocation -> buildLogReadResult(Set.of(tp1))).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); - when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any())).thenReturn(new ShareAcquiredRecords(Collections.emptyList(), 0)); + doAnswer(invocation -> buildLogReadResult(List.of(tp1))).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any())).thenReturn(new ShareAcquiredRecords(List.of(), 0)); // Release acquired records on session close request for tp1 and tp3. sharePartitionManager.releaseSession(groupId, memberId); @@ -2067,7 +2018,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo2", 0)); TopicIdPartition tp3 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo3", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp1, tp2); + List topicIdPartitions = List.of(tp1, tp2); SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); @@ -2101,7 +2052,7 @@ public class SharePartitionManagerTest { groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), - partitionMaxBytes, + topicIdPartitions, BATCH_SIZE, 100, brokerTopicStats); @@ -2121,7 +2072,7 @@ public class SharePartitionManagerTest { when(sp3.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); - partitionMaxBytes.keySet().forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + topicIdPartitions.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); sharePartitionManager = spy(SharePartitionManagerBuilder.builder() .withPartitionCacheMap(partitionCacheMap) @@ -2169,7 +2120,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); Uuid fooId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); Map partitionCacheMap = new HashMap<>(); @@ -2200,7 +2151,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); // Verify that the fetch request is completed. TestUtils.waitForCondition( future::isDone, @@ -2233,7 +2184,7 @@ public class SharePartitionManagerTest { String groupId = "grp"; TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); + List topicIdPartitions = List.of(tp0, tp1); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); @@ -2270,7 +2221,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); // Verify that the fetch request is completed. TestUtils.waitForCondition( future::isDone, @@ -2302,7 +2253,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); Uuid fooId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); Map partitionCacheMap = new HashMap<>(); @@ -2332,15 +2283,15 @@ public class SharePartitionManagerTest { // Send 3 requests for share fetch for same share partition. CompletableFuture> future1 = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); CompletableFuture> future2 = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); CompletableFuture> future3 = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); Mockito.verify(sp0, times(3)).maybeInitialize(); Mockito.verify(mockReplicaManager, times(3)).addDelayedShareFetchRequest(any(), any()); @@ -2376,7 +2327,7 @@ public class SharePartitionManagerTest { Uuid memberId = Uuid.randomUuid(); Uuid fooId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); Map partitionCacheMap = new HashMap<>(); @@ -2399,7 +2350,7 @@ public class SharePartitionManagerTest { when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new LeaderNotAvailableException("Leader not available"))); CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2415,7 +2366,7 @@ public class SharePartitionManagerTest { // Return IllegalStateException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new IllegalStateException("Illegal state"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2429,7 +2380,7 @@ public class SharePartitionManagerTest { // Return CoordinatorNotAvailableException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new CoordinatorNotAvailableException("Coordinator not available"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2443,7 +2394,7 @@ public class SharePartitionManagerTest { // Return InvalidRequestException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new InvalidRequestException("Invalid request"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2457,7 +2408,7 @@ public class SharePartitionManagerTest { // Return FencedStateEpochException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new FencedStateEpochException("Fenced state epoch"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2471,7 +2422,7 @@ public class SharePartitionManagerTest { // Return NotLeaderOrFollowerException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new NotLeaderOrFollowerException("Not leader or follower"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2485,7 +2436,7 @@ public class SharePartitionManagerTest { // Return RuntimeException to simulate initialization failure. when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new RuntimeException("Runtime exception"))); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2507,7 +2458,7 @@ public class SharePartitionManagerTest { public void testShareFetchProcessingExceptions() throws Exception { String groupId = "grp"; TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); Map partitionCacheMap = (Map) mock(Map.class); // Throw the exception for first fetch request. Return share partition for next. @@ -2521,7 +2472,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2539,7 +2490,7 @@ public class SharePartitionManagerTest { public void testSharePartitionInitializationFailure() throws Exception { String groupId = "grp"; TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); // Send map to check no share partition is created. Map partitionCacheMap = new HashMap<>(); @@ -2562,7 +2513,7 @@ public class SharePartitionManagerTest { // Validate when exception is thrown. CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2572,7 +2523,7 @@ public class SharePartitionManagerTest { // Validate when partition is not leader. future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); TestUtils.waitForCondition( future::isDone, DELAYED_SHARE_FETCH_TIMEOUT_MS, @@ -2597,7 +2548,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp1 = new TopicIdPartition(memberId1, new TopicPartition("foo", 1)); // For tp2, share partition initialization will fail. TopicIdPartition tp2 = new TopicIdPartition(memberId1, new TopicPartition("foo", 2)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1, tp2); + List topicIdPartitions = List.of(tp0, tp1, tp2); // Mark partition0 as not the leader. Partition partition0 = mock(Partition.class); @@ -2614,7 +2565,7 @@ public class SharePartitionManagerTest { when(sp1.maybeAcquireFetchLock()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp1.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); - when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any())).thenReturn(new ShareAcquiredRecords(Collections.emptyList(), 0)); + when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any())).thenReturn(new ShareAcquiredRecords(List.of(), 0)); // Fail initialization for tp2. SharePartition sp2 = mock(SharePartition.class); @@ -2629,7 +2580,7 @@ public class SharePartitionManagerTest { when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp1, 1); - doAnswer(invocation -> buildLogReadResult(Collections.singleton(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + doAnswer(invocation -> buildLogReadResult(List.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); sharePartitionManager = SharePartitionManagerBuilder.builder() .withReplicaManager(replicaManager) @@ -2641,7 +2592,7 @@ public class SharePartitionManagerTest { // Validate when exception is thrown. CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); assertTrue(future.isDone()); assertFalse(future.isCompletedExceptionally()); @@ -2672,7 +2623,7 @@ public class SharePartitionManagerTest { String groupId = "grp"; Uuid memberId = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0); + List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); when(sp0.maybeAcquireFetchLock()).thenReturn(true); @@ -2698,7 +2649,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); validateShareFetchFutureException(future, tp0, Errors.UNKNOWN_SERVER_ERROR, "Exception"); // Verify that the share partition is still in the cache on exception. assertEquals(1, partitionCacheMap.size()); @@ -2707,7 +2658,7 @@ public class SharePartitionManagerTest { doThrow(new NotLeaderOrFollowerException("Leader exception")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); validateShareFetchFutureException(future, tp0, Errors.NOT_LEADER_OR_FOLLOWER, "Leader exception"); assertTrue(partitionCacheMap.isEmpty()); // Should have 2 fetch recorded and 2 failures. @@ -2725,7 +2676,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); + List topicIdPartitions = List.of(tp0, tp1); SharePartition sp0 = mock(SharePartition.class); when(sp0.maybeAcquireFetchLock()).thenReturn(true); @@ -2760,7 +2711,7 @@ public class SharePartitionManagerTest { CompletableFuture> future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); validateShareFetchFutureException(future, tp0, Errors.FENCED_STATE_EPOCH, "Fenced exception"); // Verify that tp1 is still in the cache on exception. assertEquals(1, partitionCacheMap.size()); @@ -2775,7 +2726,7 @@ public class SharePartitionManagerTest { doThrow(new FencedStateEpochException("Fenced exception again")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, 0, - BATCH_SIZE, partitionMaxBytes); + BATCH_SIZE, topicIdPartitions); validateShareFetchFutureException(future, List.of(tp0, tp1), Errors.FENCED_STATE_EPOCH, "Fenced exception again"); assertTrue(partitionCacheMap.isEmpty()); // Should have 4 fetch recorded (2 fetch and 2 topics) and 3 failures as sp1 was not acquired @@ -2795,7 +2746,7 @@ public class SharePartitionManagerTest { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, tp0, tp1); + List topicIdPartitions = List.of(tp0, tp1); ReplicaManager mockReplicaManager = mock(ReplicaManager.class); Partition partition = mockPartition(); @@ -2807,7 +2758,7 @@ public class SharePartitionManagerTest { .build(); CompletableFuture> future = sharePartitionManager.fetchMessages( - groupId, memberId.toString(), FETCH_PARAMS, 0, BATCH_SIZE, partitionMaxBytes); + groupId, memberId.toString(), FETCH_PARAMS, 0, BATCH_SIZE, topicIdPartitions); assertTrue(future.isDone()); // Validate that the listener is registered. verify(mockReplicaManager, times(2)).maybeAddListener(any(), any()); @@ -2864,50 +2815,49 @@ public class SharePartitionManagerTest { TopicIdPartition tp4 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); TopicIdPartition tp5 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 2)); TopicIdPartition tp6 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 3)); - LinkedHashMap partitionMaxBytes = orderedMap(PARTITION_MAX_BYTES, - tp0, tp1, tp2, tp3, tp4, tp5, tp6); + List topicIdPartitions = List.of(tp0, tp1, tp2, tp3, tp4, tp5, tp6); sharePartitionManager = Mockito.spy(SharePartitionManagerBuilder.builder().withBrokerTopicStats(brokerTopicStats).build()); // Capture the arguments passed to processShareFetch. ArgumentCaptor captor = ArgumentCaptor.forClass(ShareFetch.class); sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 0, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); verify(sharePartitionManager, times(1)).processShareFetch(captor.capture()); // Verify the partitions rotation, no rotation. ShareFetch resultShareFetch = captor.getValue(); - validateRotatedMapEquals(resultShareFetch.partitionMaxBytes(), partitionMaxBytes, 0); + validateRotatedListEquals(resultShareFetch.topicIdPartitions(), topicIdPartitions, 0); // Single rotation. sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 1, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); verify(sharePartitionManager, times(2)).processShareFetch(captor.capture()); // Verify the partitions rotation, rotate by 1. resultShareFetch = captor.getValue(); - validateRotatedMapEquals(partitionMaxBytes, resultShareFetch.partitionMaxBytes(), 1); + validateRotatedListEquals(topicIdPartitions, resultShareFetch.topicIdPartitions(), 1); // Rotation by 3, less that the number of partitions. sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 3, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); verify(sharePartitionManager, times(3)).processShareFetch(captor.capture()); // Verify the partitions rotation, rotate by 3. resultShareFetch = captor.getValue(); - validateRotatedMapEquals(partitionMaxBytes, resultShareFetch.partitionMaxBytes(), 3); + validateRotatedListEquals(topicIdPartitions, resultShareFetch.topicIdPartitions(), 3); // Rotation by 12, more than the number of partitions. sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, 12, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); verify(sharePartitionManager, times(4)).processShareFetch(captor.capture()); // Verify the partitions rotation, rotate by 5 (12 % 7). resultShareFetch = captor.getValue(); - validateRotatedMapEquals(partitionMaxBytes, resultShareFetch.partitionMaxBytes(), 5); + validateRotatedListEquals(topicIdPartitions, resultShareFetch.topicIdPartitions(), 5); // Rotation by Integer.MAX_VALUE, boundary test. sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, Integer.MAX_VALUE, BATCH_SIZE, - partitionMaxBytes); + topicIdPartitions); verify(sharePartitionManager, times(5)).processShareFetch(captor.capture()); // Verify the partitions rotation, rotate by 1 (2147483647 % 7). resultShareFetch = captor.getValue(); - validateRotatedMapEquals(partitionMaxBytes, resultShareFetch.partitionMaxBytes(), 1); + validateRotatedListEquals(topicIdPartitions, resultShareFetch.topicIdPartitions(), 1); } private Timer systemTimerReaper() { @@ -2969,7 +2919,7 @@ public class SharePartitionManagerTest { if (context.getClass() == ShareSessionContext.class) { ShareSessionContext shareSessionContext = (ShareSessionContext) context; if (!shareSessionContext.isSubsequent()) { - shareSessionContext.shareFetchData().forEach((topicIdPartition, sharePartitionData) -> data.put(topicIdPartition, + shareSessionContext.shareFetchData().forEach(topicIdPartition -> data.put(topicIdPartition, topicIdPartition.topic() == null ? errorShareFetchResponse(Errors.UNKNOWN_TOPIC_ID.code()) : noErrorShareFetchResponse())); } else { @@ -2989,8 +2939,7 @@ public class SharePartitionManagerTest { private void assertPartitionsPresent(ShareSessionContext context, List partitions) { Set partitionsInContext = new HashSet<>(); if (!context.isSubsequent()) { - context.shareFetchData().forEach((topicIdPartition, sharePartitionData) -> - partitionsInContext.add(topicIdPartition)); + partitionsInContext.addAll(context.shareFetchData()); } else { context.session().partitionMap().forEach(cachedSharePartition -> { TopicIdPartition topicIdPartition = new TopicIdPartition(cachedSharePartition.topicId(), new @@ -3008,11 +2957,9 @@ public class SharePartitionManagerTest { Set expectedErroneousSet = new HashSet<>(expectedErroneous); Set expectedValidSet = new HashSet<>(expectedValid); Set actualErroneousPartitions = new HashSet<>(); - Set actualValidPartitions = new HashSet<>(); erroneousAndValidPartitionData.erroneous().forEach((topicIdPartition, partitionData) -> actualErroneousPartitions.add(topicIdPartition)); - erroneousAndValidPartitionData.validTopicIdPartitions().forEach((topicIdPartition, partitionData) -> - actualValidPartitions.add(topicIdPartition)); + Set actualValidPartitions = new HashSet<>(erroneousAndValidPartitionData.validTopicIdPartitions()); assertEquals(expectedErroneousSet, actualErroneousPartitions); assertEquals(expectedValidSet, actualValidPartitions); } @@ -3076,7 +3023,7 @@ public class SharePartitionManagerTest { }); } - static Seq> buildLogReadResult(Set topicIdPartitions) { + static Seq> buildLogReadResult(List topicIdPartitions) { List> logReadResults = new ArrayList<>(); topicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( new FetchDataInfo(new LogOffsetMetadata(0, 0, 0), MemoryRecords.EMPTY), diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 5dbfca8d335..c62926c82c7 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -148,7 +148,6 @@ class KafkaApisTest extends Logging { private val time = new MockTime private val clientId = "" private var kafkaApis: KafkaApis = _ - private val partitionMaxBytes = 40000 @AfterEach def tearDown(): Unit = { @@ -3918,10 +3917,8 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, shareSessionEpoch), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, shareSessionEpoch), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName))) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -3931,12 +3928,11 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(shareSessionEpoch). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(partitionIndex) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(partitionIndex))))) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -3991,7 +3987,7 @@ class KafkaApisTest extends Logging { val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes), false)) + new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenThrow( Errors.INVALID_REQUEST.exception() @@ -4006,20 +4002,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(partitionIndex) - .setPartitionMaxBytes(partitionMaxBytes) - setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -4039,14 +4034,13 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava) - ).asJava) + )) + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4095,10 +4089,9 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ).thenThrow(Errors.INVALID_REQUEST.exception) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( @@ -4117,12 +4110,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -4154,7 +4146,6 @@ class KafkaApisTest extends Logging { setPartitions(List( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) // partitionMaxBytes are set even on the final fetch request, this is an invalid request .setAcknowledgementBatches(List( new AcknowledgementBatch() .setFirstOffset(0) @@ -4187,10 +4178,9 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4200,12 +4190,11 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -4255,7 +4244,7 @@ class KafkaApisTest extends Logging { val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes), false)) + new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( @@ -4269,20 +4258,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -4319,7 +4307,7 @@ class KafkaApisTest extends Logging { val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes), false)) + new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( @@ -4333,20 +4321,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -4384,10 +4371,9 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4397,12 +4383,11 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -4440,7 +4425,7 @@ class KafkaApisTest extends Logging { when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), any())).thenReturn( CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + new TopicIdPartition(topicId, partitionIndex, topicName) -> new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records) @@ -4454,10 +4439,9 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ).thenThrow(Errors.SHARE_SESSION_NOT_FOUND.exception) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4467,12 +4451,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -4503,12 +4486,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId2.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4547,10 +4529,9 @@ class KafkaApisTest extends Logging { ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ).thenThrow(Errors.INVALID_SHARE_SESSION_EPOCH.exception) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4560,12 +4541,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -4593,12 +4573,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(2). // Invalid share session epoch, should have 1 for the second request. - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(0))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4685,14 +4664,13 @@ class KafkaApisTest extends Logging { val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes), false) + new TopicIdPartition(topicId, partitionIndex, topicName), false) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( @@ -4706,12 +4684,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(partitionIndex) - .setPartitionMaxBytes(partitionMaxBytes)).asJava)).asJava) + .setPartitionIndex(partitionIndex))))) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -4745,17 +4722,16 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition(). setPartitionIndex(partitionIndex). - setPartitionMaxBytes(partitionMaxBytes). - setAcknowledgementBatches(List( + setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch(). setFirstOffset(0). setLastOffset(9). - setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava)).asJava)).asJava)).asJava) + setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava))))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4783,17 +4759,16 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(2). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition(). setPartitionIndex(partitionIndex). - setPartitionMaxBytes(partitionMaxBytes). - setAcknowledgementBatches(List( + setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch(). setFirstOffset(10). setLastOffset(19). - setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava)).asJava)).asJava)).asJava) + setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava))))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4935,40 +4910,36 @@ class KafkaApisTest extends Logging { val cachedSharePartitions1 = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions1.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes), false + new TopicIdPartition(topicId1, 0, topicName1), false )) cachedSharePartitions1.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes), false + new TopicIdPartition(topicId1, 1, topicName1), false )) cachedSharePartitions1.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes), false + new TopicIdPartition(topicId2, 0, topicName2), false )) cachedSharePartitions1.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)), new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes), false + new TopicIdPartition(topicId2, 1, topicName2), false )) cachedSharePartitions1.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareFetchRequest.SharePartitionData(topicId3, partitionMaxBytes), false + new TopicIdPartition(topicId3, 0, topicName3), false )) val cachedSharePartitions2 = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions2.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareFetchRequest.SharePartitionData(topicId3, partitionMaxBytes), false + new TopicIdPartition(topicId3, 0, topicName3), false )) cachedSharePartitions2.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)), new ShareFetchRequest.SharePartitionData(topicId4, partitionMaxBytes), false + new TopicIdPartition(topicId4, 0, topicName4), false )) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)) -> - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes), - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)) -> - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)) -> - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) -> - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) + )) ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 0L, 0L, 2)) ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( @@ -5024,28 +4995,24 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava) - ).asJava) + )) + )) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -5126,15 +5093,14 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId3). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), - ).asJava) + )), + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -5161,23 +5127,22 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(2). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId4). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), - ).asJava) - .setForgottenTopicsData(List( + )), + )) + .setForgottenTopicsData(util.List.of( new ForgottenTopic() .setTopicId(topicId1) - .setPartitions(List(Integer.valueOf(0), Integer.valueOf(1)).asJava), + .setPartitions(util.List.of(Integer.valueOf(0), Integer.valueOf(1))), new ForgottenTopic() .setTopicId(topicId2) - .setPartitions(List(Integer.valueOf(0), Integer.valueOf(1)).asJava) - ).asJava) + .setPartitions(util.List.of(Integer.valueOf(0), Integer.valueOf(1))) + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -5221,78 +5186,72 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(-1). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)), + )), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(19) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)), + )) + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(43) .setLastOffset(52) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)), + )), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(17) .setLastOffset(26) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)), + )) + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId3). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(54) .setLastOffset(93) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava), - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)), + )), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId4). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(24) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), - ).asJava), - ).asJava), - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)), + )), + )), + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -5367,19 +5326,7 @@ class KafkaApisTest extends Logging { val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() - val validPartitions: util.Map[TopicIdPartition, ShareFetchRequest.SharePartitionData] = new util.HashMap() - validPartitions.put( - tp1, - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes) - ) - validPartitions.put( - tp2, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) - validPartitions.put( - tp3, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) + val validPartitions: util.List[TopicIdPartition] = util.List.of(tp1, tp2, tp3) val erroneousAndValidPartitionData: ErroneousAndValidPartitionData = new ErroneousAndValidPartitionData(erroneousPartitions, validPartitions) @@ -5392,25 +5339,22 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(shareSessionEpoch). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), - ).asJava) + )), + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -5522,11 +5466,7 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) ) - val validPartitions: util.Map[TopicIdPartition, ShareFetchRequest.SharePartitionData] = new util.HashMap() - validPartitions.put( - tp1, - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes) - ) + val validPartitions: util.List[TopicIdPartition] = util.List.of(tp1) val erroneousAndValidPartitionData: ErroneousAndValidPartitionData = new ErroneousAndValidPartitionData(erroneousPartitions, validPartitions) @@ -5539,25 +5479,22 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(shareSessionEpoch). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), - ).asJava), - ).asJava) + .setPartitionIndex(0), + )), + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -5658,19 +5595,7 @@ class KafkaApisTest extends Logging { val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() - val validPartitions: util.Map[TopicIdPartition, ShareFetchRequest.SharePartitionData] = new util.HashMap() - validPartitions.put( - tp1, - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes) - ) - validPartitions.put( - tp2, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) - validPartitions.put( - tp3, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) + val validPartitions: util.List[TopicIdPartition] = util.List.of(tp1, tp2, tp3) val erroneousAndValidPartitionData: ErroneousAndValidPartitionData = new ErroneousAndValidPartitionData(erroneousPartitions, validPartitions) @@ -5683,25 +5608,22 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(shareSessionEpoch). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), - ).asJava) + )), + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -5817,23 +5739,7 @@ class KafkaApisTest extends Logging { val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() - val validPartitions: util.Map[TopicIdPartition, ShareFetchRequest.SharePartitionData] = new util.HashMap() - validPartitions.put( - tp1, - new ShareFetchRequest.SharePartitionData(topicId1, partitionMaxBytes) - ) - validPartitions.put( - tp2, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) - validPartitions.put( - tp3, - new ShareFetchRequest.SharePartitionData(topicId2, partitionMaxBytes) - ) - validPartitions.put( - tp4, - new ShareFetchRequest.SharePartitionData(topicId3, partitionMaxBytes) - ) + val validPartitions: util.List[TopicIdPartition] = util.List.of(tp1, tp2, tp3, tp4) val erroneousAndValidPartitionData: ErroneousAndValidPartitionData = new ErroneousAndValidPartitionData(erroneousPartitions, validPartitions) @@ -5847,32 +5753,28 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(shareSessionEpoch). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes), + .setPartitionIndex(0), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId3). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(partitionMaxBytes) - ).asJava), - ).asJava) + )), + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -6013,14 +5915,13 @@ class KafkaApisTest extends Logging { val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] cachedSharePartitions.mustAdd(new CachedSharePartition( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes), false + new TopicIdPartition(topicId, 0, topicName), false )) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( - new ShareSessionContext(new ShareRequestMetadata(memberId, 0), Map( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> - new ShareFetchRequest.SharePartitionData(topicId, partitionMaxBytes) - ).asJava) + new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( + new TopicIdPartition(topicId, partitionIndex, topicName) + )) ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) ) @@ -6041,12 +5942,11 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() - .setPartitionIndex(0) - .setPartitionMaxBytes(40000)).asJava)).asJava) + .setPartitionIndex(0))))) var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) @@ -6073,20 +5973,19 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -6118,20 +6017,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -6162,20 +6060,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -6209,20 +6106,19 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(partitionIndex) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any(), any())).thenReturn(List[AuthorizationResult]( @@ -6965,50 +6861,46 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(Uuid.randomUuid().toString). setShareSessionEpoch(0). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), + .setAcknowledgeTypes(util.List.of(1.toByte)), new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(17) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)) + )), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(2.toByte)) - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of(2.toByte)) + )) + )), new ShareFetchRequestData.FetchTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(24) .setLastOffset(65) - .setAcknowledgeTypes(Collections.singletonList(3.toByte)) - ).asJava), + .setAcknowledgeTypes(util.List.of(3.toByte)) + )), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(40000) - ).asJava) - ).asJava) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val topicNames = new util.HashMap[Uuid, String] topicNames.put(topicId1, "foo1") @@ -7042,43 +6934,40 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(Uuid.randomUuid().toString). setShareSessionEpoch(0). - setTopics(List( + setTopics(util.List.of( new ShareFetchRequestData.FetchTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(7.toByte)) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) - ).asJava), + .setAcknowledgeTypes(util.List.of(7.toByte)) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) + )), new ShareFetchRequestData.FetchPartition() .setPartitionIndex(1) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.emptyList()) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of()) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) + )) + )), new ShareFetchRequestData.FetchTopic() .setTopicId(topicId2) - .setPartitions(List( + .setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setPartitionMaxBytes(40000) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareFetchRequestData.AcknowledgementBatch() .setFirstOffset(24) .setLastOffset(65) - .setAcknowledgeTypes(Collections.singletonList(3.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(3.toByte)) + )) + )) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val topicIdNames = new util.HashMap[Uuid, String] topicIdNames.put(topicId1, "foo1") // topicId2 is not present in topicIdNames diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 32677f7c4c2..e0a662f74eb 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -6082,8 +6082,7 @@ class ReplicaManagerTest { try { val groupId = "grp" val tp1 = new TopicIdPartition(Uuid.randomUuid, new TopicPartition("foo1", 0)) - val partitionMaxBytes = new util.LinkedHashMap[TopicIdPartition, Integer] - partitionMaxBytes.put(tp1, 1000) + val topicPartitions = util.List.of(tp1) val sp1 = mock(classOf[SharePartition]) val sharePartitions = new util.LinkedHashMap[TopicIdPartition, SharePartition] @@ -6095,7 +6094,7 @@ class ReplicaManagerTest { groupId, Uuid.randomUuid.toString, future, - partitionMaxBytes, + topicPartitions, 500, 100, brokerTopicStats) @@ -6109,7 +6108,7 @@ class ReplicaManagerTest { time)) val delayedShareFetchWatchKeys : util.List[DelayedShareFetchKey] = new util.ArrayList[DelayedShareFetchKey] - partitionMaxBytes.keySet.forEach((topicIdPartition: TopicIdPartition) => delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId, topicIdPartition.partition))) + topicPartitions.forEach((topicIdPartition: TopicIdPartition) => delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId, topicIdPartition.partition))) // You cannot acquire records for sp1, so request will be stored in purgatory waiting for timeout. when(sp1.maybeAcquireFetchLock).thenReturn(false) diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 843a2b4c3ba..1e5775e7b6f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -37,8 +37,7 @@ import scala.jdk.CollectionConverters._ )) @Tag("integration") class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster){ - - private final val MAX_PARTITION_BYTES = 10000 + private final val MAX_WAIT_MS = 5000 @AfterEach @@ -59,7 +58,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic1", 1)) ) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) assertEquals(Errors.UNSUPPORTED_VERSION.code(), shareFetchResponse.data().errorCode()) @@ -127,7 +126,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) // Send the share fetch request to the non-replica and verify the error code - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest, nonReplicaId) val partitionData = shareFetchResponse.responseData(topicNames).get(topicIdPartition) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode) @@ -181,7 +180,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -251,7 +250,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) // For the multi partition fetch request, the response may not be available in the first attempt // as the share partitions might not be initialized yet. So, we retry until we get the response. @@ -352,9 +351,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the first share fetch request to initialize the share partitions // Create different share fetch requests for different partitions as they may have leaders on separate brokers - var shareFetchRequest1 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send1, Seq.empty, acknowledgementsMap) - var shareFetchRequest2 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send2, Seq.empty, acknowledgementsMap) - var shareFetchRequest3 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send3, Seq.empty, acknowledgementsMap) + var shareFetchRequest1 = createShareFetchRequest(groupId, metadata, send1, Seq.empty, acknowledgementsMap) + var shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) + var shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) var shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) var shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) @@ -368,9 +367,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above // Create different share fetch requests for different partitions as they may have leaders on separate brokers - shareFetchRequest1 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send1, Seq.empty, acknowledgementsMap) - shareFetchRequest2 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send2, Seq.empty, acknowledgementsMap) - shareFetchRequest3 = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send3, Seq.empty, acknowledgementsMap) + shareFetchRequest1 = createShareFetchRequest(groupId, metadata, send1, Seq.empty, acknowledgementsMap) + shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) + shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) @@ -469,7 +468,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -517,7 +516,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a third share fetch request to check if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -585,7 +584,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch: Int = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -613,7 +612,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(0) .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -637,7 +636,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a fourth share fetch request to confirm if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -705,7 +704,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -750,7 +749,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a third share fetch request to check if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -818,7 +817,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -862,7 +861,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records releaseAcknowledgementSent = true } - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -935,7 +934,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -983,7 +982,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a third share fetch request to check if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1051,7 +1050,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -1079,7 +1078,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(0) .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(3.toByte))).asJava) // Reject the records - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1103,7 +1102,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a fourth share fetch request to confirm if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1173,7 +1172,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -1218,7 +1217,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a third share fetch request to check if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1265,7 +1264,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a fourth share fetch request to check if acknowledgements were done successfully shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1284,79 +1283,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTests( - Array( - new ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - ) - ), - new ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") - ) - ), - ) - ) - def testShareFetchBrokerDoesNotRespectPartitionsSizeLimit(): Unit = { - val groupId: String = "group" - val memberId = Uuid.randomUuid() - - val topic = "topic" - val partition = 0 - - createTopicAndReturnLeaders(topic, numPartitions = 3) - val topicIds = getTopicIds.asJava - val topicId = topicIds.get(topic) - val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topic, partition)) - - val send: Seq[TopicIdPartition] = Seq(topicIdPartition) - - // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) - - initProducer() - // Producing 3 large messages to the topic created above - produceData(topicIdPartition, 10) - produceData(topicIdPartition, "large message 1", new String(new Array[Byte](MAX_PARTITION_BYTES/3))) - produceData(topicIdPartition, "large message 2", new String(new Array[Byte](MAX_PARTITION_BYTES/3))) - produceData(topicIdPartition, "large message 3", new String(new Array[Byte](MAX_PARTITION_BYTES/3))) - - // Send the second share fetch request to fetch the records produced above - val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) - val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) - - val shareFetchResponseData = shareFetchResponse.data() - assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) - assertEquals(1, shareFetchResponseData.responses().size()) - assertEquals(topicId, shareFetchResponseData.responses().get(0).topicId()) - assertEquals(1, shareFetchResponseData.responses().get(0).partitions().size()) - - val expectedPartitionData = new ShareFetchResponseData.PartitionData() - .setPartitionIndex(partition) - .setErrorCode(Errors.NONE.code()) - .setAcknowledgeErrorCode(Errors.NONE.code()) - .setAcquiredRecords(expectedAcquiredRecords(Collections.singletonList(0), Collections.singletonList(12), Collections.singletonList(1))) - // The first 10 records will be consumed as it is. For the last 3 records, each of size MAX_PARTITION_BYTES/3, - // all 3 of then will be consumed (offsets 10, 11 and 12) because even though the inclusion of the third last record will exceed - // the max partition bytes limit. We should only consider the request level maxBytes as the hard limit. - - val partitionData = shareFetchResponseData.responses().get(0).partitions().get(0) - compareFetchResponsePartitions(expectedPartitionData, partitionData) - } - @ClusterTests( Array( new ClusterTest( @@ -1410,15 +1336,15 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // mocking the behaviour of multiple share consumers from the same share group val metadata1: ShareRequestMetadata = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap1: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest1 = createShareFetchRequest(groupId, metadata1, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap1, minBytes = 100, maxBytes = 1500) + val shareFetchRequest1 = createShareFetchRequest(groupId, metadata1, send, Seq.empty, acknowledgementsMap1, minBytes = 100, maxBytes = 1500) val metadata2: ShareRequestMetadata = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap2: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest2 = createShareFetchRequest(groupId, metadata2, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap2, minBytes = 100, maxBytes = 1500) + val shareFetchRequest2 = createShareFetchRequest(groupId, metadata2, send, Seq.empty, acknowledgementsMap2, minBytes = 100, maxBytes = 1500) val metadata3: ShareRequestMetadata = new ShareRequestMetadata(memberId3, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest3 = createShareFetchRequest(groupId, metadata3, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap3, minBytes = 100, maxBytes = 1500) + val shareFetchRequest3 = createShareFetchRequest(groupId, metadata3, send, Seq.empty, acknowledgementsMap3, minBytes = 100, maxBytes = 1500) val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) @@ -1510,15 +1436,15 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // mocking the behaviour of 3 different share groups val metadata1 = new ShareRequestMetadata(memberId1, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap1: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest1 = createShareFetchRequest(groupId1, metadata1, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap1) + val shareFetchRequest1 = createShareFetchRequest(groupId1, metadata1, send, Seq.empty, acknowledgementsMap1) val metadata2 = new ShareRequestMetadata(memberId2, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap2: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest2 = createShareFetchRequest(groupId2, metadata2, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap2) + val shareFetchRequest2 = createShareFetchRequest(groupId2, metadata2, send, Seq.empty, acknowledgementsMap2) val metadata3 = new ShareRequestMetadata(memberId3, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - val shareFetchRequest3 = createShareFetchRequest(groupId3, metadata3, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap3) + val shareFetchRequest3 = createShareFetchRequest(groupId3, metadata3, send, Seq.empty, acknowledgementsMap3) val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) @@ -1604,7 +1530,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -1632,7 +1558,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(0) .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1657,7 +1583,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(10) .setLastOffset(19) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records - shareFetchRequest = createShareFetchRequest(groupId, metadata, 0, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1714,7 +1640,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -1742,7 +1668,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(0) .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMapForFetch) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -1833,7 +1759,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setFirstOffset(0) .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Acknowledgements in the Initial Fetch Request - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -1940,7 +1866,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -1961,7 +1887,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a thord Share Fetch request with invalid share session epoch shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.nextEpoch(shareSessionEpoch)) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -2016,7 +1942,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -2098,7 +2024,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() @@ -2119,7 +2045,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Sending a third Share Fetch request with wrong member Id shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(wrongMemberId, shareSessionEpoch) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() @@ -2175,7 +2101,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the second share fetch request to fetch the records produced above var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, Map.empty) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -2260,7 +2186,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty - var shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, send, Seq.empty, acknowledgementsMap) + var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) // For the multi partition fetch request, the response may not be available in the first attempt // as the share partitions might not be initialized yet. So, we retry until we get the response. @@ -2290,7 +2216,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val forget: Seq[TopicIdPartition] = Seq(topicIdPartition1) - shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, Seq.empty, forget, acknowledgementsMap) + shareFetchRequest = createShareFetchRequest(groupId, metadata, Seq.empty, forget, acknowledgementsMap) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -2315,7 +2241,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val partitions: util.Set[Integer] = new util.HashSet() TestUtils.waitUntilTrue(() => { val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, MAX_PARTITION_BYTES, topicIdPartitions, Seq.empty, Map.empty) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, topicIdPartitions, Seq.empty, Map.empty) val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() @@ -2358,7 +2284,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo private def createShareFetchRequest(groupId: String, metadata: ShareRequestMetadata, - maxPartitionBytes: Int, send: Seq[TopicIdPartition], forget: Seq[TopicIdPartition], acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]], @@ -2366,7 +2291,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo minBytes: Int = 0, maxBytes: Int = Int.MaxValue, batchSize: Int = 500): ShareFetchRequest = { - ShareFetchRequest.Builder.forConsumer(groupId, metadata, maxWaitMs, minBytes, maxBytes, maxPartitionBytes, batchSize, send.asJava, forget.asJava, acknowledgementsMap.asJava) + ShareFetchRequest.Builder.forConsumer(groupId, metadata, maxWaitMs, minBytes, maxBytes, batchSize, send.asJava, forget.asJava, acknowledgementsMap.asJava) .build() } diff --git a/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java b/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java index aab42c901dd..8cfc9365039 100644 --- a/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java +++ b/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.ShareFetchRequest; import org.apache.kafka.common.requests.ShareFetchResponse; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; @@ -42,33 +41,30 @@ public class CachedSharePartition implements ImplicitLinkedHashCollection.Elemen private final Uuid topicId; private final int partition; private final Optional leaderEpoch; - private int maxBytes; private boolean requiresUpdateInResponse; private int cachedNext = ImplicitLinkedHashCollection.INVALID_INDEX; private int cachedPrev = ImplicitLinkedHashCollection.INVALID_INDEX; - private CachedSharePartition(String topic, Uuid topicId, int partition, int maxBytes, Optional leaderEpoch, + private CachedSharePartition(String topic, Uuid topicId, int partition, Optional leaderEpoch, boolean requiresUpdateInResponse) { this.topic = topic; this.topicId = topicId; this.partition = partition; - this.maxBytes = maxBytes; this.leaderEpoch = leaderEpoch; this.requiresUpdateInResponse = requiresUpdateInResponse; } public CachedSharePartition(String topic, Uuid topicId, int partition, boolean requiresUpdateInResponse) { - this(topic, topicId, partition, -1, Optional.empty(), requiresUpdateInResponse); + this(topic, topicId, partition, Optional.empty(), requiresUpdateInResponse); } public CachedSharePartition(TopicIdPartition topicIdPartition) { this(topicIdPartition.topic(), topicIdPartition.topicId(), topicIdPartition.partition(), false); } - public CachedSharePartition(TopicIdPartition topicIdPartition, ShareFetchRequest.SharePartitionData reqData, - boolean requiresUpdateInResponse) { - this(topicIdPartition.topic(), topicIdPartition.topicId(), topicIdPartition.partition(), reqData.maxBytes, + public CachedSharePartition(TopicIdPartition topicIdPartition, boolean requiresUpdateInResponse) { + this(topicIdPartition.topic(), topicIdPartition.topicId(), topicIdPartition.partition(), Optional.empty(), requiresUpdateInResponse); } @@ -84,15 +80,6 @@ public class CachedSharePartition implements ImplicitLinkedHashCollection.Elemen return partition; } - public ShareFetchRequest.SharePartitionData reqData() { - return new ShareFetchRequest.SharePartitionData(topicId, maxBytes); - } - - public void updateRequestParams(ShareFetchRequest.SharePartitionData reqData) { - // Update our cached request parameters. - maxBytes = reqData.maxBytes; - } - /** * Determine whether the specified cached partition should be included in the ShareFetchResponse we send back to * the fetcher and update it if requested. @@ -128,7 +115,6 @@ public class CachedSharePartition implements ImplicitLinkedHashCollection.Elemen return "CachedSharePartition(topic=" + topic + ", topicId=" + topicId + ", partition=" + partition + - ", maxBytes=" + maxBytes + ", leaderEpoch=" + leaderEpoch + ")"; } diff --git a/server/src/main/java/org/apache/kafka/server/share/ErroneousAndValidPartitionData.java b/server/src/main/java/org/apache/kafka/server/share/ErroneousAndValidPartitionData.java index 9918fbdf48e..d2a1e68a11c 100644 --- a/server/src/main/java/org/apache/kafka/server/share/ErroneousAndValidPartitionData.java +++ b/server/src/main/java/org/apache/kafka/server/share/ErroneousAndValidPartitionData.java @@ -20,10 +20,11 @@ package org.apache.kafka.server.share; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.ShareFetchRequest; import org.apache.kafka.common.requests.ShareFetchResponse; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -31,36 +32,36 @@ import java.util.Map; */ public class ErroneousAndValidPartitionData { private final Map erroneous; - private final Map validTopicIdPartitions; + private final List validTopicIdPartitions; public ErroneousAndValidPartitionData(Map erroneous, - Map validTopicIdPartitions) { + List validTopicIdPartitions) { this.erroneous = erroneous; this.validTopicIdPartitions = validTopicIdPartitions; } - public ErroneousAndValidPartitionData(Map shareFetchData) { + public ErroneousAndValidPartitionData(List shareFetchData) { erroneous = new HashMap<>(); - validTopicIdPartitions = new HashMap<>(); - shareFetchData.forEach((topicIdPartition, sharePartitionData) -> { + validTopicIdPartitions = new ArrayList<>(); + shareFetchData.forEach(topicIdPartition -> { if (topicIdPartition.topic() == null) { erroneous.put(topicIdPartition, ShareFetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)); } else { - validTopicIdPartitions.put(topicIdPartition, sharePartitionData); + validTopicIdPartitions.add(topicIdPartition); } }); } public ErroneousAndValidPartitionData() { - this.erroneous = new HashMap<>(); - this.validTopicIdPartitions = new HashMap<>(); + this.erroneous = Map.of(); + this.validTopicIdPartitions = List.of(); } public Map erroneous() { return erroneous; } - public Map validTopicIdPartitions() { + public List validTopicIdPartitions() { return validTopicIdPartitions; } } diff --git a/server/src/main/java/org/apache/kafka/server/share/context/ShareSessionContext.java b/server/src/main/java/org/apache/kafka/server/share/context/ShareSessionContext.java index 18df489af33..99eb92a85b9 100644 --- a/server/src/main/java/org/apache/kafka/server/share/context/ShareSessionContext.java +++ b/server/src/main/java/org/apache/kafka/server/share/context/ShareSessionContext.java @@ -23,8 +23,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.ShareFetchRequest; -import org.apache.kafka.common.requests.ShareFetchRequest.SharePartitionData; import org.apache.kafka.common.requests.ShareFetchResponse; import org.apache.kafka.common.requests.ShareRequestMetadata; import org.apache.kafka.server.share.CachedSharePartition; @@ -34,6 +32,7 @@ import org.apache.kafka.server.share.session.ShareSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -52,7 +51,7 @@ public class ShareSessionContext extends ShareFetchContext { private final ShareRequestMetadata reqMetadata; private final boolean isSubsequent; - private Map shareFetchData; + private List shareFetchData; private ShareSession session; /** @@ -62,7 +61,7 @@ public class ShareSessionContext extends ShareFetchContext { * @param shareFetchData The share partition data from the share fetch request. */ public ShareSessionContext(ShareRequestMetadata reqMetadata, - Map shareFetchData) { + List shareFetchData) { this.reqMetadata = reqMetadata; this.shareFetchData = shareFetchData; this.isSubsequent = false; @@ -81,7 +80,7 @@ public class ShareSessionContext extends ShareFetchContext { } // Visible for testing - public Map shareFetchData() { + public List shareFetchData() { return shareFetchData; } @@ -229,17 +228,16 @@ public class ShareSessionContext extends ShareFetchContext { return new ErroneousAndValidPartitionData(shareFetchData); } Map erroneous = new HashMap<>(); - Map valid = new HashMap<>(); + List valid = new ArrayList<>(); // Take the session lock and iterate over all the cached partitions. synchronized (session) { session.partitionMap().forEach(cachedSharePartition -> { TopicIdPartition topicIdPartition = new TopicIdPartition(cachedSharePartition.topicId(), new TopicPartition(cachedSharePartition.topic(), cachedSharePartition.partition())); - ShareFetchRequest.SharePartitionData reqData = cachedSharePartition.reqData(); if (topicIdPartition.topic() == null) { erroneous.put(topicIdPartition, ShareFetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)); } else { - valid.put(topicIdPartition, reqData); + valid.add(topicIdPartition); } }); return new ErroneousAndValidPartitionData(erroneous, valid); diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategy.java b/server/src/main/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategy.java index 42fdaa58cce..600f18165b0 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategy.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategy.java @@ -18,9 +18,10 @@ package org.apache.kafka.server.share.fetch; import org.apache.kafka.common.TopicIdPartition; -import java.util.LinkedHashMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Locale; -import java.util.Map; /** * The PartitionRotateStrategy is used to rotate the partitions based on the respective strategy. @@ -48,7 +49,7 @@ public interface PartitionRotateStrategy { * * @return the rotated topicIdPartitions */ - LinkedHashMap rotate(LinkedHashMap topicIdPartitions, PartitionRotateMetadata metadata); + List rotate(List topicIdPartitions, PartitionRotateMetadata metadata); static PartitionRotateStrategy type(StrategyType type) { return switch (type) { @@ -64,8 +65,8 @@ public interface PartitionRotateStrategy { * * @return the rotated topicIdPartitions */ - static LinkedHashMap rotateRoundRobin( - LinkedHashMap topicIdPartitions, + static List rotateRoundRobin( + List topicIdPartitions, PartitionRotateMetadata metadata ) { if (topicIdPartitions.isEmpty() || topicIdPartitions.size() == 1 || metadata.sessionEpoch < 1) { @@ -80,20 +81,11 @@ public interface PartitionRotateStrategy { return topicIdPartitions; } - // TODO: Once the partition max bytes is removed then the partition will be a linked list and rotation - // will be a simple operation. Else consider using ImplicitLinkedHashCollection. - LinkedHashMap suffixPartitions = new LinkedHashMap<>(rotateAt); - LinkedHashMap rotatedPartitions = new LinkedHashMap<>(topicIdPartitions.size()); - int i = 0; - for (Map.Entry entry : topicIdPartitions.entrySet()) { - if (i < rotateAt) { - suffixPartitions.put(entry.getKey(), entry.getValue()); - } else { - rotatedPartitions.put(entry.getKey(), entry.getValue()); - } - i++; - } - rotatedPartitions.putAll(suffixPartitions); + // Avoid modifying the original list, create copy. + List rotatedPartitions = new ArrayList<>(topicIdPartitions); + // Elements from the list should move left by the distance provided i.e. if the original list is [1,2,3], + // and rotation is by 1, then output should be [2,3,1] and not [3,1,2]. Hence, negate the distance here. + Collections.rotate(rotatedPartitions, -1 * rotateAt); return rotatedPartitions; } diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java b/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java index 521e0807268..8406f9efa91 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java @@ -26,8 +26,8 @@ import org.apache.kafka.storage.log.metrics.BrokerTopicStats; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -55,9 +55,9 @@ public class ShareFetch { */ private final String memberId; /** - * The maximum number of bytes that can be fetched for each partition. + * The topic partitions to be fetched. */ - private final LinkedHashMap partitionMaxBytes; + private final List topicIdPartitions; /** * The batch size of the fetch request. */ @@ -81,7 +81,7 @@ public class ShareFetch { String groupId, String memberId, CompletableFuture> future, - LinkedHashMap partitionMaxBytes, + List topicIdPartitions, int batchSize, int maxFetchRecords, BrokerTopicStats brokerTopicStats @@ -90,7 +90,7 @@ public class ShareFetch { this.groupId = groupId; this.memberId = memberId; this.future = future; - this.partitionMaxBytes = partitionMaxBytes; + this.topicIdPartitions = topicIdPartitions; this.batchSize = batchSize; this.maxFetchRecords = maxFetchRecords; this.brokerTopicStats = brokerTopicStats; @@ -104,8 +104,8 @@ public class ShareFetch { return memberId; } - public LinkedHashMap partitionMaxBytes() { - return partitionMaxBytes; + public List topicIdPartitions() { + return topicIdPartitions; } public FetchParams fetchParams() { @@ -151,7 +151,7 @@ public class ShareFetch { * @return true if all the partitions in the request have errored, false otherwise. */ public synchronized boolean errorInAllPartitions() { - return erroneous != null && erroneous.size() == partitionMaxBytes().size(); + return erroneous != null && erroneous.size() == topicIdPartitions().size(); } /** diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java index 97e46d62126..362f32e6197 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java @@ -18,7 +18,6 @@ package org.apache.kafka.server.share.session; import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.common.requests.ShareFetchRequest; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.server.share.CachedSharePartition; @@ -110,25 +109,21 @@ public class ShareSession { return new LastUsedKey(key, lastUsedMs); } - // Visible for testing - public synchronized long creationMs() { - return creationMs; - } - // Update the cached partition data based on the request. - public synchronized Map> update(Map shareFetchData, List toForget) { + public synchronized Map> update( + List shareFetchData, + List toForget + ) { List added = new ArrayList<>(); List updated = new ArrayList<>(); List removed = new ArrayList<>(); - shareFetchData.forEach((topicIdPartition, sharePartitionData) -> { - CachedSharePartition cachedSharePartitionKey = new CachedSharePartition(topicIdPartition, sharePartitionData, true); + shareFetchData.forEach(topicIdPartition -> { + CachedSharePartition cachedSharePartitionKey = new CachedSharePartition(topicIdPartition, true); CachedSharePartition cachedPart = partitionMap.find(cachedSharePartitionKey); if (cachedPart == null) { partitionMap.mustAdd(cachedSharePartitionKey); added.add(topicIdPartition); } else { - cachedPart.updateRequestParams(sharePartitionData); updated.add(topicIdPartition); } }); diff --git a/server/src/test/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategyTest.java b/server/src/test/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategyTest.java index f05490c8747..da53b0af6a8 100644 --- a/server/src/test/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategyTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/fetch/PartitionRotateStrategyTest.java @@ -24,9 +24,10 @@ import org.apache.kafka.server.share.fetch.PartitionRotateStrategy.StrategyType; import org.junit.jupiter.api.Test; -import java.util.LinkedHashMap; +import java.util.ArrayList; +import java.util.List; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.validateRotatedMapEquals; +import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.validateRotatedListEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -35,64 +36,64 @@ public class PartitionRotateStrategyTest { @Test public void testRoundRobinStrategy() { PartitionRotateStrategy strategy = PartitionRotateStrategy.type(StrategyType.ROUND_ROBIN); - LinkedHashMap partitions = createPartitions(3); + List partitions = createPartitions(3); - LinkedHashMap result = strategy.rotate(partitions, new PartitionRotateMetadata(1)); + List result = strategy.rotate(partitions, new PartitionRotateMetadata(1)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 1); + validateRotatedListEquals(partitions, result, 1); // Session epoch is greater than the number of partitions. result = strategy.rotate(partitions, new PartitionRotateMetadata(5)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 2); + validateRotatedListEquals(partitions, result, 2); // Session epoch is at Integer.MAX_VALUE. result = strategy.rotate(partitions, new PartitionRotateMetadata(Integer.MAX_VALUE)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 1); + validateRotatedListEquals(partitions, result, 1); // No rotation at same size as epoch. result = strategy.rotate(partitions, new PartitionRotateMetadata(3)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 0); + validateRotatedListEquals(partitions, result, 0); } @Test public void testRoundRobinStrategyWithSpecialSessionEpochs() { PartitionRotateStrategy strategy = PartitionRotateStrategy.type(StrategyType.ROUND_ROBIN); - LinkedHashMap partitions = createPartitions(3); - LinkedHashMap result = strategy.rotate( + List partitions = createPartitions(3); + List result = strategy.rotate( partitions, new PartitionRotateMetadata(ShareRequestMetadata.INITIAL_EPOCH)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 0); + validateRotatedListEquals(partitions, result, 0); result = strategy.rotate( partitions, new PartitionRotateMetadata(ShareRequestMetadata.FINAL_EPOCH)); assertEquals(3, result.size()); - validateRotatedMapEquals(partitions, result, 0); + validateRotatedListEquals(partitions, result, 0); } @Test public void testRoundRobinStrategyWithEmptyPartitions() { PartitionRotateStrategy strategy = PartitionRotateStrategy.type(StrategyType.ROUND_ROBIN); // Empty partitions. - LinkedHashMap result = strategy.rotate(new LinkedHashMap<>(), new PartitionRotateMetadata(5)); + List result = strategy.rotate(new ArrayList<>(), new PartitionRotateMetadata(5)); // The result should be empty. assertTrue(result.isEmpty()); } /** - * Create an ordered map of TopicIdPartition to partition max bytes. + * Create a list of topic partitions. * @param size The number of topic-partitions to create. - * @return The ordered map of TopicIdPartition to partition max bytes. + * @return The list of topic partitions. */ - private LinkedHashMap createPartitions(int size) { - LinkedHashMap partitions = new LinkedHashMap<>(); + private List createPartitions(int size) { + List partitions = new ArrayList<>(); for (int i = 0; i < size; i++) { - partitions.put(new TopicIdPartition(Uuid.randomUuid(), i, "foo" + i), 1 /* partition max bytes*/); + partitions.add(new TopicIdPartition(Uuid.randomUuid(), i, "foo" + i)); } return partitions; } diff --git a/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTest.java b/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTest.java index faa01d14938..f7e29e2484f 100644 --- a/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTest.java @@ -32,7 +32,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.orderedMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -60,7 +59,7 @@ public class ShareFetchTest { public void testErrorInAllPartitions() { TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, new CompletableFuture<>(), - orderedMap(10, topicIdPartition), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition), BATCH_SIZE, 100, brokerTopicStats); assertFalse(shareFetch.errorInAllPartitions()); shareFetch.addErroneous(topicIdPartition, new RuntimeException()); @@ -72,7 +71,7 @@ public class ShareFetchTest { TopicIdPartition topicIdPartition0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition topicIdPartition1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, new CompletableFuture<>(), - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); assertFalse(shareFetch.errorInAllPartitions()); shareFetch.addErroneous(topicIdPartition0, new RuntimeException()); @@ -87,7 +86,7 @@ public class ShareFetchTest { TopicIdPartition topicIdPartition0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition topicIdPartition1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, new CompletableFuture<>(), - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); Set result = shareFetch.filterErroneousTopicPartitions(Set.of(topicIdPartition0, topicIdPartition1)); // No erroneous partitions, hence all partitions should be returned. assertEquals(2, result.size()); @@ -113,7 +112,7 @@ public class ShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, future, - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); // Add both erroneous partition and complete request. shareFetch.addErroneous(topicIdPartition0, new RuntimeException()); @@ -134,7 +133,7 @@ public class ShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, future, - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); // Add an erroneous partition and complete request. shareFetch.addErroneous(topicIdPartition0, new RuntimeException()); @@ -154,7 +153,7 @@ public class ShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, future, - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); shareFetch.maybeCompleteWithException(List.of(topicIdPartition0, topicIdPartition1), new RuntimeException()); assertEquals(2, future.join().size()); @@ -173,7 +172,7 @@ public class ShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, future, - orderedMap(10, topicIdPartition0, topicIdPartition1, topicIdPartition2), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1, topicIdPartition2), BATCH_SIZE, 100, brokerTopicStats); shareFetch.maybeCompleteWithException(List.of(topicIdPartition0, topicIdPartition2), new RuntimeException()); assertEquals(2, future.join().size()); @@ -191,7 +190,7 @@ public class ShareFetchTest { CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(mock(FetchParams.class), GROUP_ID, MEMBER_ID, future, - orderedMap(10, topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); + List.of(topicIdPartition0, topicIdPartition1), BATCH_SIZE, 100, brokerTopicStats); shareFetch.addErroneous(topicIdPartition0, new RuntimeException()); shareFetch.maybeCompleteWithException(List.of(topicIdPartition1), new RuntimeException()); diff --git a/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTestUtils.java b/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTestUtils.java index db7f15ef4c3..db3aa45d687 100644 --- a/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTestUtils.java +++ b/server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTestUtils.java @@ -30,15 +30,12 @@ import com.yammer.metrics.core.Gauge; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Set; import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.jupiter.api.Assertions.assertArrayEquals; -import static org.junit.jupiter.api.Assertions.assertEquals; /** * Helper functions for writing share fetch unit tests. @@ -46,49 +43,29 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class ShareFetchTestUtils { /** - * Create an ordered map of TopicIdPartition to partition max bytes. + * Validate that the rotated list is equal to the original list rotated by the given position. * - * @param partitionMaxBytes The maximum number of bytes that can be fetched for each partition. - * @param topicIdPartitions The topic partitions to create the map for. - * @return The ordered map of TopicIdPartition to partition max bytes. + * @param original The original list. + * @param result The rotated list. + * @param rotationAt The position to rotate the elements at. */ - public static LinkedHashMap orderedMap(int partitionMaxBytes, TopicIdPartition... topicIdPartitions) { - LinkedHashMap map = new LinkedHashMap<>(); - for (TopicIdPartition tp : topicIdPartitions) { - map.put(tp, partitionMaxBytes); - } - return map; - } - - /** - * Validate that the rotated map is equal to the original map with the keys rotated by the given position. - * - * @param original The original map. - * @param result The rotated map. - * @param rotationAt The position to rotate the keys at. - */ - public static void validateRotatedMapEquals( - LinkedHashMap original, - LinkedHashMap result, + public static void validateRotatedListEquals( + List original, + List result, int rotationAt ) { - Set originalKeys = original.keySet(); - Set resultKeys = result.keySet(); - TopicIdPartition[] originalKeysArray = new TopicIdPartition[originalKeys.size()]; + TopicIdPartition[] originalKeysArray = new TopicIdPartition[original.size()]; int i = 0; - for (TopicIdPartition key : originalKeys) { + for (TopicIdPartition key : original) { if (i < rotationAt) { - originalKeysArray[originalKeys.size() - rotationAt + i] = key; + originalKeysArray[original.size() - rotationAt + i] = key; } else { originalKeysArray[i - rotationAt] = key; } i++; } - assertArrayEquals(originalKeysArray, resultKeys.toArray()); - for (TopicIdPartition key : originalKeys) { - assertEquals(original.get(key), result.get(key)); - } + assertArrayEquals(originalKeysArray, result.toArray()); } /**