diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java index 6efa9b3723c..84b0db51c7e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java @@ -477,7 +477,7 @@ public class GroupCoordinatorRecordHelpers { .setGeneration(0) .setLeader(null) .setCurrentStateTimestamp(group.currentStateTimestampOrDefault()) - .setMembers(Collections.emptyList()), + .setMembers(List.of()), GROUP_METADATA_VALUE_VERSION ) ); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 21b9cddfea6..cdf43f793d3 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -115,7 +115,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.IntSupplier; -import java.util.stream.Collectors; import static org.apache.kafka.coordinator.common.runtime.CoordinatorOperationExceptionHelper.handleOperationException; @@ -400,7 +399,7 @@ public class GroupCoordinatorService implements GroupCoordinator { return CompletableFuture.completedFuture( new StreamsGroupHeartbeatResult( new StreamsGroupHeartbeatResponseData().setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()), - Collections.emptyMap() + Map.of() ) ); } @@ -419,7 +418,7 @@ public class GroupCoordinatorService implements GroupCoordinator { new StreamsGroupHeartbeatResponseData() .setErrorCode(error.code()) .setErrorMessage(message), - Collections.emptyMap() + Map.of() ), log )); @@ -633,7 +632,7 @@ public class GroupCoordinatorService implements GroupCoordinator { .setMemberId(member.memberId()) .setGroupInstanceId(member.groupInstanceId()) .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())) - .collect(Collectors.toList()); + .toList(); return new LeaveGroupResponseData() .setMembers(memberResponses); } else { @@ -671,7 +670,7 @@ public class GroupCoordinatorService implements GroupCoordinator { exception -> { exception = Errors.maybeUnwrapException(exception); if (exception instanceof NotCoordinatorException) { - return Collections.emptyList(); + return List.of(); } else { throw new CompletionException(exception); } @@ -714,7 +713,7 @@ public class GroupCoordinatorService implements GroupCoordinator { .computeIfAbsent(topicPartitionFor(groupId), __ -> new ArrayList<>()) .add(groupId); } else { - futures.add(CompletableFuture.completedFuture(Collections.singletonList( + futures.add(CompletableFuture.completedFuture(List.of( new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId(null) .setErrorCode(Errors.INVALID_GROUP_ID.code()) @@ -766,7 +765,7 @@ public class GroupCoordinatorService implements GroupCoordinator { .computeIfAbsent(topicPartitionFor(groupId), __ -> new ArrayList<>()) .add(groupId); } else { - futures.add(CompletableFuture.completedFuture(Collections.singletonList( + futures.add(CompletableFuture.completedFuture(List.of( new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(null) .setErrorCode(Errors.INVALID_GROUP_ID.code()) @@ -817,7 +816,7 @@ public class GroupCoordinatorService implements GroupCoordinator { .computeIfAbsent(topicPartitionFor(groupId), __ -> new ArrayList<>()) .add(groupId); } else { - futures.add(CompletableFuture.completedFuture(Collections.singletonList( + futures.add(CompletableFuture.completedFuture(List.of( new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId(null) .setErrorCode(Errors.INVALID_GROUP_ID.code()) @@ -866,7 +865,7 @@ public class GroupCoordinatorService implements GroupCoordinator { groupIds.forEach(groupId -> { // For backwards compatibility, we support DescribeGroups for the empty group id. if (groupId == null) { - futures.add(CompletableFuture.completedFuture(Collections.singletonList( + futures.add(CompletableFuture.completedFuture(List.of( new DescribeGroupsResponseData.DescribedGroup() .setGroupId(null) .setErrorCode(Errors.INVALID_GROUP_ID.code()) @@ -1149,7 +1148,7 @@ public class GroupCoordinatorService implements GroupCoordinator { topicPartitionFor(request.groupId()), Duration.ofMillis(config.offsetCommitTimeoutMs()), coordinator -> new CoordinatorResult<>( - Collections.emptyList(), + List.of(), coordinator.fetchOffsets(request, Long.MAX_VALUE) ) ).exceptionally(exception -> handleOffsetFetchException( @@ -1204,7 +1203,7 @@ public class GroupCoordinatorService implements GroupCoordinator { topicPartitionFor(request.groupId()), Duration.ofMillis(config.offsetCommitTimeoutMs()), coordinator -> new CoordinatorResult<>( - Collections.emptyList(), + List.of(), coordinator.fetchAllOffsets(request, Long.MAX_VALUE) ) ).exceptionally(exception -> handleOffsetFetchException( diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index e864def4f37..f71b47690d0 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -456,7 +456,7 @@ public class GroupMetadataManager { * Package private for testing. */ static final CoordinatorResult EMPTY_RESULT = - new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null), false); + new CoordinatorResult<>(List.of(), CompletableFuture.completedFuture(null), false); /** * The share group partition assignor. @@ -574,7 +574,7 @@ public class GroupMetadataManager { return groupStream .filter(combinedFilter) .map(group -> group.asListedGroup(committedOffset)) - .collect(Collectors.toList()); + .toList(); } /** @@ -701,7 +701,7 @@ public class GroupMetadataManager { .setProtocolData(group.protocolName().get()) .setMembers(group.allMembers().stream() .map(member -> member.describe(group.protocolName().get())) - .collect(Collectors.toList()) + .toList() ) ); } else { @@ -711,7 +711,7 @@ public class GroupMetadataManager { .setProtocolType(group.protocolType().orElse("")) .setMembers(group.allMembers().stream() .map(ClassicGroupMember::describeNoMetadata) - .collect(Collectors.toList()) + .toList() ) ); } @@ -1792,7 +1792,7 @@ public class GroupMetadataManager { .map(keyValue -> new ConsumerGroupHeartbeatResponseData.TopicPartitions() .setTopicId(keyValue.getKey()) .setPartitions(new ArrayList<>(keyValue.getValue()))) - .collect(Collectors.toList()); + .toList(); } private List fromShareGroupAssignmentMap( @@ -1802,7 +1802,7 @@ public class GroupMetadataManager { .map(keyValue -> new ShareGroupHeartbeatResponseData.TopicPartitions() .setTopicId(keyValue.getKey()) .setPartitions(new ArrayList<>(keyValue.getValue()))) - .collect(Collectors.toList()); + .toList(); } /** @@ -2038,7 +2038,7 @@ public class GroupMetadataManager { group, memberId, -1, - Collections.emptyList(), + List.of(), true, true ); @@ -2048,7 +2048,7 @@ public class GroupMetadataManager { memberId, -1, instanceId, - Collections.emptyList(), + List.of(), isUnknownMember, true, records @@ -2145,7 +2145,7 @@ public class GroupMetadataManager { if (downgrade) { convertToClassicGroup( group, - Collections.emptySet(), + Set.of(), updatedMember, records ); @@ -2727,7 +2727,7 @@ public class GroupMetadataManager { if (exception != null) { log.error("[GroupId {}] Couldn't update regular expression due to: {}", groupId, exception.getMessage()); - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } if (log.isDebugEnabled()) { @@ -3197,11 +3197,11 @@ public class GroupMetadataManager { // We will write a member epoch of -2 for this departing static member. ConsumerGroupMember leavingStaticMember = new ConsumerGroupMember.Builder(member) .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .build(); return new CoordinatorResult<>( - Collections.singletonList(newConsumerGroupCurrentAssignmentRecord(group.groupId(), leavingStaticMember)), + List.of(newConsumerGroupCurrentAssignmentRecord(group.groupId(), leavingStaticMember)), new ConsumerGroupHeartbeatResponseData() .setMemberId(member.memberId()) .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) @@ -3266,7 +3266,7 @@ public class GroupMetadataManager { ) { if (members.isEmpty()) { // No members to fence. Don't bump the group epoch. - return new CoordinatorResult<>(Collections.emptyList(), response); + return new CoordinatorResult<>(List.of(), response); } List records = new ArrayList<>(); @@ -3563,7 +3563,7 @@ public class GroupMetadataManager { groupId, memberId); } - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } /** @@ -3596,7 +3596,7 @@ public class GroupMetadataManager { groupId, memberId); } - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } /** @@ -3629,7 +3629,7 @@ public class GroupMetadataManager { groupId, memberId); } - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } /** @@ -3715,7 +3715,7 @@ public class GroupMetadataManager { } else { log.debug("[GroupId {}] Ignoring rebalance timeout for {} because the member " + "left the epoch {}.", groupId, memberId, memberEpoch); - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } } catch (GroupIdNotFoundException ex) { log.debug("[GroupId {}] Could not fence {}} because the group does not exist.", @@ -3725,7 +3725,7 @@ public class GroupMetadataManager { groupId, memberId); } - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); }); } @@ -3758,7 +3758,7 @@ public class GroupMetadataManager { } else { log.debug("[GroupId {}] Ignoring rebalance timeout for {} because the member " + "is not in epoch {} anymore.", groupId, memberId, memberEpoch); - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); } } catch (GroupIdNotFoundException ex) { log.debug("[GroupId {}] Could not fence {}} because the group does not exist.", @@ -3768,7 +3768,7 @@ public class GroupMetadataManager { groupId, memberId); } - return new CoordinatorResult<>(Collections.emptyList()); + return new CoordinatorResult<>(List.of()); }); } @@ -3935,7 +3935,7 @@ public class GroupMetadataManager { if (streamsGroup.topology().isPresent()) { oldSubscribedTopicNames = streamsGroup.topology().get().requiredTopics(); } else { - oldSubscribedTopicNames = Collections.emptySet(); + oldSubscribedTopicNames = Set.of(); } if (value != null) { StreamsTopology topology = StreamsTopology.fromRecord(value); @@ -3943,7 +3943,7 @@ public class GroupMetadataManager { Set newSubscribedTopicNames = topology.requiredTopics(); updateGroupsByTopics(groupId, oldSubscribedTopicNames, newSubscribedTopicNames); } else { - updateGroupsByTopics(groupId, oldSubscribedTopicNames, Collections.emptySet()); + updateGroupsByTopics(groupId, oldSubscribedTopicNames, Set.of()); streamsGroup.setTopology(null); } } @@ -4039,7 +4039,7 @@ public class GroupMetadataManager { */ public Set groupsSubscribedToTopic(String topicName) { Set groups = groupsByTopics.get(topicName); - return groups != null ? groups : Collections.emptySet(); + return groups != null ? groups : Set.of(); } /** @@ -4180,7 +4180,7 @@ public class GroupMetadataManager { }); group.setSubscriptionMetadata(subscriptionMetadata); } else { - group.setSubscriptionMetadata(Collections.emptyMap()); + group.setSubscriptionMetadata(Map.of()); } } @@ -4287,8 +4287,8 @@ public class GroupMetadataManager { ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember) .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setAssignedPartitions(Collections.emptyMap()) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) + .setPartitionsPendingRevocation(Map.of()) .build(); group.updateMember(newMember); } @@ -4395,7 +4395,7 @@ public class GroupMetadataManager { }); streamsGroup.setPartitionMetadata(partitionMetadata); } else { - streamsGroup.setPartitionMetadata(Collections.emptyMap()); + streamsGroup.setPartitionMetadata(Map.of()); } } @@ -4657,7 +4657,7 @@ public class GroupMetadataManager { ); group.setSubscriptionMetadata(subscriptionMetadata); } else { - group.setSubscriptionMetadata(Collections.emptyMap()); + group.setSubscriptionMetadata(Map.of()); } } @@ -4735,7 +4735,7 @@ public class GroupMetadataManager { ShareGroupMember newMember = new ShareGroupMember.Builder(oldMember) .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); group.updateMember(newMember); } @@ -5410,7 +5410,7 @@ public class GroupMetadataManager { // for the current generation. responseFuture.complete(new JoinGroupResponseData() .setMembers(group.isLeader(memberId) ? - group.currentClassicGroupMembers() : Collections.emptyList()) + group.currentClassicGroupMembers() : List.of()) .setMemberId(memberId) .setGenerationId(group.generationId()) .setProtocolName(group.protocolName().orElse(null)) @@ -5455,7 +5455,7 @@ public class GroupMetadataManager { // For followers with no actual change to their metadata, just return group information // for the current generation which will allow them to issue SyncGroup. responseFuture.complete(new JoinGroupResponseData() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setMemberId(memberId) .setGenerationId(group.generationId()) .setProtocolName(group.protocolName().orElse(null)) @@ -5562,8 +5562,8 @@ public class GroupMetadataManager { } }); - List records = Collections.singletonList(GroupCoordinatorRecordHelpers.newGroupMetadataRecord( - group, Collections.emptyMap())); + List records = List.of(GroupCoordinatorRecordHelpers.newGroupMetadataRecord( + group, Map.of())); return new CoordinatorResult<>(records, appendFuture, false); @@ -5573,7 +5573,7 @@ public class GroupMetadataManager { // Complete the awaiting join group response future for all the members after rebalancing group.allMembers().forEach(member -> { - List members = Collections.emptyList(); + List members = List.of(); if (group.isLeader(member.memberId())) { members = group.currentClassicGroupMembers(); } @@ -6204,7 +6204,7 @@ public class GroupMetadataManager { boolean isLeader = group.isLeader(newMemberId); group.completeJoinFuture(newMember, new JoinGroupResponseData() - .setMembers(isLeader ? group.currentClassicGroupMembers() : Collections.emptyList()) + .setMembers(isLeader ? group.currentClassicGroupMembers() : List.of()) .setMemberId(newMemberId) .setGenerationId(group.generationId()) .setProtocolName(group.protocolName().orElse(null)) @@ -6224,7 +6224,7 @@ public class GroupMetadataManager { } }); - List records = Collections.singletonList( + List records = List.of( GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, groupAssignment) ); @@ -6371,7 +6371,7 @@ public class GroupMetadataManager { } }); - List records = Collections.singletonList( + List records = List.of( GroupCoordinatorRecordHelpers.newGroupMetadataRecord(group, assignment) ); return new CoordinatorResult<>(records, appendFuture, false); @@ -6433,7 +6433,7 @@ public class GroupMetadataManager { } }); - return new CoordinatorResult<>(Collections.emptyList(), appendFuture, false); + return new CoordinatorResult<>(List.of(), appendFuture, false); } /** @@ -6586,14 +6586,14 @@ public class GroupMetadataManager { switch (group.currentState()) { case EMPTY: return new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) ); case PREPARING_REBALANCE: rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); return new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code()) ); @@ -6604,7 +6604,7 @@ public class GroupMetadataManager { // normal heartbeat requests and reset the timer rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId())); return new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new HeartbeatResponseData() ); @@ -6687,7 +6687,7 @@ public class GroupMetadataManager { } return new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new HeartbeatResponseData().setErrorCode(error.code()) ); } @@ -6827,7 +6827,7 @@ public class GroupMetadataManager { ) throws UnknownMemberIdException { if (group.isInState(DEAD)) { return new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new LeaveGroupResponseData() .setErrorCode(COORDINATOR_NOT_AVAILABLE.code()) ); @@ -6897,7 +6897,7 @@ public class GroupMetadataManager { List validLeaveGroupMembers = memberResponses.stream() .filter(response -> response.errorCode() == Errors.NONE.code()) .map(MemberResponse::memberId) - .collect(Collectors.toList()); + .toList(); String reason = "explicit `LeaveGroup` request for (" + String.join(", ", validLeaveGroupMembers) + ") members."; CoordinatorResult coordinatorResult = EMPTY_RESULT; diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 5e0cf7589e1..cd0eaa8bf61 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -55,7 +55,6 @@ import org.apache.kafka.timeline.TimelineHashSet; import org.slf4j.Logger; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -799,7 +798,7 @@ public class OffsetMetadataManager { } catch (GroupIdNotFoundException ex) { return new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId(request.groupId()) - .setTopics(Collections.emptyList()); + .setTopics(List.of()); } final List topicResponses = new ArrayList<>(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java index 3765b53dd43..1c5fe7e6eec 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java @@ -28,7 +28,6 @@ import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType; import org.apache.kafka.coordinator.group.modern.MemberAssignmentImpl; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -306,7 +305,7 @@ public class RangeAssignor implements ConsumerGroupPartitionAssignor { SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { if (groupSpec.memberIds().isEmpty()) { - return new GroupAssignment(Collections.emptyMap()); + return new GroupAssignment(Map.of()); } else if (groupSpec.subscriptionType() == SubscriptionType.HOMOGENEOUS) { return assignHomogeneousGroup(groupSpec, subscribedTopicDescriber); } else { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java index 781b64dd604..28c89966936 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java @@ -29,7 +29,6 @@ import org.apache.kafka.server.common.TopicIdPartition; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -227,7 +226,7 @@ public class SimpleAssignor implements ShareGroupPartitionAssignor { // When combining current assignment, we need to only consider the member topic subscription in current assignment // which is being subscribed in the new assignment as well. currentAssignment.forEach((topicIdPartition, members) -> members.forEach(member -> { - if (topicToMemberSubscription.getOrDefault(topicIdPartition.topicId(), Collections.emptySet()).contains(member) + if (topicToMemberSubscription.getOrDefault(topicIdPartition.topicId(), Set.of()).contains(member) && !newAssignment.containsKey(topicIdPartition)) finalAssignment.computeIfAbsent(member, k -> new HashSet<>()).add(topicIdPartition); })); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java index 0f4aa454404..d5595860298 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java @@ -25,7 +25,7 @@ import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; +import java.util.Map; import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HOMOGENEOUS; @@ -73,7 +73,7 @@ public class UniformAssignor implements ConsumerGroupPartitionAssignor { SubscribedTopicDescriber subscribedTopicDescriber ) throws PartitionAssignorException { if (groupSpec.memberIds().isEmpty()) - return new GroupAssignment(Collections.emptyMap()); + return new GroupAssignment(Map.of()); if (groupSpec.subscriptionType().equals(HOMOGENEOUS)) { LOG.debug("Detected that all members are subscribed to the same set of topics, invoking the " diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilder.java index 3166d775c27..ed7a9e21043 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilder.java @@ -27,7 +27,6 @@ import org.apache.kafka.coordinator.group.modern.MemberAssignmentImpl; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -238,7 +237,7 @@ public class UniformHeterogeneousAssignmentBuilder { */ public GroupAssignment build() { if (subscribedTopicIds.isEmpty()) { - return new GroupAssignment(Collections.emptyMap()); + return new GroupAssignment(Map.of()); } maybeRevokePartitions(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java index 8e7fe209c82..75e831edd86 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformHomogeneousAssignmentBuilder.java @@ -26,7 +26,6 @@ import org.apache.kafka.coordinator.group.modern.MemberAssignmentImpl; import org.apache.kafka.server.common.TopicIdPartition; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -109,7 +108,7 @@ public class UniformHomogeneousAssignmentBuilder { */ public GroupAssignment build() throws PartitionAssignorException { if (subscribedTopicIds.isEmpty()) { - return new GroupAssignment(Collections.emptyMap()); + return new GroupAssignment(Map.of()); } // Compute the list of unassigned partitions. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java index b35f3920532..cf51cd6db43 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java @@ -46,7 +46,6 @@ import org.slf4j.Logger; import java.nio.ByteBuffer; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -551,7 +550,7 @@ public class ClassicGroup implements Group { // Fence potential duplicate member immediately if someone awaits join/sync future. JoinGroupResponseData joinGroupResponse = new JoinGroupResponseData() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setMemberId(oldMemberId) .setProtocolName(null) .setProtocolType(null) @@ -1150,7 +1149,7 @@ public class ClassicGroup implements Group { return Optional.empty(); } if (members.isEmpty()) { - return Optional.of(Collections.emptySet()); + return Optional.of(Set.of()); } if (protocolName.isPresent()) { @@ -1310,7 +1309,7 @@ public class ClassicGroup implements Group { .setMemberId(member.memberId()) .setGroupInstanceId(member.groupInstanceId().orElse(null)) .setMetadata(member.metadata(protocolName.orElse(null)))) - .collect(Collectors.toList()); + .toList(); } /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java index 5d20e64f8d1..7d409b1a719 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java @@ -28,7 +28,6 @@ import java.util.HashSet; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; /** * This class encapsulates a classic group member's metadata. @@ -430,7 +429,7 @@ public class ClassicGroupMember { ", protocolType='" + protocolType + '\'' + ", supportedProtocols=" + supportedProtocols.stream() .map(JoinGroupRequestProtocol::name) - .collect(Collectors.toList()) + + .toList() + ')'; } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java index ade18585650..dd21570b654 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java @@ -126,49 +126,49 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The total number of groups using the classic rebalance protocol.", - Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CLASSIC.toString()) + Map.of(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CLASSIC.toString()) ); consumerGroupCountMetricName = metrics.metricName( GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The total number of groups using the consumer rebalance protocol.", - Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CONSUMER.toString()) + Map.of(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CONSUMER.toString()) ); consumerGroupCountEmptyMetricName = metrics.metricName( CONSUMER_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of consumer groups in empty state.", - Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.EMPTY.toString()) + Map.of(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.EMPTY.toString()) ); consumerGroupCountAssigningMetricName = metrics.metricName( CONSUMER_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of consumer groups in assigning state.", - Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.ASSIGNING.toString()) + Map.of(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.ASSIGNING.toString()) ); consumerGroupCountReconcilingMetricName = metrics.metricName( CONSUMER_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of consumer groups in reconciling state.", - Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.RECONCILING.toString()) + Map.of(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.RECONCILING.toString()) ); consumerGroupCountStableMetricName = metrics.metricName( CONSUMER_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of consumer groups in stable state.", - Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.STABLE.toString()) + Map.of(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.STABLE.toString()) ); consumerGroupCountDeadMetricName = metrics.metricName( CONSUMER_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of consumer groups in dead state.", - Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.DEAD.toString()) + Map.of(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.DEAD.toString()) ); shareGroupCountMetricName = metrics.metricName( @@ -203,49 +203,49 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The total number of groups using the streams rebalance protocol.", - Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.STREAMS.toString()) + Map.of(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.STREAMS.toString()) ); streamsGroupCountEmptyMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in empty state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.EMPTY.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.EMPTY.toString()) ); streamsGroupCountAssigningMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in assigning state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.ASSIGNING.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.ASSIGNING.toString()) ); streamsGroupCountReconcilingMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in reconciling state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.RECONCILING.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.RECONCILING.toString()) ); streamsGroupCountStableMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in stable state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.STABLE.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.STABLE.toString()) ); streamsGroupCountDeadMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in dead state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.DEAD.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.DEAD.toString()) ); streamsGroupCountNotReadyMetricName = metrics.metricName( STREAMS_GROUP_COUNT_METRIC_NAME, METRICS_GROUP, "The number of streams groups in not ready state.", - Collections.singletonMap(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.NOT_READY.toString()) + Map.of(STREAMS_GROUP_COUNT_STATE_TAG, StreamsGroupState.NOT_READY.toString()) ); registerGauges(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java index 8725eee133e..a163e30d553 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java @@ -29,7 +29,6 @@ import org.apache.kafka.timeline.TimelineLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; @@ -112,10 +111,10 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard { numOffsetsTimelineGaugeCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0)); numClassicGroupsTimelineCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0)); - this.classicGroupGauges = Collections.emptyMap(); - this.consumerGroupGauges = Collections.emptyMap(); - this.streamsGroupGauges = Collections.emptyMap(); - this.shareGroupGauges = Collections.emptyMap(); + this.classicGroupGauges = Map.of(); + this.consumerGroupGauges = Map.of(); + this.streamsGroupGauges = Map.of(); + this.shareGroupGauges = Map.of(); this.globalSensors = Objects.requireNonNull(globalSensors); this.topicPartition = Objects.requireNonNull(topicPartition); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/Assignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/Assignment.java index d210b73a20f..9338f63a7de 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/Assignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/Assignment.java @@ -32,7 +32,7 @@ import java.util.stream.Collectors; * An immutable assignment for a member. */ public class Assignment implements MemberAssignment { - public static final Assignment EMPTY = new Assignment(Collections.emptyMap()); + public static final Assignment EMPTY = new Assignment(Map.of()); /** * The partitions assigned to the member. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/GroupSpecImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/GroupSpecImpl.java index ad3226b5f03..5e7356bcfe6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/GroupSpecImpl.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/GroupSpecImpl.java @@ -23,7 +23,6 @@ import org.apache.kafka.coordinator.group.api.assignor.MemberSubscription; import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType; import java.util.Collection; -import java.util.Collections; import java.util.Map; import java.util.Objects; @@ -104,7 +103,7 @@ public class GroupSpecImpl implements GroupSpec { public MemberAssignment memberAssignment(String memberId) { MemberSubscriptionAndAssignmentImpl member = members.get(memberId); if (member == null) { - return new MemberAssignmentImpl(Collections.emptyMap()); + return new MemberAssignmentImpl(Map.of()); } return member; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java index cf291e0306b..0f2b5530e10 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java @@ -270,7 +270,7 @@ public abstract class ModernGroup implements Group public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) { updateInvertedTargetAssignment( memberId, - targetAssignment.getOrDefault(memberId, new Assignment(Collections.emptyMap())), + targetAssignment.getOrDefault(memberId, new Assignment(Map.of())), newTargetAssignment ); targetAssignment.put(memberId, newTargetAssignment); @@ -294,8 +294,8 @@ public abstract class ModernGroup implements Group allTopicIds.addAll(newTargetAssignment.partitions().keySet()); for (Uuid topicId : allTopicIds) { - Set oldPartitions = oldTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet()); - Set newPartitions = newTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet()); + Set oldPartitions = oldTargetAssignment.partitions().getOrDefault(topicId, Set.of()); + Set newPartitions = newTargetAssignment.partitions().getOrDefault(topicId, Set.of()); TimelineHashMap topicPartitionAssignment = invertedTargetAssignment.computeIfAbsent( topicId, k -> new TimelineHashMap<>(snapshotRegistry, Math.max(oldPartitions.size(), newPartitions.size())) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java index 7871b04d722..b0a7f5aad77 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -72,7 +71,7 @@ public class SubscribedTopicDescriberImpl implements SubscribedTopicDescriber { */ @Override public Set racksForPartition(Uuid topicId, int partition) { - return Collections.emptySet(); + return Set.of(); } @Override diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java index 63bf81f1e08..715cefa2dd8 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java @@ -97,7 +97,7 @@ public abstract class TargetAssignmentBuilder resolvedRegularExpressions = Collections.emptyMap(); + private Map resolvedRegularExpressions = Map.of(); public ConsumerTargetAssignmentBuilder( String groupId, @@ -248,12 +248,12 @@ public abstract class TargetAssignmentBuilder members = Collections.emptyMap(); + private Map members = Map.of(); /** * The subscription metadata. */ - private Map subscriptionMetadata = Collections.emptyMap(); + private Map subscriptionMetadata = Map.of(); /** * The subscription type of the consumer group. @@ -263,13 +263,13 @@ public abstract class TargetAssignmentBuilder targetAssignment = Collections.emptyMap(); + private Map targetAssignment = Map.of(); /** * Reverse lookup map representing topic partitions with * their current member assignments. */ - private Map> invertedTargetAssignment = Collections.emptyMap(); + private Map> invertedTargetAssignment = Map.of(); /** * The topics image. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java index 16e0d0fae4a..ef95b123ff9 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java @@ -1151,7 +1151,7 @@ public class ConsumerGroup extends ModernGroup { // We should accept the empty assignment. Map> assignedPartitions; if (Arrays.equals(classicGroupMember.assignment(), EMPTY_ASSIGNMENT)) { - assignedPartitions = Collections.emptyMap(); + assignedPartitions = Map.of(); } else { ConsumerProtocolAssignment assignment = ConsumerProtocol.deserializeConsumerProtocolAssignment( ByteBuffer.wrap(classicGroupMember.assignment()) @@ -1287,7 +1287,7 @@ public class ConsumerGroup extends ModernGroup { if (member.state() == MemberState.UNRELEASED_PARTITIONS) { for (Map.Entry> entry : targetAssignment().get(member.memberId()).partitions().entrySet()) { Uuid topicId = entry.getKey(); - Set assignedPartitions = member.assignedPartitions().getOrDefault(topicId, Collections.emptySet()); + Set assignedPartitions = member.assignedPartitions().getOrDefault(topicId, Set.of()); for (int partition : entry.getValue()) { if (!assignedPartitions.contains(partition) && currentPartitionEpoch(topicId, partition) != -1) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java index c96dd277adb..4cc4895ff5c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java @@ -29,7 +29,6 @@ import org.apache.kafka.image.TopicImage; import org.apache.kafka.image.TopicsImage; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -62,11 +61,11 @@ public class ConsumerGroupMember extends ModernGroupMember { private int rebalanceTimeoutMs = -1; private String clientId = ""; private String clientHost = ""; - private Set subscribedTopicNames = Collections.emptySet(); + private Set subscribedTopicNames = Set.of(); private String subscribedTopicRegex = ""; private String serverAssignorName = null; - private Map> assignedPartitions = Collections.emptyMap(); - private Map> partitionsPendingRevocation = Collections.emptyMap(); + private Map> assignedPartitions = Map.of(); + private Map> partitionsPendingRevocation = Map.of(); private ConsumerGroupMemberMetadataValue.ClassicMemberMetadata classicMemberMetadata = null; public Builder(String memberId) { @@ -400,7 +399,7 @@ public class ConsumerGroupMember extends ModernGroupMember { .setTopicPartitions(topicPartitionsFromMap(assignedPartitions, topicsImage))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() .setTopicPartitions(topicPartitionsFromMap( - targetAssignment != null ? targetAssignment.partitions() : Collections.emptyMap(), + targetAssignment != null ? targetAssignment.partitions() : Map.of(), topicsImage ))) .setClientHost(clientHost) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilder.java index 3a7ea6692b0..74a5bd7a2e3 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilder.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.modern.Assignment; import org.apache.kafka.coordinator.group.modern.MemberState; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -204,7 +203,7 @@ public class CurrentAssignmentBuilder { for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) { Set partitionsPendingRevocation = - assignment.getOrDefault(topicPartitions.topicId(), Collections.emptySet()); + assignment.getOrDefault(topicPartitions.topicId(), Set.of()); for (Integer partitionId : topicPartitions.partitions()) { if (partitionsPendingRevocation.contains(partitionId)) { @@ -238,9 +237,9 @@ public class CurrentAssignmentBuilder { for (Uuid topicId : allTopicIds) { Set target = targetAssignment.partitions() - .getOrDefault(topicId, Collections.emptySet()); + .getOrDefault(topicId, Set.of()); Set currentAssignedPartitions = memberAssignedPartitions - .getOrDefault(topicId, Collections.emptySet()); + .getOrDefault(topicId, Set.of()); // New Assigned Partitions = Previous Assigned Partitions ∩ Target Set assignedPartitions = new HashSet<>(currentAssignedPartitions); @@ -295,7 +294,7 @@ public class CurrentAssignmentBuilder { .setState(newState) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .build(); } else if (hasUnreleasedPartitions) { // If there are no partitions to be revoked nor to be assigned but some @@ -305,7 +304,7 @@ public class CurrentAssignmentBuilder { .setState(MemberState.UNRELEASED_PARTITIONS) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .build(); } else { // Otherwise, the member transitions to the target epoch and to the @@ -314,7 +313,7 @@ public class CurrentAssignmentBuilder { .setState(MemberState.STABLE) .updateMemberEpoch(targetAssignmentEpoch) .setAssignedPartitions(newAssignedPartitions) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .build(); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java index 7cef5602dd6..2da953e4585 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java @@ -24,7 +24,7 @@ import java.util.Set; * The metadata associated with a regular expression in a Consumer Group. */ public class ResolvedRegularExpression { - public static final ResolvedRegularExpression EMPTY = new ResolvedRegularExpression(Collections.emptySet(), -1L, -1L); + public static final ResolvedRegularExpression EMPTY = new ResolvedRegularExpression(Set.of(), -1L, -1L); /** * The set of resolved topics. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java index 6bb3a818982..e8d5b118b32 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java @@ -27,7 +27,6 @@ import org.apache.kafka.image.TopicImage; import org.apache.kafka.image.TopicsImage; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -56,8 +55,8 @@ public class ShareGroupMember extends ModernGroupMember { private String rackId = null; private String clientId = ""; private String clientHost = ""; - private Set subscribedTopicNames = Collections.emptySet(); - private Map> assignedPartitions = Collections.emptyMap(); + private Set subscribedTopicNames = Set.of(); + private Map> assignedPartitions = Map.of(); public Builder(String memberId) { this.memberId = Objects.requireNonNull(memberId); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java index 3c9ba064a40..22f33eb8260 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java @@ -18,7 +18,6 @@ package org.apache.kafka.coordinator.group.streams; import org.apache.kafka.common.errors.FencedMemberEpochException; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -251,8 +250,8 @@ public class CurrentAssignmentBuilder { for (String subtopologyId : allSubtopologyIds) { hasUnreleasedTasks |= computeAssignmentDifferenceForOneSubtopology( subtopologyId, - currentAssignment.getOrDefault(subtopologyId, Collections.emptySet()), - targetAssignment.getOrDefault(subtopologyId, Collections.emptySet()), + currentAssignment.getOrDefault(subtopologyId, Set.of()), + targetAssignment.getOrDefault(subtopologyId, Set.of()), resultAssignedTasks, resultTasksPendingRevocation, resultTasksPendingAssignment, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java index 881f930d8a5..eb1ef06ceac 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; /** * This class contains helper methods to create records stored in the __consumer_offsets topic. @@ -72,7 +71,7 @@ public class StreamsCoordinatorRecordHelpers { new StreamsGroupMemberMetadataValue.KeyValue() .setKey(e.getKey()) .setValue(e.getValue()) - ).sorted(Comparator.comparing(StreamsGroupMemberMetadataValue.KeyValue::key)).collect(Collectors.toList())), + ).sorted(Comparator.comparing(StreamsGroupMemberMetadataValue.KeyValue::key)).toList()), (short) 0 ) ); @@ -401,12 +400,12 @@ public class StreamsCoordinatorRecordHelpers { List repartitionSourceTopics = subtopology.repartitionSourceTopics().stream() .map(StreamsCoordinatorRecordHelpers::convertToTopicInfo) - .collect(Collectors.toList()); + .toList(); List stateChangelogTopics = subtopology.stateChangelogTopics().stream() .map(StreamsCoordinatorRecordHelpers::convertToTopicInfo) - .collect(Collectors.toList()); + .toList(); List copartitionGroups = subtopology.copartitionGroups().stream() @@ -415,7 +414,7 @@ public class StreamsCoordinatorRecordHelpers { .setSourceTopicRegex(copartitionGroup.sourceTopicRegex()) .setRepartitionSourceTopics(copartitionGroup.repartitionSourceTopics()) ) - .collect(Collectors.toList()); + .toList(); value.subtopologies().add( new StreamsGroupTopologyValue.Subtopology() @@ -434,7 +433,7 @@ public class StreamsCoordinatorRecordHelpers { private static StreamsGroupTopologyValue.TopicInfo convertToTopicInfo(StreamsGroupHeartbeatRequestData.TopicInfo topicInfo) { List topicConfigs = topicInfo.topicConfigs() != null ? topicInfo.topicConfigs().stream() .map(config -> new StreamsGroupTopologyValue.TopicConfig().setKey(config.key()).setValue(config.value())) - .collect(Collectors.toList()) : null; + .toList() : null; return new StreamsGroupTopologyValue.TopicInfo() .setName(topicInfo.name()) .setTopicConfigs(topicConfigs) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index 85b36867ea6..951b96c9925 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -511,9 +511,9 @@ public class StreamsGroup implements Group { ) { Map> tasks = currentStandbyTaskToProcessIds.get(subtopologyId); if (tasks == null) { - return Collections.emptySet(); + return Set.of(); } else { - return tasks.getOrDefault(taskId, Collections.emptySet()); + return tasks.getOrDefault(taskId, Set.of()); } } @@ -529,9 +529,9 @@ public class StreamsGroup implements Group { ) { Map> tasks = currentWarmupTaskToProcessIds.get(subtopologyId); if (tasks == null) { - return Collections.emptySet(); + return Set.of(); } else { - return tasks.getOrDefault(taskId, Collections.emptySet()); + return tasks.getOrDefault(taskId, Set.of()); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java index 69d2eae8831..a3862abb5ae 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java @@ -341,7 +341,7 @@ public record StreamsGroupMember(String memberId, entry -> new StreamsGroupDescribeResponseData.KeyValue() .setKey(entry.getKey()) .setValue(entry.getValue()) - ).collect(Collectors.toList())) + ).toList()) .setProcessId(processId) .setTopologyEpoch(topologyEpoch) .setUserEndpoint( diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java index ea4c6f81a52..85c22384311 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java @@ -50,9 +50,9 @@ public record TasksTuple(Map> activeTasks, * An empty task tuple. */ public static final TasksTuple EMPTY = new TasksTuple( - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java index 22f908b825c..cf873992371 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.coordinator.group.streams.assignor; -import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -37,6 +36,6 @@ public record MemberAssignment(Map> activeTasks, } public static MemberAssignment empty() { - return new MemberAssignment(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); + return new MemberAssignment(Map.of(), Map.of(), Map.of()); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopic.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopic.java index 855f1ea0b58..deba335bec4 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopic.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopic.java @@ -23,7 +23,6 @@ import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.stream.Collectors; /** * Captures the properties required for configuring the internal topics we create for changelogs and repartitioning etc. @@ -57,12 +56,11 @@ public record ConfiguredInternalTopic(String name, .setPartitions(numberOfPartitions) .setReplicationFactor(replicationFactor.orElse((short) 0)) .setTopicConfigs( - topicConfigs != null ? - topicConfigs.entrySet().stream().map( - y -> new StreamsGroupDescribeResponseData.KeyValue() - .setKey(y.getKey()) - .setValue(y.getValue()) - ).collect(Collectors.toList()) : null + topicConfigs.entrySet().stream().map( + y -> new StreamsGroupDescribeResponseData.KeyValue() + .setKey(y.getKey()) + .setValue(y.getValue()) + ).toList() ); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopology.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopology.java index bfc1a86a06b..b8586776ebe 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopology.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopology.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; /** * Internal representation of a subtopology. @@ -51,12 +50,12 @@ public record ConfiguredSubtopology(Set sourceTopics, public StreamsGroupDescribeResponseData.Subtopology asStreamsGroupDescribeSubtopology(String subtopologyId) { return new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId(subtopologyId) - .setSourceTopics(sourceTopics.stream().sorted().collect(Collectors.toList())) - .setRepartitionSinkTopics(repartitionSinkTopics.stream().sorted().collect(Collectors.toList())) + .setSourceTopics(sourceTopics.stream().sorted().toList()) + .setRepartitionSinkTopics(repartitionSinkTopics.stream().sorted().toList()) .setRepartitionSourceTopics(repartitionSourceTopics.values().stream() - .map(ConfiguredInternalTopic::asStreamsGroupDescribeTopicInfo).sorted().collect(Collectors.toList())) + .map(ConfiguredInternalTopic::asStreamsGroupDescribeTopicInfo).sorted().toList()) .setStateChangelogTopics(stateChangelogTopics.values().stream() - .map(ConfiguredInternalTopic::asStreamsGroupDescribeTopicInfo).sorted().collect(Collectors.toList())); + .map(ConfiguredInternalTopic::asStreamsGroupDescribeTopicInfo).sorted().toList()); } } \ No newline at end of file diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopology.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopology.java index 6c7eede16fc..93e03050b4d 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopology.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopology.java @@ -19,12 +19,11 @@ package org.apache.kafka.coordinator.group.streams.topics; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; -import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.SortedMap; -import java.util.stream.Collectors; /** * This class captures the result of taking a topology definition sent by the client and using the current state of the topics inside the @@ -68,7 +67,7 @@ public record ConfiguredTopology(int topologyEpoch, .setSubtopologies( subtopologies.map(stringConfiguredSubtopologyMap -> stringConfiguredSubtopologyMap.entrySet().stream().map( entry -> entry.getValue().asStreamsGroupDescribeSubtopology(entry.getKey()) - ).collect(Collectors.toList())).orElse(Collections.emptyList()) + ).toList()).orElse(List.of()) ); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcer.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcer.java index b7928931dd1..d7955126a5c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcer.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcer.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; @@ -74,7 +73,7 @@ public class CopartitionedTopicsEnforcer { final Set flexibleRepartitionTopics) throws StreamsInvalidTopologyException { if (copartitionedTopics.isEmpty()) { log.debug("Ignoring unexpected empty copartitioned topics set."); - return Collections.emptyMap(); + return Map.of(); } final Map returnedPartitionCounts = new HashMap<>(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java index 33d9f2c4874..71f31244d00 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java @@ -27,7 +27,6 @@ import org.apache.kafka.coordinator.group.streams.TopicMetadata; import org.slf4j.Logger; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -290,7 +289,7 @@ public class InternalTopicManager { topicInfo.topicConfigs() != null ? topicInfo.topicConfigs().stream() .collect(Collectors.toMap(StreamsGroupTopologyValue.TopicConfig::key, StreamsGroupTopologyValue.TopicConfig::value)) - : Collections.emptyMap() + : Map.of() ); } @@ -304,6 +303,6 @@ public class InternalTopicManager { copartitionGroup.repartitionSourceTopics().stream() .map(i -> subtopology.repartitionSourceTopics().get(i).name()) ).collect(Collectors.toSet()) - ).collect(Collectors.toList()); + ).toList(); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java index 14ac93f0a4b..227d07a8def 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/Assertions.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.function.BiConsumer; import java.util.function.Consumer; -import java.util.stream.Collectors; import static org.junit.jupiter.api.AssertionFailureBuilder.assertionFailure; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -115,12 +114,12 @@ public class Assertions { slice .stream() .sorted(Comparator.comparing(Object::toString)) - .collect(Collectors.toList()), + .toList(), actualRecords .subList(j, j + slice.size()) .stream() .sorted(Comparator.comparing(Object::toString)) - .collect(Collectors.toList()) + .toList() ); j += slice.size(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index 66f23a7c01d..8b92b258bbb 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -34,7 +34,6 @@ import org.apache.kafka.coordinator.group.assignor.UniformAssignor; import org.junit.jupiter.api.Test; import java.time.Duration; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -144,7 +143,7 @@ public class GroupCoordinatorConfigTest { configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG, 555); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG, 200); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG, 55); - configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, Collections.singletonList(RangeAssignor.class)); + configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(RangeAssignor.class)); configs.put(GroupCoordinatorConfig.OFFSETS_TOPIC_SEGMENT_BYTES_CONFIG, 2222); configs.put(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG, 3333); configs.put(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, 60); @@ -243,7 +242,7 @@ public class GroupCoordinatorConfigTest { assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); configs.clear(); - configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, Collections.singletonList(Object.class)); + configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(Object.class)); assertEquals("class java.lang.Object is not an instance of org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor", assertThrows(KafkaException.class, () -> createConfig(configs)).getMessage()); @@ -297,7 +296,7 @@ public class GroupCoordinatorConfigTest { configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG, 5); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, 5); configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG, Integer.MAX_VALUE); - configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, Collections.singletonList(RangeAssignor.class)); + configs.put(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(RangeAssignor.class)); configs.put(GroupCoordinatorConfig.OFFSETS_TOPIC_SEGMENT_BYTES_CONFIG, 1000); configs.put(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG, offsetMetadataMaxSize); configs.put(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, Integer.MAX_VALUE); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java index e2aec513432..7d69ff62e8f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java @@ -55,7 +55,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -570,7 +569,7 @@ public class GroupCoordinatorRecordHelpersTest { assertThrows(IllegalStateException.class, () -> GroupCoordinatorRecordHelpers.newGroupMetadataRecord( group, - Collections.emptyMap() + Map.of() )); } @@ -620,7 +619,7 @@ public class GroupCoordinatorRecordHelpersTest { assertThrows(IllegalStateException.class, () -> GroupCoordinatorRecordHelpers.newGroupMetadataRecord( group, - Collections.emptyMap() + Map.of() )); } @@ -628,7 +627,7 @@ public class GroupCoordinatorRecordHelpersTest { public void testEmptyGroupMetadataRecord() { Time time = new MockTime(); - List expectedMembers = Collections.emptyList(); + List expectedMembers = List.of(); CoordinatorRecord expectedRecord = CoordinatorRecord.record( new GroupMetadataKey() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 4d3e79e395b..a62eb3afc61 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -113,7 +113,6 @@ import org.mockito.ArgumentMatchers; import java.net.InetAddress; import java.time.Duration; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.OptionalInt; @@ -289,7 +288,7 @@ public class GroupCoordinatorServiceTest { assertEquals( new StreamsGroupHeartbeatResult( new StreamsGroupHeartbeatResponseData().setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()), - Collections.emptyMap() + Map.of() ), future.get() ); @@ -314,7 +313,7 @@ public class GroupCoordinatorServiceTest { )).thenReturn(CompletableFuture.completedFuture( new StreamsGroupHeartbeatResult( new StreamsGroupHeartbeatResponseData(), - Collections.emptyMap() + Map.of() ) )); @@ -323,7 +322,7 @@ public class GroupCoordinatorServiceTest { request ); - assertEquals(new StreamsGroupHeartbeatResult(new StreamsGroupHeartbeatResponseData(), Collections.emptyMap()), future.get(5, TimeUnit.SECONDS)); + assertEquals(new StreamsGroupHeartbeatResult(new StreamsGroupHeartbeatResponseData(), Map.of()), future.get(5, TimeUnit.SECONDS)); } private static Stream testStreamsGroupHeartbeatWithExceptionSource() { @@ -376,7 +375,7 @@ public class GroupCoordinatorServiceTest { new StreamsGroupHeartbeatResponseData() .setErrorCode(expectedErrorCode) .setErrorMessage(expectedErrorMessage), - Collections.emptyMap() + Map.of() ), future.get(5, TimeUnit.SECONDS) ); @@ -880,9 +879,9 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("list-groups"), ArgumentMatchers.any() )).thenReturn(Arrays.asList( - CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), - CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), - CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(2))) + CompletableFuture.completedFuture(List.of(expectedResults.get(0))), + CompletableFuture.completedFuture(List.of(expectedResults.get(1))), + CompletableFuture.completedFuture(List.of(expectedResults.get(2))) )); CompletableFuture responseFuture = service.listGroups( @@ -920,8 +919,8 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("list-groups"), ArgumentMatchers.any() )).thenReturn(Arrays.asList( - CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(0))), - CompletableFuture.completedFuture(Collections.singletonList(expectedResults.get(1))), + CompletableFuture.completedFuture(List.of(expectedResults.get(0))), + CompletableFuture.completedFuture(List.of(expectedResults.get(1))), FutureUtils.failedFuture(new NotCoordinatorException("")) )); @@ -947,8 +946,8 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("list-groups"), ArgumentMatchers.any() )).thenReturn(Arrays.asList( - CompletableFuture.completedFuture(Collections.emptyList()), - CompletableFuture.completedFuture(Collections.emptyList()), + CompletableFuture.completedFuture(List.of()), + CompletableFuture.completedFuture(List.of()), FutureUtils.failedFuture(new CoordinatorLoadInProgressException("")) )); @@ -1033,7 +1032,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("describe-groups"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); CompletableFuture describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( @@ -1046,7 +1045,7 @@ public class GroupCoordinatorServiceTest { service.describeGroups(requestContext(ApiKeys.DESCRIBE_GROUPS), Arrays.asList("group-id-1", "group-id-2")); assertFalse(future.isDone()); - describedGroupFuture.complete(Collections.singletonList(describedGroup2)); + describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); } @@ -1074,7 +1073,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("describe-groups"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup))); CompletableFuture> future = service.describeGroups(requestContext(ApiKeys.DESCRIBE_GROUPS), Arrays.asList("", null)); @@ -1102,10 +1101,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.describeGroups(requestContext(ApiKeys.DESCRIBE_GROUPS), Collections.singletonList("group-id")); + service.describeGroups(requestContext(ApiKeys.DESCRIBE_GROUPS), List.of("group-id")); assertEquals( - Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup() + List.of(new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) ), @@ -1123,11 +1122,11 @@ public class GroupCoordinatorServiceTest { CompletableFuture> future = service.describeGroups( requestContext(ApiKeys.DESCRIBE_GROUPS), - Collections.singletonList("group-id") + List.of("group-id") ); assertEquals( - Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup() + List.of(new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ), @@ -1157,17 +1156,17 @@ public class GroupCoordinatorServiceTest { .setGroupId("group"); if (!fetchAllOffsets) { request - .setTopics(Collections.singletonList(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)))); + .setPartitionIndexes(List.of(0)))); } OffsetFetchResponseData.OffsetFetchResponseGroup response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group") - .setTopics(Collections.singletonList(new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setTopics(List.of(new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(Collections.singletonList(new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitions(List.of(new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100L))))); @@ -1219,9 +1218,9 @@ public class GroupCoordinatorServiceTest { .setGroupId("group"); if (!fetchAllOffsets) { request - .setTopics(Collections.singletonList(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)))); + .setPartitionIndexes(List.of(0)))); } TriFunction> fetchOffsets = @@ -1269,9 +1268,9 @@ public class GroupCoordinatorServiceTest { .setGroupId("group"); if (!fetchAllOffsets) { request - .setTopics(Collections.singletonList(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)))); + .setPartitionIndexes(List.of(0)))); } when(runtime.scheduleWriteOperation( @@ -1422,7 +1421,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); CompletableFuture describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( @@ -1435,7 +1434,7 @@ public class GroupCoordinatorServiceTest { service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), Arrays.asList("group-id-1", "group-id-2")); assertFalse(future.isDone()); - describedGroupFuture.complete(Collections.singletonList(describedGroup2)); + describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); } @@ -1463,7 +1462,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup))); CompletableFuture> future = service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), Arrays.asList("", null)); @@ -1490,10 +1489,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), Collections.singletonList("group-id")); + service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), List.of("group-id")); assertEquals( - Collections.singletonList(new ConsumerGroupDescribeResponseData.DescribedGroup() + List.of(new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) ), @@ -1517,10 +1516,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), Collections.singletonList("group-id")); + service.consumerGroupDescribe(requestContext(ApiKeys.CONSUMER_GROUP_DESCRIBE), List.of("group-id")); assertEquals( - Collections.singletonList(new ConsumerGroupDescribeResponseData.DescribedGroup() + List.of(new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ), @@ -1551,7 +1550,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("streams-group-describe"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); CompletableFuture describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( @@ -1564,7 +1563,7 @@ public class GroupCoordinatorServiceTest { service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), Arrays.asList("group-id-1", "group-id-2")); assertFalse(future.isDone()); - describedGroupFuture.complete(Collections.singletonList(describedGroup2)); + describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); } @@ -1592,7 +1591,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("streams-group-describe"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup))); CompletableFuture> future = service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), Arrays.asList("", null)); @@ -1619,10 +1618,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), Collections.singletonList("group-id")); + service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), List.of("group-id")); assertEquals( - Collections.singletonList(new StreamsGroupDescribeResponseData.DescribedGroup() + List.of(new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) ), @@ -1646,10 +1645,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), Collections.singletonList("group-id")); + service.streamsGroupDescribe(requestContext(ApiKeys.STREAMS_GROUP_DESCRIBE), List.of("group-id")); assertEquals( - Collections.singletonList(new StreamsGroupDescribeResponseData.DescribedGroup() + List.of(new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ), @@ -1667,10 +1666,10 @@ public class GroupCoordinatorServiceTest { .build(true); OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection requestTopicCollection = - new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestPartition().setPartitionIndex(0) )) ).iterator()); @@ -1679,11 +1678,11 @@ public class GroupCoordinatorServiceTest { .setTopics(requestTopicCollection); OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection responsePartitionCollection = - new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(Collections.singletonList( + new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(List.of( new OffsetDeleteResponseData.OffsetDeleteResponsePartition().setPartitionIndex(0) ).iterator()); OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection responseTopicCollection = - new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(Collections.singletonList( + new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(List.of( new OffsetDeleteResponseData.OffsetDeleteResponseTopic().setPartitions(responsePartitionCollection) ).iterator()); OffsetDeleteResponseData response = new OffsetDeleteResponseData() @@ -1716,10 +1715,10 @@ public class GroupCoordinatorServiceTest { .build(true); OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection requestTopicCollection = - new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestPartition().setPartitionIndex(0) )) ).iterator()); @@ -1760,10 +1759,10 @@ public class GroupCoordinatorServiceTest { .build(true); OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection requestTopicCollection = - new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestPartition().setPartitionIndex(0) )) ).iterator()); @@ -2272,12 +2271,12 @@ public class GroupCoordinatorServiceTest { CompletableFuture future = service.deleteGroups( requestContext(ApiKeys.DELETE_GROUPS), - Collections.singletonList("group-id"), + List.of("group-id"), BufferSupplier.NO_CACHING ); assertEquals( - new DeleteGroupsResponseData.DeletableGroupResultCollection(Collections.singletonList( + new DeleteGroupsResponseData.DeletableGroupResultCollection(List.of( new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-id") .setErrorCode(expectedErrorCode) @@ -2297,13 +2296,13 @@ public class GroupCoordinatorServiceTest { CompletableFuture future = service.deleteGroups( requestContext(ApiKeys.DELETE_GROUPS), - Collections.singletonList("foo"), + List.of("foo"), BufferSupplier.NO_CACHING ); assertEquals( new DeleteGroupsResponseData.DeletableGroupResultCollection( - Collections.singletonList(new DeleteGroupsResponseData.DeletableGroupResult() + List.of(new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("foo") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ).iterator() @@ -2325,9 +2324,9 @@ public class GroupCoordinatorServiceTest { .setTransactionalId("transactional-id") .setMemberId("member-id") .setGenerationId(10) - .setTopics(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() + .setTopics(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() + .setPartitions(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100))))); @@ -2339,9 +2338,9 @@ public class GroupCoordinatorServiceTest { assertEquals( new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() + .setTopics(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() + .setPartitions(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()))))), future.get() @@ -2363,9 +2362,9 @@ public class GroupCoordinatorServiceTest { .setTransactionalId("transactional-id") .setMemberId("member-id") .setGenerationId(10) - .setTopics(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() + .setTopics(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() + .setPartitions(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100))))); @@ -2377,9 +2376,9 @@ public class GroupCoordinatorServiceTest { assertEquals( new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() + .setTopics(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() + .setPartitions(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.INVALID_GROUP_ID.code()))))), future.get() @@ -2402,16 +2401,16 @@ public class GroupCoordinatorServiceTest { .setProducerEpoch((short) 5) .setMemberId("member-id") .setGenerationId(10) - .setTopics(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() + .setTopics(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() + .setPartitions(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100))))); TxnOffsetCommitResponseData response = new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() + .setTopics(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() + .setPartitions(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()))))); @@ -2457,16 +2456,16 @@ public class GroupCoordinatorServiceTest { .setProducerEpoch((short) 5) .setMemberId("member-id") .setGenerationId(10) - .setTopics(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() + .setTopics(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() + .setPartitions(List.of(new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100))))); TxnOffsetCommitResponseData response = new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() + .setTopics(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName(TOPIC_NAME) - .setPartitions(Collections.singletonList(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() + .setPartitions(List.of(new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))); @@ -2582,7 +2581,7 @@ public class GroupCoordinatorServiceTest { // The exception is logged and swallowed. assertDoesNotThrow(() -> service.onPartitionsDeleted( - Collections.singletonList(new TopicPartition("foo", 0)), + List.of(new TopicPartition("foo", 0)), BufferSupplier.NO_CACHING ) ); @@ -2597,7 +2596,7 @@ public class GroupCoordinatorServiceTest { .build(); assertThrows(CoordinatorNotAvailableException.class, () -> service.onPartitionsDeleted( - Collections.singletonList(new TopicPartition("foo", 0)), + List.of(new TopicPartition("foo", 0)), BufferSupplier.NO_CACHING )); } @@ -2708,7 +2707,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup1))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); CompletableFuture describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( @@ -2721,7 +2720,7 @@ public class GroupCoordinatorServiceTest { service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), Arrays.asList("share-group-id-1", "share-group-id-2")); assertFalse(future.isDone()); - describedGroupFuture.complete(Collections.singletonList(describedGroup2)); + describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); } @@ -2749,7 +2748,7 @@ public class GroupCoordinatorServiceTest { ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), ArgumentMatchers.any() - )).thenReturn(CompletableFuture.completedFuture(Collections.singletonList(describedGroup))); + )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup))); CompletableFuture> future = service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), Arrays.asList("", null)); @@ -2776,10 +2775,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), Collections.singletonList("share-group-id")); + service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), List.of("share-group-id")); assertEquals( - Collections.singletonList(new ShareGroupDescribeResponseData.DescribedGroup() + List.of(new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId("share-group-id") .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) ), @@ -2803,10 +2802,10 @@ public class GroupCoordinatorServiceTest { )); CompletableFuture> future = - service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), Collections.singletonList("share-group-id")); + service.shareGroupDescribe(requestContext(ApiKeys.SHARE_GROUP_DESCRIBE), List.of("share-group-id")); assertEquals( - Collections.singletonList(new ShareGroupDescribeResponseData.DescribedGroup() + List.of(new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId("share-group-id") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) ), diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java index 0aa48202104..05db80b8cba 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java @@ -102,7 +102,6 @@ import org.mockito.Mockito; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -151,7 +150,7 @@ public class GroupCoordinatorShardTest { RequestContext context = requestContext(ApiKeys.CONSUMER_GROUP_HEARTBEAT); ConsumerGroupHeartbeatRequestData request = new ConsumerGroupHeartbeatRequestData(); CoordinatorResult result = new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new ConsumerGroupHeartbeatResponseData() ); @@ -183,8 +182,8 @@ public class GroupCoordinatorShardTest { RequestContext context = requestContext(ApiKeys.STREAMS_GROUP_HEARTBEAT); StreamsGroupHeartbeatRequestData request = new StreamsGroupHeartbeatRequestData(); CoordinatorResult result = new CoordinatorResult<>( - Collections.emptyList(), - new StreamsGroupHeartbeatResult(new StreamsGroupHeartbeatResponseData(), Collections.emptyMap()) + List.of(), + new StreamsGroupHeartbeatResult(new StreamsGroupHeartbeatResponseData(), Map.of()) ); when(groupMetadataManager.streamsGroupHeartbeat( @@ -215,7 +214,7 @@ public class GroupCoordinatorShardTest { RequestContext context = requestContext(ApiKeys.OFFSET_COMMIT); OffsetCommitRequestData request = new OffsetCommitRequestData(); CoordinatorResult result = new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new OffsetCommitResponseData() ); @@ -247,7 +246,7 @@ public class GroupCoordinatorShardTest { RequestContext context = requestContext(ApiKeys.TXN_OFFSET_COMMIT); TxnOffsetCommitRequestData request = new TxnOffsetCommitRequestData(); CoordinatorResult result = new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new TxnOffsetCommitResponseData() ); @@ -1440,7 +1439,7 @@ public class GroupCoordinatorShardTest { records.add(offsetCommitTombstone); return true; }); - when(offsetMetadataManager.cleanupExpiredOffsets("other-group-id", Collections.emptyList())).thenReturn(false); + when(offsetMetadataManager.cleanupExpiredOffsets("other-group-id", List.of())).thenReturn(false); doAnswer(invocation -> { List records = recordsCapture.getValue(); records.add(groupMetadataTombstone); @@ -1551,18 +1550,18 @@ public class GroupCoordinatorShardTest { metricsShard ); - List records = Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord( + List records = List.of(GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord( "group", "foo", 0 )); when(offsetMetadataManager.onPartitionsDeleted( - Collections.singletonList(new TopicPartition("foo", 0)) + List.of(new TopicPartition("foo", 0)) )).thenReturn(records); CoordinatorResult result = coordinator.onPartitionsDeleted( - Collections.singletonList(new TopicPartition("foo", 0)) + List.of(new TopicPartition("foo", 0)) ); assertEquals(records, result.records()); @@ -1611,7 +1610,7 @@ public class GroupCoordinatorShardTest { RequestContext context = requestContext(ApiKeys.SHARE_GROUP_HEARTBEAT); ShareGroupHeartbeatRequestData request = new ShareGroupHeartbeatRequestData(); CoordinatorResult result = new CoordinatorResult<>( - Collections.emptyList(), + List.of(), new ShareGroupHeartbeatResponseData() ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 8ec5ec2eb4e..4933e505067 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -125,7 +125,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -241,7 +240,7 @@ public class GroupMetadataManagerTest { .setGroupId("foo") .setMemberEpoch(0) .setRebalanceTimeoutMs(5000) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); assertEquals("Either SubscribedTopicNames or SubscribedTopicRegex must be non-null when (re-)joining.", ex.getMessage()); // InstanceId must be non-empty if provided in all requests. @@ -278,7 +277,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); assertEquals("InstanceId can't be null.", ex.getMessage()); } @@ -287,7 +286,7 @@ public class GroupMetadataManagerTest { public void testConsumerHeartbeatRegexValidation() { String memberId = Uuid.randomUuid().toString(); MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .build(); @@ -301,7 +300,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("[") - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); assertEquals("SubscribedTopicRegex `[` is not a valid regular expression: missing closing ].", ex.getMessage()); // Subscribing with a valid regular expression succeeds. @@ -312,7 +311,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex(".*") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(1, result.response().memberEpoch()); // Updating the subscription to an invalid regular expression fails. @@ -323,7 +322,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(1) .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("[") - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); assertEquals("SubscribedTopicRegex `[` is not a valid regular expression: missing closing ].", ex.getMessage()); // Updating the subscription to topic names succeeds (checking when the regex becomes null). @@ -334,7 +333,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(1) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(2, result.response().memberEpoch()); } @@ -347,7 +346,7 @@ public class GroupMetadataManagerTest { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.emptyMap() + Map.of() )); CoordinatorResult result = context.consumerGroupHeartbeat( @@ -359,7 +358,7 @@ public class GroupMetadataManagerTest { .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()), + .setTopicPartitions(List.of()), (short) 0 ); @@ -400,7 +399,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(100) // Epoch must be > 0. .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -422,7 +421,7 @@ public class GroupMetadataManagerTest { .setServerAssignor(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); // The second member is rejected because the member id is unknown and // the member epoch is not zero. @@ -434,7 +433,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(1) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -558,7 +557,7 @@ public class GroupMetadataManagerTest { .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -788,7 +787,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -903,7 +902,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -1012,7 +1011,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -1135,7 +1134,7 @@ public class GroupMetadataManagerTest { .setInstanceId(memberId2) .setMemberEpoch(-2) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); // Member epoch of the response would be set to -2. assertResponseEquals( @@ -1163,7 +1162,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -1341,7 +1340,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of("foo", "bar")) // bar is new. - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -1493,7 +1492,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); // member epoch of the response would be set to -2 assertResponseEquals( @@ -1583,7 +1582,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -1654,7 +1653,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -1703,7 +1702,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -1749,7 +1748,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(11) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -1882,7 +1881,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -1928,7 +1927,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -1965,7 +1964,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -2005,7 +2004,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -2030,7 +2029,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo.*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -2139,7 +2138,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -2329,7 +2328,7 @@ public class GroupMetadataManagerTest { result.response() ); - assertEquals(Collections.emptyList(), result.records()); + assertEquals(List.of(), result.records()); assertEquals(MemberState.UNREVOKED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId2)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); @@ -2382,7 +2381,7 @@ public class GroupMetadataManagerTest { result.response() ); - assertEquals(Collections.emptyList(), result.records()); + assertEquals(List.of(), result.records()); assertEquals(MemberState.UNRELEASED_PARTITIONS, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); @@ -2544,7 +2543,7 @@ public class GroupMetadataManagerTest { .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -2630,7 +2629,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -2873,9 +2872,9 @@ public class GroupMetadataManagerTest { .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .build(); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("bar")); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("bar")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); // M1 in group 1 subscribes to foo and bar. context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId1, @@ -2885,7 +2884,7 @@ public class GroupMetadataManagerTest { assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("foo")); assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar")); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); // M1 in group 2 subscribes to foo, bar and zar. context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2, @@ -2928,7 +2927,7 @@ public class GroupMetadataManagerTest { // M1 in group 2 subscribes to nothing. context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2, new ConsumerGroupMember.Builder("group2-m1") - .setSubscribedTopicNames(Collections.emptyList()) + .setSubscribedTopicNames(List.of()) .build())); assertEquals(Set.of(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo")); @@ -2948,22 +2947,22 @@ public class GroupMetadataManagerTest { // M2 in group 2 subscribes to nothing. context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2, new ConsumerGroupMember.Builder("group2-m2") - .setSubscribedTopicNames(Collections.emptyList()) + .setSubscribedTopicNames(List.of()) .build())); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar")); assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar")); // M2 in group 1 subscribes to nothing. context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId1, new ConsumerGroupMember.Builder("group1-m2") - .setSubscribedTopicNames(Collections.emptyList()) + .setSubscribedTopicNames(List.of()) .build())); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("bar")); - assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("foo")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("bar")); + assertEquals(Set.of(), context.groupMetadataManager.groupsSubscribedToTopic("zar")); } @Test @@ -3099,7 +3098,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(1, result.response().memberEpoch()); // Verify that there is a session time. @@ -3107,7 +3106,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3124,7 +3123,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3174,7 +3173,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(1, result.response().memberEpoch()); // Verify that there is a session time. @@ -3192,7 +3191,7 @@ public class GroupMetadataManagerTest { GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2) ) ) @@ -3239,7 +3238,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(1, result.response().memberEpoch()); // Verify that there is a session time. @@ -3254,7 +3253,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(-2, result.response().memberEpoch()); @@ -3273,7 +3272,7 @@ public class GroupMetadataManagerTest { GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2) ) ) @@ -3318,7 +3317,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(180000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -3334,7 +3333,7 @@ public class GroupMetadataManagerTest { ); assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3356,7 +3355,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -3368,7 +3367,7 @@ public class GroupMetadataManagerTest { ); assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3401,7 +3400,7 @@ public class GroupMetadataManagerTest { context.assertRebalanceTimeout(groupId, memberId1, 12000); assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3428,7 +3427,7 @@ public class GroupMetadataManagerTest { // Execute the scheduled revocation timeout captured earlier to simulate a // stale timeout. This should be a no-op. - assertEquals(Collections.emptyList(), scheduledTimeout.operation.generateRecords().records()); + assertEquals(List.of(), scheduledTimeout.operation.generateRecords().records()); } @Test @@ -3463,7 +3462,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(10000) // Use timeout smaller than session timeout. .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -3479,7 +3478,7 @@ public class GroupMetadataManagerTest { ); assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3501,7 +3500,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(10000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -3513,7 +3512,7 @@ public class GroupMetadataManagerTest { ); assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -3729,9 +3728,9 @@ public class GroupMetadataManagerTest { group6.transitionTo(STABLE); group7.transitionTo(DEAD); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupIds.get(1), false, Collections.emptyList()) + context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupIds.get(1), false, List.of()) .removeMember(consumerMemberIds.get(0)); - context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupIds.get(3), false, Collections.emptyList()) + context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupIds.get(3), false, List.of()) .updateMember(new ConsumerGroupMember.Builder(consumerMemberIds.get(2)).setMemberEpoch(10).build()); context.groupMetadataManager.updateGroupSizeCounter(); @@ -4068,7 +4067,7 @@ public class GroupMetadataManagerTest { List firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendClassicGroupJoin( request, requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); List memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED); assertEquals(groupMaxSize + 1, memberIds.size()); @@ -4081,7 +4080,7 @@ public class GroupMetadataManagerTest { List secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin( request.setMemberId(memberId), requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); // Advance clock by group initial rebalance delay to complete first inital delayed join. // This will extend the initial rebalance as new members have joined. @@ -4100,7 +4099,7 @@ public class GroupMetadataManagerTest { List thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin( request.setMemberId(memberId), requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); verifyClassicGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED); } @@ -4126,7 +4125,7 @@ public class GroupMetadataManagerTest { List firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendClassicGroupJoin( request, requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); assertEquals(groupMaxSize, group.numMembers()); assertEquals(groupMaxSize, group.numAwaitingJoinResponse()); @@ -4146,7 +4145,7 @@ public class GroupMetadataManagerTest { List secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin( request.setMemberId(memberId), requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); verifyClassicGroupJoinResponses(secondRoundJoinResults, 10, Errors.GROUP_MAX_SIZE_REACHED); assertEquals(groupMaxSize, group.numMembers()); @@ -4160,7 +4159,7 @@ public class GroupMetadataManagerTest { List groupInstanceIds = IntStream.range(0, groupMaxSize + 1) .mapToObj(i -> "instance-id-" + i) - .collect(Collectors.toList()); + .toList(); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConfig(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, groupMaxSize) @@ -4178,7 +4177,7 @@ public class GroupMetadataManagerTest { // First round of join requests. This will trigger a rebalance. List firstRoundJoinResults = groupInstanceIds.stream() .map(instanceId -> context.sendClassicGroupJoin(request.setGroupInstanceId(instanceId))) - .collect(Collectors.toList()); + .toList(); assertEquals(groupMaxSize, group.numMembers()); assertEquals(groupMaxSize, group.numAwaitingJoinResponse()); @@ -4200,7 +4199,7 @@ public class GroupMetadataManagerTest { request .setMemberId(memberIds.get(i)) .setGroupInstanceId(groupInstanceIds.get(i)) - )).collect(Collectors.toList()); + )).toList(); verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED); assertEquals(groupMaxSize, group.numMembers()); @@ -4228,7 +4227,7 @@ public class GroupMetadataManagerTest { // First round of join requests. Generate member ids. List firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1) .mapToObj(__ -> context.sendClassicGroupJoin(request, requiredKnownMemberId)) - .collect(Collectors.toList()); + .toList(); assertEquals(0, group.numMembers()); assertEquals(groupMaxSize + 1, group.numPendingJoinMembers()); @@ -4255,7 +4254,7 @@ public class GroupMetadataManagerTest { List thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin( request.setMemberId(memberId), requiredKnownMemberId - )).collect(Collectors.toList()); + )).toList(); // Advance clock by group initial rebalance delay to complete first inital delayed join. // This will extend the initial rebalance as new members have joined. @@ -4285,7 +4284,7 @@ public class GroupMetadataManagerTest { List memberIds = IntStream.range(0, groupMaxSize + 2) .mapToObj(i -> group.generateMemberId("client-id", Optional.empty())) - .collect(Collectors.toList()); + .toList(); memberIds.forEach(memberId -> group.add( new ClassicGroupMember( @@ -4309,7 +4308,7 @@ public class GroupMetadataManagerTest { .withDefaultProtocolTypeAndProtocols() .withRebalanceTimeoutMs(10000) .build() - )).collect(Collectors.toList()); + )).toList(); assertEquals(groupMaxSize, group.numMembers()); assertEquals(groupMaxSize, group.numAwaitingJoinResponse()); @@ -5396,7 +5395,7 @@ public class GroupMetadataManagerTest { String newMemberId = group.staticMemberId("group-instance-id"); JoinGroupResponseData expectedResponse = new JoinGroupResponseData() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setLeader(oldMemberId) .setMemberId(newMemberId) .setGenerationId(1) @@ -5521,7 +5520,7 @@ public class GroupMetadataManagerTest { assertTrue(joinResult.joinFuture.isDone()); JoinGroupResponseData expectedResponse = new JoinGroupResponseData() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setLeader(oldMemberId) .setMemberId(UNKNOWN_MEMBER_ID) .setGenerationId(1) @@ -5597,7 +5596,7 @@ public class GroupMetadataManagerTest { assertTrue(joinResult.joinFuture.isDone()); JoinGroupResponseData expectedResponse = new JoinGroupResponseData() - .setMembers(supportSkippingAssignment ? toJoinResponseMembers(group) : Collections.emptyList()) + .setMembers(supportSkippingAssignment ? toJoinResponseMembers(group) : List.of()) .setLeader(supportSkippingAssignment ? joinResult.joinFuture.get().memberId() : oldMemberId) .setMemberId(joinResult.joinFuture.get().memberId()) .setGenerationId(1) @@ -5719,7 +5718,7 @@ public class GroupMetadataManagerTest { List expectedRecords = List.of(GroupMetadataManagerTestContext.newGroupMetadataRecord( group.groupId(), new GroupMetadataValue() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setGeneration(2) .setLeader(null) .setProtocolType("consumer") @@ -5850,7 +5849,7 @@ public class GroupMetadataManagerTest { oldFollowerJoinResult.joinFuture.get(), group, PREPARING_REBALANCE, - Collections.emptySet() + Set.of() ); } @@ -5928,7 +5927,7 @@ public class GroupMetadataManagerTest { oldFollowerJoinResult.joinFuture.get(), group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); assertTrue(group.isInState(COMPLETING_REBALANCE)); @@ -6048,14 +6047,14 @@ public class GroupMetadataManagerTest { .setLeader(rebalanceResult.leaderId) .setProtocolName("range") .setProtocolType("consumer") - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedDuplicateFollowerResponse, duplicateFollowerJoinResult.joinFuture.get(), group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); assertTrue(duplicateFollowerJoinResult.joinFuture.isDone()); @@ -6067,14 +6066,14 @@ public class GroupMetadataManagerTest { .setLeader(UNKNOWN_MEMBER_ID) .setProtocolName(null) .setProtocolType(null) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedOldFollowerResponse, oldFollowerJoinResult.joinFuture.get(), group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); } @@ -6165,7 +6164,7 @@ public class GroupMetadataManagerTest { joinResult.joinFuture.get().memberId() : rebalanceResult.leaderId; List members = supportSkippingAssignment ? - toJoinResponseMembers(group) : Collections.emptyList(); + toJoinResponseMembers(group) : List.of(); JoinGroupResponseData expectedJoinResponse = new JoinGroupResponseData() .setErrorCode(Errors.NONE.code()) @@ -6182,7 +6181,7 @@ public class GroupMetadataManagerTest { joinResult.joinFuture.get(), group, STABLE, - supportSkippingAssignment ? Set.of("leader-instance-id", "follower-instance-id") : Collections.emptySet() + supportSkippingAssignment ? Set.of("leader-instance-id", "follower-instance-id") : Set.of() ); GroupMetadataManagerTestContext.JoinResult oldLeaderJoinResult = context.sendClassicGroupJoin( @@ -6482,14 +6481,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedResponse, followerJoinResult.joinFuture.get(), group, STABLE, - Collections.emptySet() + Set.of() ); // Join with old member id will not fail because the member id is not updated because of persistence failure @@ -6590,14 +6589,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedResponse, followerJoinResult.joinFuture.get(), group, STABLE, - Collections.emptySet() + Set.of() ); // Join with old member id will fail because the member id is updated @@ -6614,7 +6613,7 @@ public class GroupMetadataManagerTest { .withGroupInstanceId("follower-instance-id") .withGenerationId(rebalanceResult.generationId) .withMemberId(rebalanceResult.followerId) - .withAssignment(Collections.emptyList()) + .withAssignment(List.of()) .build(); GroupMetadataManagerTestContext.SyncResult syncResult = context.sendClassicGroupSync(syncRequest); @@ -6706,14 +6705,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedFollowerResponse, followerJoinResult.joinFuture.get(), group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); // The follower protocol changed from protocolSuperset to general protocols. @@ -6805,14 +6804,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedFollowerResponse, followerJoinResult.joinFuture.get(), group, STABLE, - Collections.emptySet() + Set.of() ); assertNotEquals(rebalanceResult.followerId, followerJoinResult.joinFuture.get().memberId()); @@ -6872,14 +6871,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedFollowerResponse, followerJoinResult.joinFuture.get(), group, STABLE, - Collections.emptySet() + Set.of() ); } @@ -7148,14 +7147,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedNewMemberResponse, newMemberJoinResult.joinFuture.get(), group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); } @@ -7244,14 +7243,14 @@ public class GroupMetadataManagerTest { .setProtocolName("range") .setProtocolType("consumer") .setSkipAssignment(false) - .setMembers(Collections.emptyList()); + .setMembers(List.of()); checkJoinGroupResponse( expectedNewMemberResponse, newFollowerResponse, group, COMPLETING_REBALANCE, - Collections.emptySet() + Set.of() ); } @@ -8530,7 +8529,7 @@ public class GroupMetadataManagerTest { assertTrue(syncResult.records.isEmpty()); assertFalse(syncResult.syncFuture.isDone()); return syncResult.syncFuture; - }).collect(Collectors.toList()); + }).toList(); // Advance clock by 1/2 rebalance timeout to expire the pending sync. Leader should be kicked out. List> timeouts = context.sleep(rebalanceTimeoutMs / 2); @@ -8600,7 +8599,7 @@ public class GroupMetadataManagerTest { } assertTrue(syncResult.syncFuture.isDone()); return syncResult.syncFuture; - }).collect(Collectors.toList()); + }).toList(); for (CompletableFuture syncFuture : syncFutures) { assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); @@ -8679,7 +8678,7 @@ public class GroupMetadataManagerTest { context.replay(GroupMetadataManagerTestContext.newGroupMetadataRecord( classicGroupId, new GroupMetadataValue() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setGeneration(2) .setLeader(null) .setProtocolType("classic") @@ -8696,7 +8695,7 @@ public class GroupMetadataManagerTest { // Test list group response without a group state or group type filter. Map actualAllGroupMap = - context.sendListGroups(Collections.emptyList(), Collections.emptyList()).stream() + context.sendListGroups(List.of(), List.of()).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); Map expectAllGroupMap = @@ -8722,7 +8721,7 @@ public class GroupMetadataManagerTest { // List group with case-insensitive ‘empty’. actualAllGroupMap = - context.sendListGroups(List.of("empty"), Collections.emptyList()) + context.sendListGroups(List.of("empty"), List.of()) .stream().collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); assertEquals(expectAllGroupMap, actualAllGroupMap); @@ -8730,7 +8729,7 @@ public class GroupMetadataManagerTest { context.commit(); // Test list group response to check assigning state in the consumer group. - actualAllGroupMap = context.sendListGroups(List.of("assigning"), Collections.emptyList()).stream() + actualAllGroupMap = context.sendListGroups(List.of("assigning"), List.of()).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( @@ -8744,7 +8743,7 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); // Test list group response with group state filter and no group type filter. - actualAllGroupMap = context.sendListGroups(List.of("Empty"), Collections.emptyList()).stream() + actualAllGroupMap = context.sendListGroups(List.of("Empty"), List.of()).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( new ListGroupsResponseData.ListedGroup() @@ -8762,7 +8761,7 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); // Test list group response with no group state filter and with group type filter. - actualAllGroupMap = context.sendListGroups(Collections.emptyList(), List.of(Group.GroupType.CLASSIC.toString())).stream() + actualAllGroupMap = context.sendListGroups(List.of(), List.of(Group.GroupType.CLASSIC.toString())).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( new ListGroupsResponseData.ListedGroup() @@ -8775,7 +8774,7 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); // Test list group response with no group state filter and with group type filter in a different case. - actualAllGroupMap = context.sendListGroups(Collections.emptyList(), List.of("Consumer")).stream() + actualAllGroupMap = context.sendListGroups(List.of(), List.of("Consumer")).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( new ListGroupsResponseData.ListedGroup() @@ -8787,7 +8786,7 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); - actualAllGroupMap = context.sendListGroups(Collections.emptyList(), List.of("Share")).stream() + actualAllGroupMap = context.sendListGroups(List.of(), List.of("Share")).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( new ListGroupsResponseData.ListedGroup() @@ -8799,7 +8798,7 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); - actualAllGroupMap = context.sendListGroups(List.of("empty", "Assigning"), Collections.emptyList()).stream() + actualAllGroupMap = context.sendListGroups(List.of("empty", "Assigning"), List.of()).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); expectAllGroupMap = Stream.of( new ListGroupsResponseData.ListedGroup() @@ -8822,16 +8821,16 @@ public class GroupMetadataManagerTest { assertEquals(expectAllGroupMap, actualAllGroupMap); // Test list group response with no group state filter and with invalid group type filter . - actualAllGroupMap = context.sendListGroups(Collections.emptyList(), List.of("Invalid")).stream() + actualAllGroupMap = context.sendListGroups(List.of(), List.of("Invalid")).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); - expectAllGroupMap = Collections.emptyMap(); + expectAllGroupMap = Map.of(); assertEquals(expectAllGroupMap, actualAllGroupMap); // Test list group response with invalid group state filter and with no group type filter . - actualAllGroupMap = context.sendListGroups(List.of("Invalid"), Collections.emptyList()).stream() + actualAllGroupMap = context.sendListGroups(List.of("Invalid"), List.of()).stream() .collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity())); - expectAllGroupMap = Collections.emptyMap(); + expectAllGroupMap = Map.of(); assertEquals(expectAllGroupMap, actualAllGroupMap); } @@ -8865,7 +8864,7 @@ public class GroupMetadataManagerTest { .setGroupId(consumerGroupIds.get(1)) .setMembers(List.of( memberBuilder.build().asConsumerGroupDescribeMember( - new Assignment(Collections.emptyMap()), + new Assignment(Map.of()), new MetadataImageBuilder().build().topics() ) )) @@ -8921,7 +8920,7 @@ public class GroupMetadataManagerTest { context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(consumerGroupId, memberBuilder1.build())); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, epoch + 1)); - Map> assignmentMap = Map.of(topicId, Collections.emptySet()); + Map> assignmentMap = Map.of(topicId, Set.of()); ConsumerGroupMember.Builder memberBuilder2 = new ConsumerGroupMember.Builder(memberId2); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(consumerGroupId, memberBuilder2.build())); @@ -8946,7 +8945,7 @@ public class GroupMetadataManagerTest { describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId(consumerGroupId) .setMembers(List.of( - memberBuilder1.build().asConsumerGroupDescribeMember(new Assignment(Collections.emptyMap()), metadataImage.topics()), + memberBuilder1.build().asConsumerGroupDescribeMember(new Assignment(Map.of()), metadataImage.topics()), memberBuilder2.build().asConsumerGroupDescribeMember(new Assignment(assignmentMap), metadataImage.topics()) )) .setGroupState(ConsumerGroup.ConsumerGroupState.ASSIGNING.toString()) @@ -8964,7 +8963,7 @@ public class GroupMetadataManagerTest { int epoch = 10; String memberId = "member-id"; StreamsGroupMember.Builder memberBuilder = streamsGroupMemberBuilderWithDefaults(memberId) - .setClientTags(Collections.singletonMap("clientTag", "clientValue")) + .setClientTags(Map.of("clientTag", "clientValue")) .setProcessId("processId") .setMemberEpoch(epoch) .setPreviousMemberEpoch(epoch - 1); @@ -8984,7 +8983,7 @@ public class GroupMetadataManagerTest { new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupEpoch(epoch) .setGroupId(streamsGroupIds.get(1)) - .setMembers(Collections.singletonList( + .setMembers(List.of( memberBuilder.build().asStreamsGroupDescribeMember( TasksTuple.EMPTY ) @@ -9001,14 +9000,12 @@ public class GroupMetadataManagerTest { String groupId = "groupId"; GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder().build(); - List actual = context.sendStreamsGroupDescribe(Collections.singletonList(groupId)); + List actual = context.sendStreamsGroupDescribe(List.of(groupId)); StreamsGroupDescribeResponseData.DescribedGroup describedGroup = new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupId) .setErrorCode(Errors.GROUP_ID_NOT_FOUND.code()) .setErrorMessage("Group groupId not found."); - List expected = Collections.singletonList( - describedGroup - ); + List expected = List.of(describedGroup); assertEquals(expected, actual); } @@ -9040,7 +9037,7 @@ public class GroupMetadataManagerTest { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(streamsGroupId, memberBuilder2.build())); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 2)); - List actual = context.groupMetadataManager.streamsGroupDescribe(Collections.singletonList(streamsGroupId), context.lastCommittedOffset); + List actual = context.groupMetadataManager.streamsGroupDescribe(List.of(streamsGroupId), context.lastCommittedOffset); StreamsGroupDescribeResponseData.DescribedGroup describedGroup = new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(streamsGroupId) .setErrorCode(Errors.GROUP_ID_NOT_FOUND.code()) @@ -9051,7 +9048,7 @@ public class GroupMetadataManagerTest { // Commit the offset and test again context.commit(); - actual = context.groupMetadataManager.streamsGroupDescribe(Collections.singletonList(streamsGroupId), context.lastCommittedOffset); + actual = context.groupMetadataManager.streamsGroupDescribe(List.of(streamsGroupId), context.lastCommittedOffset); describedGroup = new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(streamsGroupId) .setMembers(Arrays.asList( @@ -9246,7 +9243,7 @@ public class GroupMetadataManagerTest { context.sleep(rebalanceTimeoutMs); // Only static leader is maintained, and group is stuck at PreparingRebalance stage assertTrue(group.allDynamicMemberIds().isEmpty()); - assertEquals(Collections.singleton(rebalanceResult.leaderId), group.allMemberIds()); + assertEquals(Set.of(rebalanceResult.leaderId), group.allMemberIds()); assertTrue(group.allDynamicMemberIds().isEmpty()); assertEquals(2, group.generationId()); assertTrue(group.isInState(PREPARING_REBALANCE)); @@ -9748,11 +9745,11 @@ public class GroupMetadataManagerTest { records = new ArrayList<>(); group.transitionTo(PREPARING_REBALANCE); context.groupMetadataManager.maybeDeleteGroup("group-id", records); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); records = new ArrayList<>(); context.groupMetadataManager.maybeDeleteGroup("invalid-group-id", records); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); } @Test @@ -9794,7 +9791,7 @@ public class GroupMetadataManagerTest { .setPreviousMemberEpoch(10) .build())); context.groupMetadataManager.maybeDeleteGroup(groupId, records); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); } @Test @@ -9841,7 +9838,7 @@ public class GroupMetadataManagerTest { .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); verify(context.metrics).record(CONSUMER_GROUP_REBALANCES_SENSOR_NAME); } @@ -9871,7 +9868,7 @@ public class GroupMetadataManagerTest { .setServerAssignor(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -9897,7 +9894,7 @@ public class GroupMetadataManagerTest { .setServerAssignor(NoOpPartitionAssignor.NAME) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) .setState(MemberState.STABLE) @@ -9908,7 +9905,7 @@ public class GroupMetadataManagerTest { .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) .setSubscribedTopicNames(List.of("foo", "bar")) .setServerAssignorName(NoOpPartitionAssignor.NAME) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); assertEquals(Errors.NONE.code(), result.response().errorCode()); @@ -9917,7 +9914,7 @@ public class GroupMetadataManagerTest { GroupCoordinatorRecordHelpers.newGroupMetadataTombstoneRecord(classicGroupId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(classicGroupId, expectedMember), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(classicGroupId, 1), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(classicGroupId, memberId, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(classicGroupId, memberId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(classicGroupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(classicGroupId, expectedMember) ), @@ -10043,7 +10040,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(0) @@ -10071,7 +10068,7 @@ public class GroupMetadataManagerTest { .setServerAssignorName("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) .setRebalanceTimeoutMs(5000) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); List expectedRecords = List.of( @@ -10241,7 +10238,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(0) @@ -10284,7 +10281,7 @@ public class GroupMetadataManagerTest { .setServerAssignorName("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) .setRebalanceTimeoutMs(5000) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); List expectedRecords = List.of( @@ -10434,7 +10431,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) - .setTopicPartitions(Collections.emptyList()); + .setTopicPartitions(List.of()); if (expectUpgrade) { context.consumerGroupHeartbeat(consumerGroupHeartbeatRequestData); @@ -10516,7 +10513,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor(NoOpPartitionAssignor.NAME) .setSubscribedTopicNames(List.of(fooTopicName)) - .setTopicPartitions(Collections.emptyList()), + .setTopicPartitions(List.of()), ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() ); @@ -10661,7 +10658,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor(NoOpPartitionAssignor.NAME) .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(1) @@ -10676,7 +10673,7 @@ public class GroupMetadataManagerTest { .setSessionTimeoutMs(5000) .setSupportedProtocols(ConsumerGroupMember.classicProtocolListFromJoinRequestProtocolCollection(protocols)) ) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(memberId2) @@ -10688,7 +10685,7 @@ public class GroupMetadataManagerTest { .setServerAssignorName(NoOpPartitionAssignor.NAME) .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) .setRebalanceTimeoutMs(5000) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); List expectedRecords = List.of( @@ -10713,7 +10710,7 @@ public class GroupMetadataManagerTest { // Newly joining member 2 bumps the group epoch. A new target assignment is computed. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 2), // Member 2 has no pending revoking partition or pending release partition. @@ -10827,7 +10824,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor(NoOpPartitionAssignor.NAME) .setSubscribedTopicNames(new ArrayList<>(member1.subscribedTopicNames())) - .setTopicPartitions(Collections.emptyList()), + .setTopicPartitions(List.of()), ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() ); @@ -11028,7 +11025,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setServerAssignor("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); ConsumerGroupMember expectedMember1 = new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(1) @@ -11071,7 +11068,7 @@ public class GroupMetadataManagerTest { .setServerAssignorName("range") .setSubscribedTopicNames(List.of(fooTopicName, barTopicName)) .setRebalanceTimeoutMs(5000) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .build(); List expectedRecords = List.of( @@ -11155,12 +11152,12 @@ public class GroupMetadataManagerTest { assertTrue(joinResult2.joinFuture.isDone()); assertEquals(new JoinGroupResponseData() .setMemberId(joinResult1.joinFuture.get().memberId()) - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setErrorCode(NOT_COORDINATOR.code()), joinResult1.joinFuture.get()); assertEquals(new JoinGroupResponseData() .setMemberId(joinResult2.joinFuture.get().memberId()) - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setErrorCode(NOT_COORDINATOR.code()), joinResult2.joinFuture.get()); } @@ -11323,7 +11320,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); byte[] assignment = Utils.toArray(ConsumerProtocol.serializeAssignment(new ConsumerPartitionAssignor.Assignment(List.of( @@ -11501,7 +11498,7 @@ public class GroupMetadataManagerTest { .setMemberId(memberId2) .setMemberEpoch(10) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); // Verify that there is a session timeout. context.assertSessionTimeout(groupId, memberId2, 45000); @@ -12114,7 +12111,7 @@ public class GroupMetadataManagerTest { .withMemberId(UNKNOWN_MEMBER_ID) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName), - Collections.emptyList(), + List.of(), version)) .build(); @@ -12254,7 +12251,7 @@ public class GroupMetadataManagerTest { .withMemberId(newMemberId) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName), - Collections.emptyList())) + List.of())) .build(); GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request); @@ -12308,7 +12305,7 @@ public class GroupMetadataManagerTest { .withGroupInstanceId(instanceId) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName), - Collections.emptyList())) + List.of())) .build(); GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request); @@ -12342,7 +12339,7 @@ public class GroupMetadataManagerTest { )), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember) @@ -12404,7 +12401,7 @@ public class GroupMetadataManagerTest { .withGroupInstanceId(instanceId) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName), - Collections.emptyList())) + List.of())) .build(); // The static member joins with UNKNOWN_MEMBER_ID. @@ -12565,7 +12562,7 @@ public class GroupMetadataManagerTest { .withMemberId(memberId1) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName, zarTopicName), - Collections.emptyList())) + List.of())) .build(); GroupMetadataManagerTestContext.JoinResult joinResult = context.sendClassicGroupJoin(request); @@ -12587,7 +12584,7 @@ public class GroupMetadataManagerTest { .setSupportedProtocols(ConsumerGroupMember.classicProtocolListFromJoinRequestProtocolCollection( GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName, zarTopicName), - Collections.emptyList() + List.of() ) )) ) @@ -12792,7 +12789,7 @@ public class GroupMetadataManagerTest { .withSessionTimeoutMs(5000) .withProtocols(GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName, zarTopicName), - Collections.emptyList())) + List.of())) .build(); GroupMetadataManagerTestContext.JoinResult joinResult1 = context.sendClassicGroupJoin(request); @@ -12813,7 +12810,7 @@ public class GroupMetadataManagerTest { .setSupportedProtocols(ConsumerGroupMember.classicProtocolListFromJoinRequestProtocolCollection( GroupMetadataManagerTestContext.toConsumerProtocol( List.of(fooTopicName, barTopicName, zarTopicName), - Collections.emptyList() + List.of() ) )) ) @@ -12877,7 +12874,7 @@ public class GroupMetadataManagerTest { .setGroupId(groupId) .setMemberId(memberId2) .setMemberEpoch(10) - .setTopicPartitions(Collections.emptyList()) + .setTopicPartitions(List.of()) ); // Member 1 heartbeats to be notified to rejoin. @@ -13129,7 +13126,7 @@ public class GroupMetadataManagerTest { ConsumerGroupMember expectedMember2 = new ConsumerGroupMember.Builder(expectedMember1) .setMemberEpoch(11) .setState(MemberState.UNRELEASED_PARTITIONS) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0), mkTopicAssignment(zarTopicId, 0))) @@ -13187,7 +13184,7 @@ public class GroupMetadataManagerTest { .setGroupId(groupId) .setMemberId(memberId2) .setMemberEpoch(10) - .setTopicPartitions(Collections.emptyList()) + .setTopicPartitions(List.of()) ); // Member 1 heartbeats to be notified to rejoin. @@ -13446,7 +13443,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13494,7 +13491,7 @@ public class GroupMetadataManagerTest { 10, null, null, - Collections.emptyList() + List.of() ); } @@ -13510,7 +13507,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13552,7 +13549,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13598,7 +13595,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13657,7 +13654,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13714,7 +13711,7 @@ public class GroupMetadataManagerTest { .setMemberId(memberId) .setGenerationId(memberId.equals(memberId1) ? 9 : 10) ); - assertEquals(Collections.emptyList(), heartbeatResult.records()); + assertEquals(List.of(), heartbeatResult.records()); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResult.response().errorCode()); context.assertSessionTimeout(groupId, memberId, sessionTimeout); context.assertJoinTimeout(groupId, memberId, rebalanceTimeout); @@ -13759,7 +13756,7 @@ public class GroupMetadataManagerTest { .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(5000) - .setSupportedProtocols(Collections.emptyList()) + .setSupportedProtocols(List.of()) ) .build())) .build(); @@ -13785,7 +13782,7 @@ public class GroupMetadataManagerTest { .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(5000) - .setSupportedProtocols(Collections.emptyList()) + .setSupportedProtocols(List.of()) ) .build())) .build(); @@ -13831,7 +13828,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -13891,7 +13888,7 @@ public class GroupMetadataManagerTest { new ConsumerPartitionAssignor.Subscription( List.of("foo"), null, - Collections.emptyList() + List.of() ) ))) ); @@ -14145,7 +14142,7 @@ public class GroupMetadataManagerTest { GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId3) ), // Update subscription metadata. - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap())), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of())), // Bump the group epoch. List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)) ); @@ -14295,7 +14292,7 @@ public class GroupMetadataManagerTest { leaveResult.response() ); - assertEquals(Collections.emptyList(), leaveResult.records()); + assertEquals(List.of(), leaveResult.records()); } @Test @@ -14642,7 +14639,7 @@ public class GroupMetadataManagerTest { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder().build(); // GroupId is not required - List groups = context.sendShareGroupDescribe(Collections.emptyList()); + List groups = context.sendShareGroupDescribe(List.of()); assertEquals(0, groups.size()); // Group id not found @@ -14659,7 +14656,7 @@ public class GroupMetadataManagerTest { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.emptyMap() + Map.of() )); List groupIds = List.of("group-id-1", "group-id-2"); @@ -14716,7 +14713,7 @@ public class GroupMetadataManagerTest { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.emptyMap() + Map.of() )); String memberId = Uuid.randomUuid().toString(); @@ -14979,7 +14976,7 @@ public class GroupMetadataManagerTest { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.emptyMap() + Map.of() )); context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 100)); @@ -15102,7 +15099,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(90000) .setSubscribedTopicNames(List.of("foo")) - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals(1, result.response().memberEpoch()); // Verify heartbeat interval @@ -15113,7 +15110,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -15139,7 +15136,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -15198,7 +15195,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -15224,7 +15221,7 @@ public class GroupMetadataManagerTest { // Advance time. assertEquals( - Collections.emptyList(), + List.of(), context.sleep(result.response().heartbeatIntervalMs()) ); @@ -15418,7 +15415,7 @@ public class GroupMetadataManagerTest { .setTopologyEpoch(10) .setProcessId("processid") .setUserEndpoint(new Endpoint().setHost("localhost").setPort(9999)) - .setClientTags(Collections.singletonMap("key", "value")) + .setClientTags(Map.of("key", "value")) .build(); // The group and the member are created if they do not exist. @@ -15850,7 +15847,7 @@ public class GroupMetadataManagerTest { .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(List.of("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); + .setTopicPartitions(List.of()))); } @Test @@ -16131,7 +16128,7 @@ public class GroupMetadataManagerTest { List.of(memberId1, memberId2).forEach(memberId -> assertEquals( - Collections.singleton(fooTopicId), + Set.of(fooTopicId), spec.memberSubscription(memberId).subscribedTopicIds(), String.format("Member %s has unexpected subscribed topic ids", memberId) ) @@ -16165,7 +16162,7 @@ public class GroupMetadataManagerTest { mkTopicAssignment(fooTopicId, 0, 1))) .build()) .withResolvedRegularExpression("foo*", new ResolvedRegularExpression( - Collections.singleton(fooTopicName), + Set.of(fooTopicName), 100L, 12345L)) .withAssignment(memberId1, mkAssignment( @@ -16180,7 +16177,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(0) .setRebalanceTimeoutMs(10000) .setSubscribedTopicRegex("foo*") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertEquals( new ConsumerGroupHeartbeatResponseData() @@ -16234,7 +16231,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -16333,7 +16330,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo*|bar*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -16414,7 +16411,7 @@ public class GroupMetadataManagerTest { String barTopicName = "bar"; MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) @@ -16433,7 +16430,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList())); + .setTopicPartitions(List.of())); assertResponseEquals( new ConsumerGroupHeartbeatResponseData() @@ -16461,7 +16458,7 @@ public class GroupMetadataManagerTest { // The group epoch is bumped. GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), // The target assignment is created. - GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), // The member current state is created. GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1) @@ -16519,7 +16516,7 @@ public class GroupMetadataManagerTest { // The pending task was a no-op. MockCoordinatorExecutor.ExecutorResult task = tasks.get(0); assertEquals(groupId + "-regex", task.key); - assertRecordsEquals(Collections.emptyList(), task.result.records()); + assertRecordsEquals(List.of(), task.result.records()); // The member heartbeats again. It triggers a new resolution. result = context.consumerGroupHeartbeat( @@ -16587,7 +16584,7 @@ public class GroupMetadataManagerTest { String foooTopicName = "fooo"; MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); MetadataImage image = new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) @@ -16793,7 +16790,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo*|bar*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList()), + .setTopicPartitions(List.of()), ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() ); @@ -16803,7 +16800,7 @@ public class GroupMetadataManagerTest { .setMemberEpoch(10) .setHeartbeatIntervalMs(5000) .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(Collections.singletonList( + .setTopicPartitions(List.of( new ConsumerGroupHeartbeatResponseData.TopicPartitions() .setTopicId(fooTopicId) .setPartitions(List.of(3, 4, 5))))), @@ -16875,7 +16872,7 @@ public class GroupMetadataManagerTest { .setRebalanceTimeoutMs(5000) .setSubscribedTopicRegex("foo|bar*") .setServerAssignor("range") - .setTopicPartitions(Collections.emptyList()), + .setTopicPartitions(List.of()), ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion() ); @@ -16953,10 +16950,10 @@ public class GroupMetadataManagerTest { String barTopicName = "bar"; MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, Collections.singletonList(assignor)) + .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -17036,7 +17033,7 @@ public class GroupMetadataManagerTest { // Verify the expired timeout. assertEquals( - Collections.singletonList(new ExpiredTimeout( + List.of(new ExpiredTimeout( groupSessionTimeoutKey(groupId, memberId2), new CoordinatorResult<>( List.of( @@ -17044,7 +17041,7 @@ public class GroupMetadataManagerTest { GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId2), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId2), GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "bar*"), - GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap()), + GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12) ) ) @@ -17067,10 +17064,10 @@ public class GroupMetadataManagerTest { String barTopicName = "bar"; MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, Collections.singletonList(assignor)) + .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) .addTopic(barTopicId, barTopicName, 3) @@ -17208,7 +17205,7 @@ public class GroupMetadataManagerTest { @Test public void testSharePartitionDeleteRequest() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); - assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor)) .build(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index 4f245296d47..033069855da 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -406,7 +406,7 @@ public class GroupMetadataManagerTestContext { String protocolType = "consumer"; String protocolName = "range"; int generationId = 0; - List assignment = Collections.emptyList(); + List assignment = List.of(); SyncGroupRequestBuilder withGroupId(String groupId) { this.groupId = groupId; @@ -1131,7 +1131,7 @@ public class GroupMetadataManagerTestContext { SyncResult followerSyncResult = sendClassicGroupSync( syncRequest.setGroupInstanceId(followerInstanceId) .setMemberId(followerId) - .setAssignments(Collections.emptyList()) + .setAssignments(List.of()) ); assertTrue(followerSyncResult.records.isEmpty()); @@ -1273,7 +1273,7 @@ public class GroupMetadataManagerTestContext { List expectedRecords = List.of(newGroupMetadataRecord( group.groupId(), new GroupMetadataValue() - .setMembers(Collections.emptyList()) + .setMembers(List.of()) .setGeneration(group.generationId()) .setLeader(null) .setProtocolType("consumer") @@ -1498,10 +1498,10 @@ public class GroupMetadataManagerTestContext { public void verifyDescribeGroupsBeforeV6ReturnsDeadGroup(String groupId) { List describedGroups = - describeGroups(Collections.singletonList(groupId), (short) 5); + describeGroups(List.of(groupId), (short) 5); assertEquals( - Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup() + List.of(new DescribeGroupsResponseData.DescribedGroup() .setGroupId(groupId) .setGroupState(DEAD.toString()) ), @@ -1527,7 +1527,7 @@ public class GroupMetadataManagerTestContext { .withProtocolType(protocolType) .build() ); - assertEquals(Collections.emptyList(), syncResult.records); + assertEquals(List.of(), syncResult.records); assertFalse(syncResult.syncFuture.isDone()); // Simulate a successful write to log. diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index 1ab76f31772..03a7e725607 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -73,12 +73,13 @@ import java.net.InetAddress; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; +import java.util.Set; import static org.apache.kafka.common.requests.OffsetFetchResponse.INVALID_OFFSET; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_COMMITS_SENSOR_NAME; @@ -140,7 +141,7 @@ public class OffsetMetadataManagerTest { .withMetadataImage(metadataImage) .withGroupCoordinatorMetricsShard(metrics) .withGroupConfigManager(configManager) - .withConfig(GroupCoordinatorConfig.fromProps(Collections.emptyMap())) + .withConfig(GroupCoordinatorConfig.fromProps(Map.of())) .build(); } @@ -529,10 +530,10 @@ public class OffsetMetadataManagerTest { Errors expectedError ) { final OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection requestTopicCollection = - new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestTopic() .setName(topic) - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestPartition().setPartitionIndex(partition) )) ).iterator()); @@ -546,15 +547,15 @@ public class OffsetMetadataManagerTest { ); final OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection expectedResponseTopicCollection = - new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(Collections.singletonList( + new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(List.of( new OffsetDeleteResponseData.OffsetDeleteResponseTopic() .setName(topic) .setPartitions(expectedResponsePartitionCollection) ).iterator()); - List expectedRecords = Collections.emptyList(); + List expectedRecords = List.of(); if (hasOffset(groupId, topic, partition) && expectedError == Errors.NONE) { - expectedRecords = Collections.singletonList( + expectedRecords = List.of( GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord(groupId, topic, partition) ); } @@ -598,10 +599,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -628,10 +629,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -657,10 +658,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -694,10 +695,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -727,10 +728,10 @@ public class OffsetMetadataManagerTest { .setMemberId("member") .setGroupInstanceId("instanceid") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -760,10 +761,10 @@ public class OffsetMetadataManagerTest { .setMemberId("member") .setGroupInstanceId("old-instance-id") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -797,10 +798,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(1) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -832,10 +833,10 @@ public class OffsetMetadataManagerTest { assertThrows(UnknownMemberIdException.class, () -> context.commitOffset( new OffsetCommitRequestData() .setGroupId("foo") - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -870,10 +871,10 @@ public class OffsetMetadataManagerTest { .setMemberId("member") .setGenerationIdOrMemberEpoch(1) .setRetentionTimeMs(1234L) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -883,10 +884,10 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -896,7 +897,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -938,7 +939,7 @@ public class OffsetMetadataManagerTest { // Advance time by half of the session timeout. No timeouts are // expired. - assertEquals(Collections.emptyList(), context.sleep(5000 / 2)); + assertEquals(List.of(), context.sleep(5000 / 2)); // Commit. context.commitOffset( @@ -947,10 +948,10 @@ public class OffsetMetadataManagerTest { .setMemberId("member") .setGenerationIdOrMemberEpoch(1) .setRetentionTimeMs(1234L) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -960,7 +961,7 @@ public class OffsetMetadataManagerTest { // Advance time by half of the session timeout. No timeouts are // expired. - assertEquals(Collections.emptyList(), context.sleep(5000 / 2)); + assertEquals(List.of(), context.sleep(5000 / 2)); // Advance time by half of the session timeout again. The timeout should // expire and the member is removed from the group. @@ -977,10 +978,10 @@ public class OffsetMetadataManagerTest { CoordinatorResult result = context.commitOffset( new OffsetCommitRequestData() .setGroupId("foo") - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -990,10 +991,10 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1003,7 +1004,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1036,10 +1037,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") // Instance id should be ignored. .setGroupInstanceId("instance-id") - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1049,10 +1050,10 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1062,7 +1063,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1094,10 +1095,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1128,10 +1129,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(9) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1168,10 +1169,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(9) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1199,10 +1200,10 @@ public class OffsetMetadataManagerTest { CoordinatorResult result = context.commitOffset( new OffsetCommitRequestData() .setGroupId("foo") - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1212,10 +1213,10 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1225,7 +1226,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1263,10 +1264,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1278,10 +1279,10 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1291,7 +1292,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1331,7 +1332,7 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationIdOrMemberEpoch(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") .setPartitions(Arrays.asList( @@ -1351,7 +1352,7 @@ public class OffsetMetadataManagerTest { assertEquals( new OffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") .setPartitions(Arrays.asList( @@ -1367,7 +1368,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 1, @@ -1405,10 +1406,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1420,10 +1421,10 @@ public class OffsetMetadataManagerTest { assertEquals( new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1433,7 +1434,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1458,10 +1459,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1487,10 +1488,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1523,10 +1524,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(100) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1562,10 +1563,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(1) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1577,10 +1578,10 @@ public class OffsetMetadataManagerTest { assertEquals( new TxnOffsetCommitResponseData() - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code()) @@ -1590,7 +1591,7 @@ public class OffsetMetadataManagerTest { ); assertEquals( - Collections.singletonList(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( "foo", "bar", 0, @@ -1615,10 +1616,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1644,10 +1645,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(10) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1683,10 +1684,10 @@ public class OffsetMetadataManagerTest { .setGroupId("foo") .setMemberId("member") .setGenerationId(100) - .setTopics(Collections.singletonList( + .setTopics(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(100L) @@ -1714,7 +1715,7 @@ public class OffsetMetadataManagerTest { .setPartitionIndexes(Arrays.asList(0, 1)), new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("bar") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); List expectedResponse = Arrays.asList( @@ -1726,7 +1727,7 @@ public class OffsetMetadataManagerTest { )), new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkInvalidOffsetPartitionResponse(0) )) ); @@ -1744,7 +1745,7 @@ public class OffsetMetadataManagerTest { .setPartitionIndexes(Arrays.asList(0, 1)), new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("bar") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); List expectedResponse = Arrays.asList( @@ -1756,7 +1757,7 @@ public class OffsetMetadataManagerTest { )), new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkInvalidOffsetPartitionResponse(0) )) ); @@ -2001,13 +2002,13 @@ public class OffsetMetadataManagerTest { ); group.transitionTo(ClassicGroupState.DEAD); - assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", Long.MAX_VALUE)); + assertEquals(List.of(), context.fetchAllOffsets("group", Long.MAX_VALUE)); } @Test public void testFetchAllOffsetsWithUnknownGroup() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); - assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", Long.MAX_VALUE)); + assertEquals(List.of(), context.fetchAllOffsets("group", Long.MAX_VALUE)); } @Test @@ -2029,7 +2030,7 @@ public class OffsetMetadataManagerTest { assertEquals(5, context.lastWrittenOffset); // Fetching with 0 should no offsets. - assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", 0L)); + assertEquals(List.of(), context.fetchAllOffsets("group", 0L)); // Fetching with 1 should return data up to offset 1. assertEquals(Arrays.asList( @@ -2181,25 +2182,25 @@ public class OffsetMetadataManagerTest { context.commitOffset("group", "foo", 0, 100L, 1); // Fetch offsets case. - List topics = Collections.singletonList( + List topics = List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); - assertEquals(Collections.singletonList( + assertEquals(List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkOffsetPartitionResponse(0, 100L, 1, "metadata") )) ), context.fetchOffsets("group", "member", 0, topics, Long.MAX_VALUE)); // Fetch all offsets case. - assertEquals(Collections.singletonList( + assertEquals(List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkOffsetPartitionResponse(0, 100L, 1, "metadata") )) ), context.fetchAllOffsets("group", "member", 0, Long.MAX_VALUE)); @@ -2216,25 +2217,25 @@ public class OffsetMetadataManagerTest { context.commitOffset("group", "foo", 0, 100L, 1); // Fetch offsets case. - List topics = Collections.singletonList( + List topics = List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); - assertEquals(Collections.singletonList( + assertEquals(List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkOffsetPartitionResponse(0, 100L, 1, "metadata") )) ), context.fetchOffsets("group", topics, Long.MAX_VALUE)); // Fetch all offsets case. - assertEquals(Collections.singletonList( + assertEquals(List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(Collections.singletonList( + .setPartitions(List.of( mkOffsetPartitionResponse(0, 100L, 1, "metadata") )) ), context.fetchAllOffsets("group", Long.MAX_VALUE)); @@ -2246,10 +2247,10 @@ public class OffsetMetadataManagerTest { context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup("group", true); // Fetch offsets case. - List topics = Collections.singletonList( + List topics = List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); // Fetch offsets cases. @@ -2272,10 +2273,10 @@ public class OffsetMetadataManagerTest { group.updateMember(new ConsumerGroupMember.Builder("member").build()); // Fetch offsets case. - List topics = Collections.singletonList( + List topics = List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); // Fetch offsets case. @@ -2296,10 +2297,10 @@ public class OffsetMetadataManagerTest { .build() ); - List topics = Collections.singletonList( + List topics = List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(Collections.singletonList(0)) + .setPartitionIndexes(List.of(0)) ); // Fetch offsets case. @@ -2319,7 +2320,7 @@ public class OffsetMetadataManagerTest { true ); context.commitOffset("foo", "bar", 0, 100L, 0); - group.setSubscribedTopics(Optional.of(Collections.emptySet())); + group.setSubscribedTopics(Optional.of(Set.of())); context.testOffsetDeleteWith("foo", "bar", 0, Errors.NONE); assertFalse(context.hasOffset("foo", "bar", 0)); } @@ -2331,7 +2332,7 @@ public class OffsetMetadataManagerTest { "foo", true ); - group.setSubscribedTopics(Optional.of(Collections.singleton("bar"))); + group.setSubscribedTopics(Optional.of(Set.of("bar"))); context.commitOffset("foo", "bar", 0, 100L, 0); // Delete the offset whose topic partition doesn't exist. @@ -2348,7 +2349,7 @@ public class OffsetMetadataManagerTest { true ); context.commitOffset(10L, "foo", "bar", 0, 100L, 0, context.time.milliseconds()); - group.setSubscribedTopics(Optional.of(Collections.emptySet())); + group.setSubscribedTopics(Optional.of(Set.of())); context.testOffsetDeleteWith("foo", "bar", 0, Errors.NONE); assertFalse(context.hasOffset("foo", "bar", 0)); } @@ -2377,7 +2378,7 @@ public class OffsetMetadataManagerTest { .addRacks() .build(); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("bar")) + .setSubscribedTopicNames(List.of("bar")) .build(); group.computeSubscriptionMetadata( group.computeSubscribedTopicNames(null, member1), @@ -2469,7 +2470,7 @@ public class OffsetMetadataManagerTest { List records = new ArrayList<>(); assertTrue(context.cleanupExpiredOffsets("unknown-group-id", records)); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); } @Test @@ -2500,7 +2501,7 @@ public class OffsetMetadataManagerTest { List records = new ArrayList<>(); assertFalse(context.cleanupExpiredOffsets("group-id", records)); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); } @Test @@ -2524,7 +2525,7 @@ public class OffsetMetadataManagerTest { // firstTopic-0: group is still subscribed to firstTopic. Do not expire. // secondTopic-0: should expire as offset retention has passed. // secondTopic-1: has not passed offset retention. Do not expire. - List expectedRecords = Collections.singletonList( + List expectedRecords = List.of( GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord("group-id", "secondTopic", 0) ); @@ -2540,7 +2541,7 @@ public class OffsetMetadataManagerTest { // Expire secondTopic-1. context.time.sleep(500); - expectedRecords = Collections.singletonList( + expectedRecords = List.of( GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord("group-id", "secondTopic", 1) ); @@ -2589,7 +2590,7 @@ public class OffsetMetadataManagerTest { // foo-0 should not be expired because it has a pending transactional offset commit. List records = new ArrayList<>(); assertFalse(context.cleanupExpiredOffsets("group-id", records)); - assertEquals(Collections.emptyList(), records); + assertEquals(List.of(), records); } private static OffsetFetchResponseData.OffsetFetchResponsePartitions mkOffsetPartitionResponse( @@ -2972,7 +2973,7 @@ public class OffsetMetadataManagerTest { .setMemberId("member") .setGenerationIdOrMemberEpoch(1) .setRetentionTimeMs(1234L) - .setTopics(Collections.singletonList( + .setTopics(List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") .setPartitions(Arrays.asList( @@ -3010,7 +3011,7 @@ public class OffsetMetadataManagerTest { // firstTopic-0: group is still subscribed to firstTopic. Do not expire. // secondTopic-0: should expire as offset retention has passed. // secondTopic-1: has not passed offset retention. Do not expire. - List expectedRecords = Collections.singletonList( + List expectedRecords = List.of( GroupCoordinatorRecordHelpers.newOffsetCommitTombstoneRecord("group-id", "secondTopic", 0) ); @@ -3048,10 +3049,10 @@ public class OffsetMetadataManagerTest { context.commitOffset("foo", "bar", 0, 100L, 0); context.commitOffset("foo", "bar", 1, 150L, 0); - group.setSubscribedTopics(Optional.of(Collections.emptySet())); + group.setSubscribedTopics(Optional.of(Set.of())); OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection requestTopicCollection = - new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(Collections.singletonList( + new OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection(List.of( new OffsetDeleteRequestData.OffsetDeleteRequestTopic() .setName("bar") .setPartitions(Arrays.asList( @@ -3173,7 +3174,7 @@ public class OffsetMetadataManagerTest { 5000, "consumer", new JoinGroupRequestData.JoinGroupRequestProtocolCollection( - Collections.singletonList(new JoinGroupRequestData.JoinGroupRequestProtocol() + List.of(new JoinGroupRequestData.JoinGroupRequestProtocol() .setName("range") .setMetadata(new byte[0]) ).iterator() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java index dbd3f162ed4..eb2f1f69502 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.coordinator.group.modern.MemberSubscriptionAndAssignment import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -108,6 +107,6 @@ public class GroupSpecImplTest { )); assertEquals(topicPartitions, groupSpec.memberAssignment(TEST_MEMBER).partitions()); - assertEquals(Collections.emptyMap(), groupSpec.memberAssignment("unknown-member").partitions()); + assertEquals(Map.of(), groupSpec.memberAssignment("unknown-member").partitions()); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java index 39f3dc1f5d7..e47aadb482a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.coordinator.group.modern.TopicMetadata; import org.junit.jupiter.api.Test; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -63,7 +62,7 @@ public class OptimizedUniformAssignmentBuilderTest { @Test public void testOneMemberNoTopicSubscription() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -73,12 +72,12 @@ public class OptimizedUniformAssignmentBuilderTest { ) ); - Map members = Collections.singletonMap( + Map members = Map.of( memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.emptySet(), + Set.of(), Assignment.EMPTY ) ); @@ -86,7 +85,7 @@ public class OptimizedUniformAssignmentBuilderTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); GroupAssignment groupAssignment = assignor.assign( @@ -94,13 +93,13 @@ public class OptimizedUniformAssignmentBuilderTest { subscribedTopicMetadata ); - assertEquals(Collections.emptyMap(), groupAssignment.members()); + assertEquals(Map.of(), groupAssignment.members()); } @Test public void testOneMemberSubscribedToNonexistentTopic() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -110,12 +109,12 @@ public class OptimizedUniformAssignmentBuilderTest { ) ); - Map members = Collections.singletonMap( + Map members = Map.of( memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic2Uuid), + Set.of(topic2Uuid), Assignment.EMPTY ) ); @@ -123,7 +122,7 @@ public class OptimizedUniformAssignmentBuilderTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); assertThrows(PartitionAssignorException.class, @@ -172,7 +171,7 @@ public class OptimizedUniformAssignmentBuilderTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -199,21 +198,21 @@ public class OptimizedUniformAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); members.put(memberC, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); @@ -226,13 +225,13 @@ public class OptimizedUniformAssignmentBuilderTest { mkTopicAssignment(topic3Uuid, 1) )); expectedAssignment.put(memberC, - Collections.emptyMap() + Map.of() ); GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -270,7 +269,7 @@ public class OptimizedUniformAssignmentBuilderTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -563,7 +562,7 @@ public class OptimizedUniformAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic2Uuid), + Set.of(topic2Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0), mkTopicAssignment(topic2Uuid, 0) @@ -573,7 +572,7 @@ public class OptimizedUniformAssignmentBuilderTest { members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic2Uuid), + Set.of(topic2Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 1), mkTopicAssignment(topic2Uuid, 1) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java index 3c157246c0e..5ab032e47ca 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java @@ -32,7 +32,6 @@ import org.apache.kafka.coordinator.group.modern.TopicMetadata; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -62,7 +61,7 @@ public class RangeAssignorTest { @Test public void testOneMemberNoTopic() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -72,12 +71,12 @@ public class RangeAssignorTest { ) ); - Map members = Collections.singletonMap( + Map members = Map.of( memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.emptySet(), + Set.of(), Assignment.EMPTY ) ); @@ -85,7 +84,7 @@ public class RangeAssignorTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); GroupAssignment groupAssignment = assignor.assign( @@ -93,9 +92,9 @@ public class RangeAssignorTest { subscribedTopicMetadata ); - Map expectedAssignment = Collections.singletonMap( + Map expectedAssignment = Map.of( memberA, - new MemberAssignmentImpl(Collections.emptyMap()) + new MemberAssignmentImpl(Map.of()) ); assertEquals(expectedAssignment, groupAssignment.members()); @@ -104,7 +103,7 @@ public class RangeAssignorTest { @Test public void testOneMemberSubscribedToNonExistentTopic() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -114,7 +113,7 @@ public class RangeAssignorTest { ) ); - Map members = Collections.singletonMap( + Map members = Map.of( memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), @@ -127,7 +126,7 @@ public class RangeAssignorTest { GroupSpec groupSpec = new GroupSpecImpl( members, HOMOGENEOUS, - Collections.emptyMap() + Map.of() ); assertThrows(PartitionAssignorException.class, @@ -328,7 +327,7 @@ public class RangeAssignorTest { @Test public void testStaticMembership() throws PartitionAssignorException { SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -342,13 +341,13 @@ public class RangeAssignorTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceA"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceB"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); @@ -368,7 +367,7 @@ public class RangeAssignorTest { members.put("memberA1", new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceA"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); @@ -398,7 +397,7 @@ public class RangeAssignorTest { @Test public void testMixedStaticMembership() throws PartitionAssignorException { SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -413,13 +412,13 @@ public class RangeAssignorTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceA"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); members.put(memberC, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceC"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); @@ -427,7 +426,7 @@ public class RangeAssignorTest { members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); @@ -447,7 +446,7 @@ public class RangeAssignorTest { members.put("memberA1", new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.of("instanceA"), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); @@ -545,7 +544,7 @@ public class RangeAssignorTest { mkTopicAssignment(topic2Uuid, 1) )); // Member C shouldn't get any assignment. - expectedAssignment.put(memberC, Collections.emptyMap()); + expectedAssignment.put(memberC, Map.of()); assertAssignment(expectedAssignment, computedAssignment); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java index 22425476c6f..decffabb6f4 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java @@ -31,7 +31,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -91,7 +90,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { @Test public void testTwoMembersNoTopicSubscription() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -105,20 +104,20 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.emptySet(), + Set.of(), Assignment.EMPTY )); members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.emptySet(), + Set.of(), Assignment.EMPTY )); GroupSpec groupSpec = new TestGroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + Map.of() ); GroupAssignment groupAssignment = assignor.assign( @@ -126,13 +125,13 @@ public class UniformHeterogeneousAssignmentBuilderTest { subscribedTopicMetadata ); - assertEquals(Collections.emptyMap(), groupAssignment.members()); + assertEquals(Map.of(), groupAssignment.members()); } @Test public void testTwoMembersSubscribedToNonexistentTopics() { SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl( - Collections.singletonMap( + Map.of( topic1Uuid, new TopicMetadata( topic1Uuid, @@ -146,20 +145,20 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic2Uuid), + Set.of(topic2Uuid), Assignment.EMPTY )); GroupSpec groupSpec = new TestGroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + Map.of() ); assertThrows( @@ -193,14 +192,14 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); GroupSpec groupSpec = new TestGroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -240,28 +239,28 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic3Uuid), + Set.of(topic3Uuid), Assignment.EMPTY )); members.put(memberC, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), Assignment.EMPTY )); GroupSpec groupSpec = new TestGroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -276,7 +275,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { mkTopicAssignment(topic3Uuid, 0) )); expectedAssignment.put(memberB, - Collections.emptyMap() + Map.of() ); expectedAssignment.put(memberC, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1) @@ -309,7 +308,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2) )) @@ -456,7 +455,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0, 2), mkTopicAssignment(topic2Uuid, 0) @@ -542,7 +541,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic2Uuid), + Set.of(topic2Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic2Uuid, 3, 4, 5, 6) )) @@ -594,7 +593,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberA, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.singleton(topic1Uuid), + Set.of(topic1Uuid), new Assignment(mkAssignment( mkTopicAssignment(topic1Uuid, 0, 2), mkTopicAssignment(topic2Uuid, 1, 3) @@ -725,14 +724,14 @@ public class UniformHeterogeneousAssignmentBuilderTest { members.put(memberB, new MemberSubscriptionAndAssignmentImpl( Optional.empty(), Optional.empty(), - Collections.emptySet(), + Set.of(), Assignment.EMPTY )); GroupSpec groupSpec = new TestGroupSpecImpl( members, HETEROGENEOUS, - Collections.emptyMap() + Map.of() ); SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(topicMetadata); @@ -745,7 +744,7 @@ public class UniformHeterogeneousAssignmentBuilderTest { expectedAssignment.put(memberA, mkAssignment( mkTopicAssignment(topic1Uuid, 0, 1, 2) )); - expectedAssignment.put(memberB, Collections.emptyMap()); + expectedAssignment.put(memberB, Map.of()); assertAssignment(expectedAssignment, computedAssignment); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java index 93c382dc4d3..a8b4d639e1c 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java @@ -57,7 +57,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Optional; @@ -429,7 +428,7 @@ public class ClassicGroupTest { group.initNextGeneration(); - assertEquals(Optional.of(Collections.emptySet()), group.subscribedTopics()); + assertEquals(Optional.of(Set.of()), group.subscribedTopics()); protocols = new JoinGroupRequestProtocolCollection(); protocols.add(new JoinGroupRequestProtocol() @@ -852,7 +851,7 @@ public class ClassicGroupTest { assertTrue(group.addPendingSyncMember(memberId)); assertEquals(Set.of(memberId), group.allPendingSyncMembers()); group.removePendingSyncMember(memberId); - assertEquals(Collections.emptySet(), group.allPendingSyncMembers()); + assertEquals(Set.of(), group.allPendingSyncMembers()); } @Test @@ -877,7 +876,7 @@ public class ClassicGroupTest { assertTrue(group.addPendingSyncMember(memberId)); assertEquals(Set.of(memberId), group.allPendingSyncMembers()); group.remove(memberId); - assertEquals(Collections.emptySet(), group.allPendingSyncMembers()); + assertEquals(Set.of(), group.allPendingSyncMembers()); } @Test @@ -903,7 +902,7 @@ public class ClassicGroupTest { assertTrue(group.addPendingSyncMember(memberId)); assertEquals(Set.of(memberId), group.allPendingSyncMembers()); group.initNextGeneration(); - assertEquals(Collections.emptySet(), group.allPendingSyncMembers()); + assertEquals(Set.of(), group.allPendingSyncMembers()); } @Test @@ -1251,7 +1250,7 @@ public class ClassicGroupTest { group.transitionTo(PREPARING_REBALANCE); group.initNextGeneration(); assertTrue(group.isInState(EMPTY)); - assertEquals(Optional.of(Collections.emptySet()), group.computeSubscribedTopics()); + assertEquals(Optional.of(Set.of()), group.computeSubscribedTopics()); assertTrue(group.usesConsumerGroupProtocol()); assertFalse(group.isSubscribedToTopic("topic")); @@ -1458,7 +1457,7 @@ public class ClassicGroupTest { ClassicGroup classicGroup = ClassicGroup.fromConsumerGroup( consumerGroup, - Collections.emptySet(), + Set.of(), newMember2, logContext, time, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java index 8e136064689..3aa0a861725 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java @@ -37,9 +37,9 @@ import com.yammer.metrics.core.MetricsRegistry; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.stream.IntStream; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME; @@ -77,43 +77,43 @@ public class GroupCoordinatorMetricsTest { metrics.metricName( "group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", "classic")), + Map.of("protocol", "classic")), metrics.metricName( "group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", "consumer")), + Map.of("protocol", "consumer")), metrics.metricName( "consumer-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", ConsumerGroupState.EMPTY.toString())), + Map.of("state", ConsumerGroupState.EMPTY.toString())), metrics.metricName( "consumer-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", ConsumerGroupState.ASSIGNING.toString())), + Map.of("state", ConsumerGroupState.ASSIGNING.toString())), metrics.metricName( "consumer-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", ConsumerGroupState.RECONCILING.toString())), + Map.of("state", ConsumerGroupState.RECONCILING.toString())), metrics.metricName( "consumer-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", ConsumerGroupState.STABLE.toString())), + Map.of("state", ConsumerGroupState.STABLE.toString())), metrics.metricName( "consumer-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", ConsumerGroupState.DEAD.toString())), + Map.of("state", ConsumerGroupState.DEAD.toString())), metrics.metricName( "group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())), + Map.of("protocol", Group.GroupType.SHARE.toString())), metrics.metricName( "rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())), + Map.of("protocol", Group.GroupType.SHARE.toString())), metrics.metricName( "rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())), + Map.of("protocol", Group.GroupType.SHARE.toString())), metrics.metricName( "share-group-count", GroupCoordinatorMetrics.METRICS_GROUP, @@ -132,33 +132,33 @@ public class GroupCoordinatorMetricsTest { metrics.metricName( "group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("protocol", Group.GroupType.STREAMS.toString())), + Map.of("protocol", Group.GroupType.STREAMS.toString())), metrics.metricName("streams-group-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP), metrics.metricName("streams-group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.EMPTY.toString())), + Map.of("state", StreamsGroupState.EMPTY.toString())), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.ASSIGNING.toString())), + Map.of("state", StreamsGroupState.ASSIGNING.toString())), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.RECONCILING.toString())), + Map.of("state", StreamsGroupState.RECONCILING.toString())), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.STABLE.toString())), + Map.of("state", StreamsGroupState.STABLE.toString())), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.DEAD.toString())), + Map.of("state", StreamsGroupState.DEAD.toString())), metrics.metricName( "streams-group-count", GroupCoordinatorMetrics.METRICS_GROUP, - Collections.singletonMap("state", StreamsGroupState.NOT_READY.toString())) + Map.of("state", StreamsGroupState.NOT_READY.toString())) )); try { @@ -176,7 +176,7 @@ public class GroupCoordinatorMetricsTest { assertMetricsForTypeEqual(registry, "kafka.coordinator.group", expectedRegistry); expectedMetrics.forEach(metricName -> assertTrue(metrics.metrics().containsKey(metricName), metricName + " is missing")); } - assertMetricsForTypeEqual(registry, "kafka.coordinator.group", Collections.emptySet()); + assertMetricsForTypeEqual(registry, "kafka.coordinator.group", Set.of()); expectedMetrics.forEach(metricName -> assertFalse(metrics.metrics().containsKey(metricName))); } finally { registry.shutdown(); @@ -211,13 +211,13 @@ public class GroupCoordinatorMetricsTest { Utils.mkEntry(ClassicGroupState.DEAD, 1L) )); - shard0.setConsumerGroupGauges(Collections.singletonMap(ConsumerGroupState.ASSIGNING, 5L)); + shard0.setConsumerGroupGauges(Map.of(ConsumerGroupState.ASSIGNING, 5L)); shard1.setConsumerGroupGauges(Map.of( ConsumerGroupState.RECONCILING, 1L, ConsumerGroupState.DEAD, 1L )); - shard0.setStreamsGroupGauges(Collections.singletonMap(StreamsGroupState.ASSIGNING, 2L)); + shard0.setStreamsGroupGauges(Map.of(StreamsGroupState.ASSIGNING, 2L)); shard1.setStreamsGroupGauges(Map.of( StreamsGroupState.RECONCILING, 1L, StreamsGroupState.DEAD, 1L, @@ -240,7 +240,7 @@ public class GroupCoordinatorMetricsTest { assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 9); assertGaugeValue( metrics, - metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "classic")), + metrics.metricName("group-count", METRICS_GROUP, Map.of("protocol", "classic")), 9 ); @@ -256,7 +256,7 @@ public class GroupCoordinatorMetricsTest { assertEquals(1, shard1.numOffsets()); assertGaugeValue( metrics, - metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "consumer")), + metrics.metricName("group-count", METRICS_GROUP, Map.of("protocol", "consumer")), 7 ); assertGaugeValue(registry, metricName("GroupMetadataManager", "NumOffsets"), 7); @@ -265,7 +265,7 @@ public class GroupCoordinatorMetricsTest { assertEquals(6, shard1.numShareGroups()); assertGaugeValue( metrics, - metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "share")), + metrics.metricName("group-count", METRICS_GROUP, Map.of("protocol", "share")), 8 ); @@ -273,7 +273,7 @@ public class GroupCoordinatorMetricsTest { assertEquals(3, shard1.numStreamsGroups()); assertGaugeValue( metrics, - metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "streams")), + metrics.metricName("group-count", METRICS_GROUP, Map.of("protocol", "streams")), 5 ); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java index c267195eeae..f33eed56918 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.image.TopicsImage; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -323,11 +322,11 @@ public class TargetAssignmentBuilderTest { ); TargetAssignmentBuilder.TargetAssignmentResult result = context.build(); - assertEquals(Collections.singletonList(newConsumerGroupTargetAssignmentEpochRecord( + assertEquals(List.of(newConsumerGroupTargetAssignmentEpochRecord( "my-group", 20 )), result.records()); - assertEquals(Collections.emptyMap(), result.targetAssignment()); + assertEquals(Map.of(), result.targetAssignment()); } @Test @@ -362,7 +361,7 @@ public class TargetAssignmentBuilderTest { TargetAssignmentBuilder.TargetAssignmentResult result = context.build(); - assertEquals(Collections.singletonList(newConsumerGroupTargetAssignmentEpochRecord( + assertEquals(List.of(newConsumerGroupTargetAssignmentEpochRecord( "my-group", 20 )), result.records()); @@ -870,10 +869,10 @@ public class TargetAssignmentBuilderTest { context.addGroupMember("member-2", Arrays.asList("foo", "bar", "zar"), mkAssignment()); - context.addGroupMember("member-3", Collections.emptyList(), "foo*", mkAssignment()); + context.addGroupMember("member-3", List.of(), "foo*", mkAssignment()); context.addResolvedRegularExpression("foo*", new ResolvedRegularExpression( - Collections.singleton("foo"), + Set.of("foo"), 10L, 12345L )); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java index 550b3b49019..653233923d0 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.image.TopicsImage; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -40,12 +39,12 @@ public class TopicIdsTest { @Test public void testTopicsImageCannotBeNull() { - assertThrows(NullPointerException.class, () -> new TopicIds(Collections.emptySet(), (TopicsImage) null)); + assertThrows(NullPointerException.class, () -> new TopicIds(Set.of(), (TopicsImage) null)); } @Test public void testTopicResolverCannotBeNull() { - assertThrows(NullPointerException.class, () -> new TopicIds(Collections.emptySet(), (TopicIds.TopicResolver) null)); + assertThrows(NullPointerException.class, () -> new TopicIds(Set.of(), (TopicIds.TopicResolver) null)); } @Test @@ -57,7 +56,7 @@ public class TopicIdsTest { @Test public void testIsEmpty() { - Set topicNames = Collections.emptySet(); + Set topicNames = Set.of(); Set topicIds = new TopicIds(topicNames, TopicsImage.EMPTY); assertEquals(topicNames.size(), topicIds.size()); } @@ -176,14 +175,14 @@ public class TopicIdsTest { @Test public void testEquals() { Uuid topicId = Uuid.randomUuid(); - TopicIds topicIds1 = new TopicIds(Collections.singleton("topic"), + TopicIds topicIds1 = new TopicIds(Set.of("topic"), new MetadataImageBuilder() .addTopic(topicId, "topicId", 3) .build() .topics() ); - TopicIds topicIds2 = new TopicIds(Collections.singleton("topic"), + TopicIds topicIds2 = new TopicIds(Set.of("topic"), new MetadataImageBuilder() .addTopic(topicId, "topicId", 3) .build() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/UnionSetTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/UnionSetTest.java index 2653c7385f1..fc65baab791 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/UnionSetTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/UnionSetTest.java @@ -20,7 +20,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Set; import java.util.stream.IntStream; @@ -34,8 +33,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class UnionSetTest { @Test public void testSetsCannotBeNull() { - assertThrows(NullPointerException.class, () -> new UnionSet(Collections.emptySet(), null)); - assertThrows(NullPointerException.class, () -> new UnionSet(null, Collections.emptySet())); + assertThrows(NullPointerException.class, () -> new UnionSet(Set.of(), null)); + assertThrows(NullPointerException.class, () -> new UnionSet(null, Set.of())); } @Test @@ -72,21 +71,21 @@ public class UnionSetTest { union = new UnionSet<>( Set.of(1, 2, 3), - Collections.emptySet() + Set.of() ); assertFalse(union.isEmpty()); union = new UnionSet<>( - Collections.emptySet(), + Set.of(), Set.of(2, 3, 4, 5) ); assertFalse(union.isEmpty()); union = new UnionSet<>( - Collections.emptySet(), - Collections.emptySet() + Set.of(), + Set.of() ); assertTrue(union.isEmpty()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java index 658e6c23260..47b50699ab1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java @@ -31,7 +31,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -39,7 +38,6 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.Set; -import java.util.stream.Collectors; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; @@ -178,7 +176,7 @@ public class ConsumerGroupMemberTest { .maybeUpdateRackId(Optional.of("new-rack-id")) .maybeUpdateInstanceId(Optional.of("new-instance-id")) .maybeUpdateServerAssignorName(Optional.of("new-assignor")) - .maybeUpdateSubscribedTopicNames(Optional.of(Collections.singletonList("zar"))) + .maybeUpdateSubscribedTopicNames(Optional.of(List.of("zar"))) .maybeUpdateSubscribedTopicRegex(Optional.of("new-regex")) .maybeUpdateRebalanceTimeoutMs(OptionalInt.of(6000)) .build(); @@ -231,10 +229,10 @@ public class ConsumerGroupMemberTest { ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() + .setAssignedPartitions(List.of(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId1) .setPartitions(Arrays.asList(0, 1, 2)))) - .setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() + .setPartitionsPendingRevocation(List.of(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId2) .setPartitions(Arrays.asList(3, 4, 5)))); @@ -266,10 +264,10 @@ public class ConsumerGroupMemberTest { ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(epoch) .setPreviousMemberEpoch(epoch - 1) - .setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() + .setAssignedPartitions(List.of(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId1) .setPartitions(assignedPartitions))) - .setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() + .setPartitionsPendingRevocation(List.of(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(topicId2) .setPartitions(Arrays.asList(3, 4, 5)))); String memberId = Uuid.randomUuid().toString(); @@ -306,7 +304,7 @@ public class ConsumerGroupMemberTest { .setSubscribedTopicRegex(subscribedTopicRegex) .setAssignment( new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(Collections.singletonList(new ConsumerGroupDescribeResponseData.TopicPartitions() + .setTopicPartitions(List.of(new ConsumerGroupDescribeResponseData.TopicPartitions() .setTopicId(topicId1) .setTopicName("topic1") .setPartitions(assignedPartitions) @@ -319,7 +317,7 @@ public class ConsumerGroupMemberTest { .setTopicId(item.getKey()) .setTopicName("topic4") .setPartitions(new ArrayList<>(item.getValue())) - ).collect(Collectors.toList())) + ).toList()) ) .setMemberType(withClassicMemberMetadata ? (byte) 0 : (byte) 1); @@ -341,7 +339,7 @@ public class ConsumerGroupMemberTest { public void testAsConsumerGroupDescribeWithTopicNameNotFound() { Uuid memberId = Uuid.randomUuid(); ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue() - .setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() + .setAssignedPartitions(List.of(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions() .setTopicId(Uuid.randomUuid()) .setPartitions(Arrays.asList(0, 1, 2)))); ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId.toString()) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java index 886820e5513..333df21d9c3 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java @@ -57,7 +57,6 @@ import org.junit.jupiter.params.provider.CsvSource; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -247,7 +246,7 @@ public class ConsumerGroupTest { member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) - .setAssignedPartitions(Collections.emptyMap()) + .setAssignedPartitions(Map.of()) .setPartitionsPendingRevocation(mkAssignment( mkTopicAssignment(fooTopicId, 1))) .build(); @@ -260,7 +259,7 @@ public class ConsumerGroupTest { .setMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 1))) - .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingRevocation(Map.of()) .build(); consumerGroup.updateMember(member); @@ -641,20 +640,20 @@ public class ConsumerGroupTest { .build(); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") - .setSubscribedTopicNames(Collections.singletonList("bar")) + .setSubscribedTopicNames(List.of("bar")) .build(); ConsumerGroupMember member3 = new ConsumerGroupMember.Builder("member3") - .setSubscribedTopicNames(Collections.singletonList("zar")) + .setSubscribedTopicNames(List.of("zar")) .build(); ConsumerGroup consumerGroup = createConsumerGroup("group-foo"); // It should be empty by default. assertEquals( - Collections.emptyMap(), + Map.of(), consumerGroup.computeSubscriptionMetadata( consumerGroup.computeSubscribedTopicNames(null, null), image.topics(), @@ -691,7 +690,7 @@ public class ConsumerGroupTest { // Compute while taking into account removal of member 1. assertEquals( - Collections.emptyMap(), + Map.of(), consumerGroup.computeSubscriptionMetadata( consumerGroup.computeSubscribedTopicNames(member1, null), image.topics(), @@ -785,7 +784,7 @@ public class ConsumerGroupTest { // Compute while taking into account removal of member 1, member 2 and member 3 assertEquals( - Collections.emptyMap(), + Map.of(), consumerGroup.computeSubscriptionMetadata( consumerGroup.computeSubscribedTopicNames(new HashSet<>(Arrays.asList(member1, member2, member3))), image.topics(), @@ -812,7 +811,7 @@ public class ConsumerGroupTest { mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) ), consumerGroup.computeSubscriptionMetadata( - consumerGroup.computeSubscribedTopicNames(Collections.singleton(member1)), + consumerGroup.computeSubscribedTopicNames(Set.of(member1)), image.topics(), image.cluster() ) @@ -826,7 +825,7 @@ public class ConsumerGroupTest { mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) ), consumerGroup.computeSubscriptionMetadata( - consumerGroup.computeSubscribedTopicNames(Collections.emptySet()), + consumerGroup.computeSubscribedTopicNames(Set.of()), image.topics(), image.cluster() ) @@ -836,7 +835,7 @@ public class ConsumerGroupTest { @Test public void testUpdateSubscribedTopicNamesAndSubscriptionType() { ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") .setSubscribedTopicNames(Arrays.asList("bar", "foo")) @@ -849,7 +848,7 @@ public class ConsumerGroupTest { // It should be empty by default. assertEquals( - Collections.emptyMap(), + Map.of(), consumerGroup.subscribedTopicNames() ); @@ -910,9 +909,9 @@ public class ConsumerGroupTest { String memberId2 = "member2"; // Initial assignment for member1 - Assignment initialAssignment = new Assignment(Collections.singletonMap( + Assignment initialAssignment = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(0)) + new HashSet<>(List.of(0)) )); consumerGroup.updateTargetAssignment(memberId1, initialAssignment); @@ -925,9 +924,9 @@ public class ConsumerGroupTest { ); // New assignment for member1 - Assignment newAssignment = new Assignment(Collections.singletonMap( + Assignment newAssignment = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(1)) + new HashSet<>(List.of(1)) )); consumerGroup.updateTargetAssignment(memberId1, newAssignment); @@ -940,9 +939,9 @@ public class ConsumerGroupTest { ); // New assignment for member2 to add partition 1 - Assignment newAssignment2 = new Assignment(Collections.singletonMap( + Assignment newAssignment2 = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(1)) + new HashSet<>(List.of(1)) )); consumerGroup.updateTargetAssignment(memberId2, newAssignment2); @@ -955,9 +954,9 @@ public class ConsumerGroupTest { ); // New assignment for member1 to revoke partition 1 and assign partition 0 - Assignment newAssignment1 = new Assignment(Collections.singletonMap( + Assignment newAssignment1 = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(0)) + new HashSet<>(List.of(0)) )); consumerGroup.updateTargetAssignment(memberId1, newAssignment1); @@ -1121,14 +1120,14 @@ public class ConsumerGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( snapshotRegistry, - Collections.emptyMap(), + Map.of(), new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) ); ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard); snapshotRegistry.idempotentCreateSnapshot(0); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); group.updateMember(new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build()); snapshotRegistry.idempotentCreateSnapshot(1); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); @@ -1222,10 +1221,10 @@ public class ConsumerGroupTest { .build(); ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") - .setSubscribedTopicNames(Collections.singletonList("bar")) + .setSubscribedTopicNames(List.of("bar")) .build(); ConsumerGroup consumerGroup = createConsumerGroup("group-foo"); @@ -1263,7 +1262,7 @@ public class ConsumerGroupTest { assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); group.updateMember(new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .setServerAssignorName("assignorName") .build()); group.updateMember(new ConsumerGroupMember.Builder("member2") @@ -1279,7 +1278,7 @@ public class ConsumerGroupTest { .setMembers(Arrays.asList( new ConsumerGroupDescribeResponseData.Member() .setMemberId("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .setSubscribedTopicRegex("") .setMemberType((byte) 1), new ConsumerGroupDescribeResponseData.Member().setMemberId("member2") @@ -1297,21 +1296,21 @@ public class ConsumerGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( snapshotRegistry, - Collections.emptyMap(), + Map.of(), new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) ); ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard); snapshotRegistry.idempotentCreateSnapshot(0); - assertTrue(group.isInStates(Collections.singleton("empty"), 0)); - assertFalse(group.isInStates(Collections.singleton("Empty"), 0)); + assertTrue(group.isInStates(Set.of("empty"), 0)); + assertFalse(group.isInStates(Set.of("Empty"), 0)); group.updateMember(new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build()); snapshotRegistry.idempotentCreateSnapshot(1); - assertTrue(group.isInStates(Collections.singleton("empty"), 0)); - assertTrue(group.isInStates(Collections.singleton("stable"), 1)); - assertFalse(group.isInStates(Collections.singleton("empty"), 1)); + assertTrue(group.isInStates(Set.of("empty"), 0)); + assertTrue(group.isInStates(Set.of("stable"), 1)); + assertFalse(group.isInStates(Set.of("empty"), 1)); } @Test @@ -1512,7 +1511,7 @@ public class ConsumerGroupTest { 5000, 500, ConsumerProtocol.PROTOCOL_TYPE, - new JoinGroupRequestData.JoinGroupRequestProtocolCollection(Collections.singletonList( + new JoinGroupRequestData.JoinGroupRequestProtocolCollection(List.of( new JoinGroupRequestData.JoinGroupRequestProtocol() .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( @@ -1562,7 +1561,7 @@ public class ConsumerGroupTest { .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(member.sessionTimeoutMs()) - .setSupportedProtocols(Collections.singletonList( + .setSupportedProtocols(List.of( new ConsumerGroupMemberMetadataValue.ClassicProtocol() .setName("range") .setMetadata(Utils.toArray(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( @@ -1927,7 +1926,7 @@ public class ConsumerGroupTest { consumerGroup.computeSubscribedTopicNames( member3, new ConsumerGroupMember.Builder(member3) - .setSubscribedTopicNames(Collections.emptyList()) + .setSubscribedTopicNames(List.of()) .build() ) ); @@ -1942,7 +1941,7 @@ public class ConsumerGroupTest { consumerGroup.computeSubscribedTopicNames( member3, new ConsumerGroupMember.Builder(member3) - .setSubscribedTopicNames(Collections.emptyList()) + .setSubscribedTopicNames(List.of()) .setSubscribedTopicRegex("") .build() ) @@ -2122,8 +2121,8 @@ public class ConsumerGroupTest { assertEquals( HOMOGENEOUS, ConsumerGroup.subscriptionType( - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), 0 ) ); @@ -2131,7 +2130,7 @@ public class ConsumerGroupTest { assertEquals( HOMOGENEOUS, ConsumerGroup.subscriptionType( - Collections.emptyMap(), + Map.of(), Map.of("foo", new SubscriptionCount(5, 0)), 5 ) @@ -2140,7 +2139,7 @@ public class ConsumerGroupTest { assertEquals( HETEROGENEOUS, ConsumerGroup.subscriptionType( - Collections.emptyMap(), + Map.of(), Map.of( "foo", new SubscriptionCount(4, 0), "bar", new SubscriptionCount(1, 0) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilderTest.java index 8e4e36712ba..3a4931efad9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/CurrentAssignmentBuilderTest.java @@ -25,7 +25,7 @@ import org.apache.kafka.coordinator.group.modern.MemberState; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; +import java.util.List; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; @@ -194,7 +194,7 @@ public class CurrentAssignmentBuilderTest { .withCurrentPartitionEpoch((topicId, __) -> topicId2.equals(topicId) ? 10 : -1 ) - .withOwnedTopicPartitions(Collections.emptyList()) + .withOwnedTopicPartitions(List.of()) .build(); assertEquals( @@ -560,7 +560,7 @@ public class CurrentAssignmentBuilderTest { mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6)))) .withCurrentPartitionEpoch((topicId, partitionId) -> 11) - .withOwnedTopicPartitions(Collections.emptyList()) + .withOwnedTopicPartitions(List.of()) .build(); assertEquals( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java index a991a3ace0c..15026944721 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.image.MetadataImage; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.Set; @@ -117,7 +116,7 @@ public class ShareGroupMemberTest { updatedMember = new ShareGroupMember.Builder(member) .maybeUpdateRackId(Optional.of("new-rack-id")) - .maybeUpdateSubscribedTopicNames(Optional.of(Collections.singletonList("zar"))) + .maybeUpdateSubscribedTopicNames(Optional.of(List.of("zar"))) .build(); assertNull(member.instanceId()); @@ -180,7 +179,7 @@ public class ShareGroupMemberTest { .setSubscribedTopicNames(subscribedTopicNames) .setAssignment( new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(Collections.singletonList(new ShareGroupDescribeResponseData.TopicPartitions() + .setTopicPartitions(List.of(new ShareGroupDescribeResponseData.TopicPartitions() .setTopicId(topicId1) .setTopicName("topic1") .setPartitions(assignedPartitions) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java index 8dc7167d898..c54f45035be 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java @@ -38,8 +38,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -156,20 +158,20 @@ public class ShareGroupTest { .build(); ShareGroupMember member1 = new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ShareGroupMember member2 = new ShareGroupMember.Builder("member2") - .setSubscribedTopicNames(Collections.singletonList("bar")) + .setSubscribedTopicNames(List.of("bar")) .build(); ShareGroupMember member3 = new ShareGroupMember.Builder("member3") - .setSubscribedTopicNames(Collections.singletonList("zar")) + .setSubscribedTopicNames(List.of("zar")) .build(); ShareGroup shareGroup = createShareGroup("group-foo"); // It should be empty by default. assertEquals( - Collections.emptyMap(), + Map.of(), shareGroup.computeSubscriptionMetadata( shareGroup.computeSubscribedTopicNames(null, null), image.topics(), @@ -206,7 +208,7 @@ public class ShareGroupTest { // Compute while taking into account removal of member 1. assertEquals( - Collections.emptyMap(), + Map.of(), shareGroup.computeSubscriptionMetadata( shareGroup.computeSubscribedTopicNames(member1, null), image.topics(), @@ -300,7 +302,7 @@ public class ShareGroupTest { // Compute while taking into account removal of member 1, member 2 and member 3 assertEquals( - Collections.emptyMap(), + Map.of(), shareGroup.computeSubscriptionMetadata( shareGroup.computeSubscribedTopicNames(new HashSet<>(Arrays.asList(member1, member2, member3))), image.topics(), @@ -327,7 +329,7 @@ public class ShareGroupTest { mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) ), shareGroup.computeSubscriptionMetadata( - shareGroup.computeSubscribedTopicNames(Collections.singleton(member1)), + shareGroup.computeSubscribedTopicNames(Set.of(member1)), image.topics(), image.cluster() ) @@ -341,7 +343,7 @@ public class ShareGroupTest { mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) ), shareGroup.computeSubscriptionMetadata( - shareGroup.computeSubscribedTopicNames(Collections.emptySet()), + shareGroup.computeSubscribedTopicNames(Set.of()), image.topics(), image.cluster() ) @@ -351,7 +353,7 @@ public class ShareGroupTest { @Test public void testUpdateSubscribedTopicNamesAndSubscriptionType() { ShareGroupMember member1 = new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ShareGroupMember member2 = new ShareGroupMember.Builder("member2") .setSubscribedTopicNames(Arrays.asList("bar", "foo")) @@ -364,7 +366,7 @@ public class ShareGroupTest { // It should be empty by default. assertEquals( - Collections.emptyMap(), + Map.of(), shareGroup.subscribedTopicNames() ); @@ -424,9 +426,9 @@ public class ShareGroupTest { String memberId2 = "member2"; // Initial assignment for member1 - Assignment initialAssignment = new Assignment(Collections.singletonMap( + Assignment initialAssignment = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(0)) + new HashSet<>(List.of(0)) )); shareGroup.updateTargetAssignment(memberId1, initialAssignment); @@ -439,9 +441,9 @@ public class ShareGroupTest { ); // New assignment for member1 - Assignment newAssignment = new Assignment(Collections.singletonMap( + Assignment newAssignment = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(1)) + new HashSet<>(List.of(1)) )); shareGroup.updateTargetAssignment(memberId1, newAssignment); @@ -454,9 +456,9 @@ public class ShareGroupTest { ); // New assignment for member2 to add partition 1 - Assignment newAssignment2 = new Assignment(Collections.singletonMap( + Assignment newAssignment2 = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(1)) + new HashSet<>(List.of(1)) )); shareGroup.updateTargetAssignment(memberId2, newAssignment2); @@ -469,9 +471,9 @@ public class ShareGroupTest { ); // New assignment for member1 to revoke partition 1 and assign partition 0 - Assignment newAssignment1 = new Assignment(Collections.singletonMap( + Assignment newAssignment1 = new Assignment(Map.of( topicId, - new HashSet<>(Collections.singletonList(0)) + new HashSet<>(List.of(0)) )); shareGroup.updateTargetAssignment(memberId1, newAssignment1); @@ -563,7 +565,7 @@ public class ShareGroupTest { assertEquals(ShareGroupState.EMPTY, shareGroup.state(0)); assertEquals("Empty", shareGroup.stateAsString(0)); shareGroup.updateMember(new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build()); snapshotRegistry.idempotentCreateSnapshot(1); assertEquals(ShareGroupState.EMPTY, shareGroup.state(0)); @@ -630,10 +632,10 @@ public class ShareGroupTest { .build(); ShareGroupMember member1 = new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build(); ShareGroupMember member2 = new ShareGroupMember.Builder("member2") - .setSubscribedTopicNames(Collections.singletonList("bar")) + .setSubscribedTopicNames(List.of("bar")) .build(); ShareGroup shareGroup = createShareGroup("group-foo"); @@ -671,7 +673,7 @@ public class ShareGroupTest { assertEquals(ShareGroupState.EMPTY.toString(), shareGroup.stateAsString(0)); shareGroup.updateMember(new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build()); shareGroup.updateMember(new ShareGroupMember.Builder("member2") .build()); @@ -686,7 +688,7 @@ public class ShareGroupTest { .setMembers(Arrays.asList( new ShareGroupDescribeResponseData.Member() .setMemberId("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")), + .setSubscribedTopicNames(List.of("foo")), new ShareGroupDescribeResponseData.Member().setMemberId("member2") )); ShareGroupDescribeResponseData.DescribedGroup actual = shareGroup.asDescribedGroup(1, "assignorName", @@ -700,16 +702,16 @@ public class ShareGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); ShareGroup shareGroup = new ShareGroup(snapshotRegistry, "group-foo"); snapshotRegistry.idempotentCreateSnapshot(0); - assertTrue(shareGroup.isInStates(Collections.singleton("empty"), 0)); - assertFalse(shareGroup.isInStates(Collections.singleton("Empty"), 0)); + assertTrue(shareGroup.isInStates(Set.of("empty"), 0)); + assertFalse(shareGroup.isInStates(Set.of("Empty"), 0)); shareGroup.updateMember(new ShareGroupMember.Builder("member1") - .setSubscribedTopicNames(Collections.singletonList("foo")) + .setSubscribedTopicNames(List.of("foo")) .build()); snapshotRegistry.idempotentCreateSnapshot(1); - assertTrue(shareGroup.isInStates(Collections.singleton("empty"), 0)); - assertTrue(shareGroup.isInStates(Collections.singleton("stable"), 1)); - assertFalse(shareGroup.isInStates(Collections.singleton("empty"), 1)); + assertTrue(shareGroup.isInStates(Set.of("empty"), 0)); + assertTrue(shareGroup.isInStates(Set.of("stable"), 1)); + assertFalse(shareGroup.isInStates(Set.of("empty"), 1)); } private ShareGroup createShareGroup(String groupId) { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java index 4d55a8419fb..075795f6e11 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java @@ -24,7 +24,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import java.util.Collections; +import java.util.Set; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; @@ -63,8 +63,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 3, 4))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -108,8 +108,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 3, 4))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -150,8 +150,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -191,8 +191,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 4, 5))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -235,8 +235,8 @@ public class CurrentAssignmentBuilderTest { .withTargetAssignment(memberEpoch + 1, TasksTuple.EMPTY) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -278,8 +278,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -321,8 +321,8 @@ public class CurrentAssignmentBuilderTest { SUBTOPOLOGY_ID2.equals(subtopologyId) ? PROCESS_ID : null ) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .withOwnedAssignment(mkTasksTuple(taskRole)) .build(); @@ -365,8 +365,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 5, 6))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .withOwnedAssignment(mkTasksTuple(taskRole, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6))) @@ -412,9 +412,9 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 6))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) .withCurrentWarmupTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()); + (subtopologyId, partitionId) -> Set.of()); assertEquals( member, @@ -511,8 +511,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .withOwnedAssignment(mkTasksTuple(taskRole, mkTasks(SUBTOPOLOGY_ID1, 2, 3), mkTasks(SUBTOPOLOGY_ID2, 5, 6)) @@ -556,9 +556,9 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 5, 6))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + (subtopologyId, partitionId) -> Set.of(PROCESS_ID)) .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> - Collections.singleton(PROCESS_ID)) + Set.of(PROCESS_ID)) .build(); assertEquals( @@ -598,8 +598,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -639,9 +639,9 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + (subtopologyId, partitionId) -> Set.of(PROCESS_ID)) .withCurrentWarmupTaskProcessIds( - (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + (subtopologyId, partitionId) -> Set.of(PROCESS_ID)) .build(); assertEquals(member, updatedMember); @@ -671,10 +671,10 @@ public class CurrentAssignmentBuilderTest { .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) .withCurrentStandbyTaskProcessIds( (subtopologyId, partitionId) -> (taskRole == TaskRole.STANDBY) - ? Collections.emptySet() : Collections.singleton(PROCESS_ID)) + ? Set.of() : Set.of(PROCESS_ID)) .withCurrentWarmupTaskProcessIds( (subtopologyId, partitionId) -> (taskRole == TaskRole.STANDBY) - ? Collections.singleton(PROCESS_ID) : Collections.emptySet()) + ? Set.of(PROCESS_ID) : Set.of()) .build(); assertEquals(member, updatedMember); @@ -716,9 +716,9 @@ public class CurrentAssignmentBuilderTest { (subtopologyId.equals(SUBTOPOLOGY_ID1) && partitionId == 4) ? "anyOtherProcess" : null) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) .withCurrentWarmupTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) .build(); assertEquals(expectedMember, updatedMember); @@ -748,8 +748,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 6))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .build(); assertEquals( @@ -803,8 +803,8 @@ public class CurrentAssignmentBuilderTest { mkTasks(SUBTOPOLOGY_ID2, 6))) .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) .withCurrentStandbyTaskProcessIds( - (subtopologyId, partitionId) -> Collections.emptySet()) - .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + (subtopologyId, partitionId) -> Set.of()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Set.of()) .withOwnedAssignment(mkTasksTuple(taskRole)) .build(); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java index f6c33df6f13..023e491d646 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java @@ -155,7 +155,7 @@ public class StreamsGroupMemberTest { .setUserEndpoint(USER_ENDPOINT) .setClientTags(CLIENT_TAGS.entrySet().stream() .map(e -> new KeyValue().setKey(e.getKey()).setValue(e.getValue())) - .collect(Collectors.toList())); + .toList()); StreamsGroupMember member = new StreamsGroupMember.Builder("member-id") .updateWith(record) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java index 3614a590265..a2d30c2db44 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java @@ -68,7 +68,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.stream.Collectors; -import static java.util.Collections.emptyMap; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; @@ -221,19 +220,19 @@ public class StreamsGroupTest { streamsGroup.updateMember(member); assertEquals("process", streamsGroup.currentActiveTaskProcessId(fooSubtopology, 1)); - assertEquals(Collections.singleton("process"), + assertEquals(Set.of("process"), streamsGroup.currentStandbyTaskProcessIds(fooSubtopology, 2)); - assertEquals(Collections.singleton("process"), + assertEquals(Set.of("process"), streamsGroup.currentWarmupTaskProcessIds(fooSubtopology, 3)); assertEquals("process", streamsGroup.currentActiveTaskProcessId(barSubtopology, 4)); - assertEquals(Collections.singleton("process"), + assertEquals(Set.of("process"), streamsGroup.currentStandbyTaskProcessIds(barSubtopology, 5)); - assertEquals(Collections.singleton("process"), + assertEquals(Set.of("process"), streamsGroup.currentWarmupTaskProcessIds(barSubtopology, 6)); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 7)); - assertEquals(Collections.emptySet(), + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); - assertEquals(Collections.emptySet(), + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); member = new StreamsGroupMember.Builder(member) @@ -257,19 +256,19 @@ public class StreamsGroupTest { streamsGroup.updateMember(member); assertEquals("process1", streamsGroup.currentActiveTaskProcessId(fooSubtopology, 1)); - assertEquals(Collections.singleton("process1"), + assertEquals(Set.of("process1"), streamsGroup.currentStandbyTaskProcessIds(fooSubtopology, 2)); - assertEquals(Collections.singleton("process1"), + assertEquals(Set.of("process1"), streamsGroup.currentWarmupTaskProcessIds(fooSubtopology, 3)); assertEquals("process1", streamsGroup.currentActiveTaskProcessId(barSubtopology, 4)); - assertEquals(Collections.singleton("process1"), + assertEquals(Set.of("process1"), streamsGroup.currentStandbyTaskProcessIds(barSubtopology, 5)); - assertEquals(Collections.singleton("process1"), + assertEquals(Set.of("process1"), streamsGroup.currentWarmupTaskProcessIds(barSubtopology, 6)); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 7)); - assertEquals(Collections.emptySet(), + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); - assertEquals(Collections.emptySet(), + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); } @@ -284,9 +283,9 @@ public class StreamsGroupTest { .setProcessId("process") .setAssignedTasks( new TasksTuple( - emptyMap(), - emptyMap(), - emptyMap() + Map.of(), + Map.of(), + Map.of() ) ) .setTasksPendingRevocation( @@ -329,8 +328,8 @@ public class StreamsGroupTest { .setAssignedTasks( new TasksTuple( mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), - emptyMap(), - emptyMap() + Map.of(), + Map.of() ) ) .build(); @@ -342,8 +341,8 @@ public class StreamsGroupTest { .setAssignedTasks( new TasksTuple( mkTasksPerSubtopology(mkTasks(fooSubtopologyId, 1)), - emptyMap(), - emptyMap() + Map.of(), + Map.of() ) ) .build(); @@ -436,26 +435,26 @@ public class StreamsGroupTest { streamsGroup.updateMember(member); assertEquals("process", streamsGroup.currentActiveTaskProcessId(fooSubtopology, 1)); - assertEquals(Collections.singleton("process"), streamsGroup.currentStandbyTaskProcessIds(fooSubtopology, 2)); - assertEquals(Collections.singleton("process"), streamsGroup.currentWarmupTaskProcessIds(fooSubtopology, 3)); + assertEquals(Set.of("process"), streamsGroup.currentStandbyTaskProcessIds(fooSubtopology, 2)); + assertEquals(Set.of("process"), streamsGroup.currentWarmupTaskProcessIds(fooSubtopology, 3)); assertEquals("process", streamsGroup.currentActiveTaskProcessId(barSubtopology, 4)); - assertEquals(Collections.singleton("process"), streamsGroup.currentStandbyTaskProcessIds(barSubtopology, 5)); - assertEquals(Collections.singleton("process"), streamsGroup.currentWarmupTaskProcessIds(barSubtopology, 6)); + assertEquals(Set.of("process"), streamsGroup.currentStandbyTaskProcessIds(barSubtopology, 5)); + assertEquals(Set.of("process"), streamsGroup.currentWarmupTaskProcessIds(barSubtopology, 6)); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 7)); - assertEquals(Collections.emptySet(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); - assertEquals(Collections.emptySet(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); streamsGroup.removeMember(member.memberId()); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 1)); - assertEquals(Collections.emptySet(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 2)); - assertEquals(Collections.emptySet(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 3)); + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 2)); + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 3)); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 3)); - assertEquals(Collections.emptySet(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 4)); - assertEquals(Collections.emptySet(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 5)); + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 4)); + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 5)); assertNull(streamsGroup.currentActiveTaskProcessId(zarSubtopology, 7)); - assertEquals(Collections.emptySet(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); - assertEquals(Collections.emptySet(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); + assertEquals(Set.of(), streamsGroup.currentStandbyTaskProcessIds(zarSubtopology, 8)); + assertEquals(Set.of(), streamsGroup.currentWarmupTaskProcessIds(zarSubtopology, 9)); } @Test @@ -475,7 +474,7 @@ public class StreamsGroupTest { assertEquals(MemberState.STABLE, member1.state()); assertEquals(StreamsGroup.StreamsGroupState.NOT_READY, streamsGroup.state()); - streamsGroup.setTopology(new StreamsTopology(1, Collections.emptyMap())); + streamsGroup.setTopology(new StreamsTopology(1, Map.of())); assertEquals(MemberState.STABLE, member1.state()); assertEquals(StreamsGroup.StreamsGroupState.ASSIGNING, streamsGroup.state()); @@ -672,7 +671,7 @@ public class StreamsGroupTest { mock(GroupCoordinatorMetricsShard.class) ); group.setGroupEpoch(1); - group.setTopology(new StreamsTopology(1, Collections.emptyMap())); + group.setTopology(new StreamsTopology(1, Map.of())); group.setTargetAssignmentEpoch(1); group.updateMember(new StreamsGroupMember.Builder("member1") .setMemberEpoch(1) @@ -737,7 +736,7 @@ public class StreamsGroupTest { assertEquals(StreamsGroup.StreamsGroupState.NOT_READY, streamsGroup.state()); assertThrows(GroupNotEmptyException.class, streamsGroup::validateDeleteGroup); - streamsGroup.setTopology(new StreamsTopology(1, Collections.emptyMap())); + streamsGroup.setTopology(new StreamsTopology(1, Map.of())); assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, streamsGroup.state()); assertThrows(GroupNotEmptyException.class, streamsGroup::validateDeleteGroup); @@ -781,7 +780,7 @@ public class StreamsGroupTest { assertEquals(StreamsGroup.StreamsGroupState.EMPTY.toString(), group.stateAsString(0)); group.setGroupEpoch(1); - group.setTopology(new StreamsTopology(1, Collections.emptyMap())); + group.setTopology(new StreamsTopology(1, Map.of())); group.setTargetAssignmentEpoch(1); group.updateMember(new StreamsGroupMember.Builder("member1") .setMemberEpoch(1) @@ -795,9 +794,9 @@ public class StreamsGroupTest { .setTopologyEpoch(1) .setProcessId("process1") .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host1").setPort(9092)) - .setClientTags(Collections.singletonMap("tag1", "value1")) - .setAssignedTasks(new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())) - .setTasksPendingRevocation(new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())) + .setClientTags(Map.of("tag1", "value1")) + .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) .build()); group.updateMember(new StreamsGroupMember.Builder("member2") .setMemberEpoch(1) @@ -811,9 +810,9 @@ public class StreamsGroupTest { .setTopologyEpoch(1) .setProcessId("process2") .setUserEndpoint(new StreamsGroupMemberMetadataValue.Endpoint().setHost("host2").setPort(9092)) - .setClientTags(Collections.singletonMap("tag2", "value2")) - .setAssignedTasks(new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())) - .setTasksPendingRevocation(new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap())) + .setClientTags(Map.of("tag2", "value2")) + .setAssignedTasks(new TasksTuple(Map.of(), Map.of(), Map.of())) + .setTasksPendingRevocation(new TasksTuple(Map.of(), Map.of(), Map.of())) .build()); snapshotRegistry.idempotentCreateSnapshot(1); @@ -821,7 +820,7 @@ public class StreamsGroupTest { .setGroupId("group-id-1") .setGroupState(StreamsGroup.StreamsGroupState.STABLE.toString()) .setGroupEpoch(1) - .setTopology(new StreamsGroupDescribeResponseData.Topology().setEpoch(1).setSubtopologies(Collections.emptyList())) + .setTopology(new StreamsGroupDescribeResponseData.Topology().setEpoch(1).setSubtopologies(List.of())) .setAssignmentEpoch(1) .setMembers(Arrays.asList( new StreamsGroupDescribeResponseData.Member() @@ -834,7 +833,7 @@ public class StreamsGroupTest { .setTopologyEpoch(1) .setProcessId("process1") .setUserEndpoint(new StreamsGroupDescribeResponseData.Endpoint().setHost("host1").setPort(9092)) - .setClientTags(Collections.singletonList(new StreamsGroupDescribeResponseData.KeyValue().setKey("tag1").setValue("value1"))) + .setClientTags(List.of(new StreamsGroupDescribeResponseData.KeyValue().setKey("tag1").setValue("value1"))) .setAssignment(new StreamsGroupDescribeResponseData.Assignment()) .setTargetAssignment(new StreamsGroupDescribeResponseData.Assignment()), new StreamsGroupDescribeResponseData.Member() @@ -847,7 +846,7 @@ public class StreamsGroupTest { .setTopologyEpoch(1) .setProcessId("process2") .setUserEndpoint(new StreamsGroupDescribeResponseData.Endpoint().setHost("host2").setPort(9092)) - .setClientTags(Collections.singletonList(new StreamsGroupDescribeResponseData.KeyValue().setKey("tag2").setValue("value2"))) + .setClientTags(List.of(new StreamsGroupDescribeResponseData.KeyValue().setKey("tag2").setValue("value2"))) .setAssignment(new StreamsGroupDescribeResponseData.Assignment()) .setTargetAssignment(new StreamsGroupDescribeResponseData.Assignment()) )); @@ -861,20 +860,20 @@ public class StreamsGroupTest { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT); GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( snapshotRegistry, - emptyMap(), + Map.of(), new TopicPartition("__consumer_offsets", 0) ); StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo", metricsShard); snapshotRegistry.idempotentCreateSnapshot(0); - assertTrue(group.isInStates(Collections.singleton("empty"), 0)); - assertFalse(group.isInStates(Collections.singleton("Empty"), 0)); + assertTrue(group.isInStates(Set.of("empty"), 0)); + assertFalse(group.isInStates(Set.of("Empty"), 0)); group.updateMember(new StreamsGroupMember.Builder("member1") .build()); snapshotRegistry.idempotentCreateSnapshot(1); - assertTrue(group.isInStates(Collections.singleton("empty"), 0)); - assertTrue(group.isInStates(Collections.singleton("not_ready"), 1)); - assertFalse(group.isInStates(Collections.singleton("empty"), 1)); + assertTrue(group.isInStates(Set.of("empty"), 0)); + assertTrue(group.isInStates(Set.of("not_ready"), 1)); + assertFalse(group.isInStates(Set.of("empty"), 1)); } @Test @@ -883,7 +882,7 @@ public class StreamsGroupTest { GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); StreamsGroup streamsGroup = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "test-group", metricsShard); - StreamsTopology topology = new StreamsTopology(1, Collections.emptyMap()); + StreamsTopology topology = new StreamsTopology(1, Map.of()); ConfiguredTopology topo = mock(ConfiguredTopology.class); when(topo.isReady()).thenReturn(true); @@ -925,7 +924,7 @@ public class StreamsGroupTest { assertTrue(streamsGroup.configuredTopology().isEmpty(), "Configured topology should not be present"); assertEquals(partitionMetadata, streamsGroup.partitionMetadata()); - StreamsTopology topology = new StreamsTopology(1, Collections.emptyMap()); + StreamsTopology topology = new StreamsTopology(1, Map.of()); ConfiguredTopology topo = mock(ConfiguredTopology.class); when(topo.isReady()).thenReturn(true); try (MockedStatic mocked = mockStatic(InternalTopicManager.class)) { @@ -955,7 +954,7 @@ public class StreamsGroupTest { assertTrue(streamsGroup.configuredTopology().isEmpty(), "Configured topology should not be present"); assertEquals(partitionMetadata, streamsGroup.partitionMetadata()); - StreamsTopology topology = new StreamsTopology(1, Collections.emptyMap()); + StreamsTopology topology = new StreamsTopology(1, Map.of()); streamsGroup.setTopology(topology); ConfiguredTopology topo = mock(ConfiguredTopology.class); when(topo.isReady()).thenReturn(true); @@ -995,7 +994,7 @@ public class StreamsGroupTest { when(topicImage.partitions()).thenReturn(Collections.singletonMap(0, null)); when(topicsImage.getTopic("topic1")).thenReturn(topicImage); StreamsTopology topology = mock(StreamsTopology.class); - when(topology.requiredTopics()).thenReturn(Collections.singleton("topic1")); + when(topology.requiredTopics()).thenReturn(Set.of("topic1")); Map partitionMetadata = streamsGroup.computePartitionMetadata(topicsImage, topology); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java index 0380d4cf5e7..8b51b1b58b4 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java @@ -33,7 +33,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -74,12 +73,12 @@ public class TargetAssignmentBuilderTest { TargetAssignmentBuilder.TargetAssignmentResult result = builder.build(); - List expectedRecords = Collections.singletonList( + List expectedRecords = List.of( StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, groupEpoch) ); assertEquals(expectedRecords, result.records()); - assertEquals(Collections.emptyMap(), result.targetAssignment()); + assertEquals(Map.of(), result.targetAssignment()); } @ParameterizedTest diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java index 73c43a6d088..836e42fb6e9 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java @@ -22,7 +22,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -42,9 +41,9 @@ public class TasksTupleTest { @Test public void testTasksCannotBeNull() { - assertThrows(NullPointerException.class, () -> new TasksTuple(null, Collections.emptyMap(), Collections.emptyMap())); - assertThrows(NullPointerException.class, () -> new TasksTuple(Collections.emptyMap(), null, Collections.emptyMap())); - assertThrows(NullPointerException.class, () -> new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), null)); + assertThrows(NullPointerException.class, () -> new TasksTuple(null, Map.of(), Map.of())); + assertThrows(NullPointerException.class, () -> new TasksTuple(Map.of(), null, Map.of())); + assertThrows(NullPointerException.class, () -> new TasksTuple(Map.of(), Map.of(), null)); } @Test @@ -61,11 +60,11 @@ public class TasksTupleTest { TasksTuple tuple = new TasksTuple(activeTasks, standbyTasks, warmupTasks); assertEquals(activeTasks, tuple.activeTasks()); - assertThrows(UnsupportedOperationException.class, () -> tuple.activeTasks().put("not allowed", Collections.emptySet())); + assertThrows(UnsupportedOperationException.class, () -> tuple.activeTasks().put("not allowed", Set.of())); assertEquals(standbyTasks, tuple.standbyTasks()); - assertThrows(UnsupportedOperationException.class, () -> tuple.standbyTasks().put("not allowed", Collections.emptySet())); + assertThrows(UnsupportedOperationException.class, () -> tuple.standbyTasks().put("not allowed", Set.of())); assertEquals(warmupTasks, tuple.warmupTasks()); - assertThrows(UnsupportedOperationException.class, () -> tuple.warmupTasks().put("not allowed", Collections.emptySet())); + assertThrows(UnsupportedOperationException.class, () -> tuple.warmupTasks().put("not allowed", Set.of())); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/GroupSpecImplTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/GroupSpecImplTest.java index 5deccb9717f..be2aa696e8f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/GroupSpecImplTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/GroupSpecImplTest.java @@ -19,7 +19,6 @@ package org.apache.kafka.coordinator.group.streams.assignor; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -39,13 +38,13 @@ public class GroupSpecImplTest { members.put("test-member", new AssignmentMemberSpec( Optional.of("test-instance"), Optional.of("test-rack"), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() )); groupSpec = new GroupSpecImpl( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/MockAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/MockAssignorTest.java index d44b24549e0..8584716065f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/MockAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/MockAssignorTest.java @@ -18,7 +18,6 @@ package org.apache.kafka.coordinator.group.streams.assignor; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -43,10 +42,10 @@ public class MockAssignorTest { TaskAssignorException ex = assertThrows(TaskAssignorException.class, () -> assignor.assign( new GroupSpecImpl( - Collections.emptyMap(), + Map.of(), new HashMap<>() ), - new TopologyDescriberImpl(5, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(5, List.of("test-subtopology")) )); assertEquals("No member available to assign task 0 of subtopology test-subtopology", ex.getMessage()); @@ -58,25 +57,25 @@ public class MockAssignorTest { final AssignmentMemberSpec memberSpec1 = new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonMap("test-subtopology", new HashSet<>(List.of(0))), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of("test-subtopology", new HashSet<>(List.of(0))), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final AssignmentMemberSpec memberSpec2 = new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.singletonMap("test-subtopology", new HashSet<>(List.of(0))), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of("test-subtopology", new HashSet<>(List.of(0))), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); TaskAssignorException ex = assertThrows(TaskAssignorException.class, () -> assignor.assign( @@ -84,7 +83,7 @@ public class MockAssignorTest { Map.of("member1", memberSpec1, "member2", memberSpec2), new HashMap<>() ), - new TopologyDescriberImpl(5, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(5, List.of("test-subtopology")) )); assertEquals("Task 0 of subtopology test-subtopology is assigned to multiple members", ex.getMessage()); @@ -95,10 +94,10 @@ public class MockAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl( - Collections.emptyMap(), + Map.of(), new HashMap<>() ), - new TopologyDescriberImpl(5, Collections.emptyList()) + new TopologyDescriberImpl(5, List.of()) ); assertEquals(0, result.members().size()); @@ -111,18 +110,18 @@ public class MockAssignorTest { final AssignmentMemberSpec memberSpec = new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final GroupAssignment result = assignor.assign( new GroupSpecImpl( - Collections.singletonMap("test_member", memberSpec), + Map.of("test_member", memberSpec), new HashMap<>() ), new TopologyDescriberImpl(4, List.of("test-subtopology")) @@ -143,25 +142,25 @@ public class MockAssignorTest { final AssignmentMemberSpec memberSpec1 = new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final AssignmentMemberSpec memberSpec2 = new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final GroupAssignment result = assignor.assign( @@ -200,12 +199,12 @@ public class MockAssignorTest { mkEntry("test-subtopology1", new HashSet<>(List.of(0, 2, 3))), mkEntry("test-subtopology2", new HashSet<>(List.of(0))) ), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final AssignmentMemberSpec memberSpec2 = new AssignmentMemberSpec( @@ -215,12 +214,12 @@ public class MockAssignorTest { mkEntry("test-subtopology1", new HashSet<>(List.of(1))), mkEntry("test-subtopology2", new HashSet<>(List.of(3))) ), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), "test-process", - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap() + Map.of(), + Map.of(), + Map.of() ); final GroupAssignment result = assignor.assign( new GroupSpecImpl( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java index 542aad73dee..c0493887266 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/assignor/StickyTaskAssignorTest.java @@ -22,7 +22,6 @@ import org.mockito.internal.util.collections.Sets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -59,7 +58,7 @@ public class StickyTaskAssignorTest { mkMap(mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)), new HashMap<>() ), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); assertEquals(3, result.members().size()); @@ -132,13 +131,13 @@ public class StickyTaskAssignorTest { @Test public void shouldNotMigrateActiveTaskToOtherProcess() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Collections.singleton(0))), Collections.emptyMap()); - AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Set.of(0))), Map.of()); + AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); Map members = mkMap(mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); @@ -151,13 +150,13 @@ public class StickyTaskAssignorTest { testMember1.activeTasks().get("test-subtopology").size() + testMember2.activeTasks().get("test-subtopology").size()); // flip the previous active tasks assignment around. - memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Collections.singleton(2))), Collections.emptyMap()); + memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Set.of(2))), Map.of()); members = mkMap(mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); testMember2 = result.members().get("member2"); @@ -172,15 +171,15 @@ public class StickyTaskAssignorTest { @Test public void shouldMigrateActiveTasksToNewProcessWithoutChangingAllAssignments() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 2))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 2))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3"); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); @@ -208,7 +207,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); @@ -227,7 +226,7 @@ public class StickyTaskAssignorTest { final Map> activeTasks = mkMap( mkEntry("test-subtopology1", Sets.newSet(0, 1, 2, 3, 4, 5)), mkEntry("test-subtopology2", Sets.newSet(0))); - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", activeTasks, Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", activeTasks, Map.of()); final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); @@ -254,9 +253,9 @@ public class StickyTaskAssignorTest { @Test public void shouldKeepActiveTaskStickinessWhenMoreClientThanActiveTasks() { - AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Collections.singleton(0))), Collections.emptyMap()); - AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(2))), Collections.emptyMap()); - AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); + AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Set.of(0))), Map.of()); + AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(2))), Map.of()); + AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); AssignmentMemberSpec memberSpec4 = createAssignmentMemberSpec("process4"); AssignmentMemberSpec memberSpec5 = createAssignmentMemberSpec("process5"); Map members = mkMap( @@ -265,21 +264,21 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); assertNotNull(testMember1); assertEquals(1, testMember1.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember1.activeTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember1.activeTasks().get("test-subtopology")); MemberAssignment testMember2 = result.members().get("member2"); assertNotNull(testMember2); assertEquals(1, testMember2.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(2), testMember2.activeTasks().get("test-subtopology")); + assertEquals(Set.of(2), testMember2.activeTasks().get("test-subtopology")); MemberAssignment testMember3 = result.members().get("member3"); assertNotNull(testMember3); assertEquals(1, testMember3.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember3.activeTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember3.activeTasks().get("test-subtopology")); MemberAssignment testMember4 = result.members().get("member4"); assertNotNull(testMember4); assertNull(testMember4.activeTasks().get("test-subtopology")); @@ -289,17 +288,17 @@ public class StickyTaskAssignorTest { // change up the assignment and make sure it is still sticky memberSpec1 = createAssignmentMemberSpec("process1"); - memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(0))), Collections.emptyMap()); + memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(0))), Map.of()); memberSpec3 = createAssignmentMemberSpec("process3"); - memberSpec4 = createAssignmentMemberSpec("process4", mkMap(mkEntry("test-subtopology", Collections.singleton(2))), Collections.emptyMap()); - memberSpec5 = createAssignmentMemberSpec("process5", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); + memberSpec4 = createAssignmentMemberSpec("process4", mkMap(mkEntry("test-subtopology", Set.of(2))), Map.of()); + memberSpec5 = createAssignmentMemberSpec("process5", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3), mkEntry("member4", memberSpec4), mkEntry("member5", memberSpec5)); result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); testMember1 = result.members().get("member1"); @@ -308,113 +307,113 @@ public class StickyTaskAssignorTest { testMember2 = result.members().get("member2"); assertNotNull(testMember2); assertEquals(1, testMember2.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember2.activeTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember2.activeTasks().get("test-subtopology")); testMember3 = result.members().get("member3"); assertNotNull(testMember3); assertNull(testMember3.activeTasks().get("test-subtopology")); testMember4 = result.members().get("member4"); assertNotNull(testMember4); assertEquals(1, testMember4.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(2), testMember4.activeTasks().get("test-subtopology")); + assertEquals(Set.of(2), testMember4.activeTasks().get("test-subtopology")); testMember5 = result.members().get("member5"); assertNotNull(testMember5); assertEquals(1, testMember5.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember5.activeTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember5.activeTasks().get("test-subtopology")); } @Test public void shouldAssignTasksToClientWithPreviousStandbyTasks() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Collections.singleton(2)))); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Collections.singleton(1)))); - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Collections.singleton(0)))); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", Map.of(), mkMap(mkEntry("test-subtopology", Set.of(2)))); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Map.of(), mkMap(mkEntry("test-subtopology", Set.of(1)))); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", Map.of(), mkMap(mkEntry("test-subtopology", Set.of(0)))); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); assertNotNull(testMember1); assertEquals(1, testMember1.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(2), testMember1.activeTasks().get("test-subtopology")); + assertEquals(Set.of(2), testMember1.activeTasks().get("test-subtopology")); MemberAssignment testMember2 = result.members().get("member2"); assertNotNull(testMember2); assertEquals(1, testMember2.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember2.activeTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember2.activeTasks().get("test-subtopology")); MemberAssignment testMember3 = result.members().get("member3"); assertNotNull(testMember3); assertEquals(1, testMember3.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember3.activeTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember3.activeTasks().get("test-subtopology")); } @Test public void shouldNotAssignStandbyTasksToClientWithPreviousStandbyTasksAndCurrentActiveTasks() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Collections.singleton(0)))); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Collections.singleton(1)))); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", Map.of(), mkMap(mkEntry("test-subtopology", Set.of(0)))); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Map.of(), mkMap(mkEntry("test-subtopology", Set.of(1)))); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(2, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(2, true, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); assertNotNull(testMember1); assertEquals(1, testMember1.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember1.activeTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember1.activeTasks().get("test-subtopology")); assertEquals(1, testMember1.standbyTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember1.standbyTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember1.standbyTasks().get("test-subtopology")); MemberAssignment testMember2 = result.members().get("member2"); assertNotNull(testMember2); assertEquals(1, testMember2.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember2.activeTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember2.activeTasks().get("test-subtopology")); assertEquals(1, testMember2.standbyTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember2.standbyTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember2.standbyTasks().get("test-subtopology")); } @Test public void shouldAssignBasedOnCapacityWhenMultipleClientHaveStandbyTasks() { final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", - mkMap(mkEntry("test-subtopology", Collections.singleton(0))), - mkMap(mkEntry("test-subtopology", Collections.singleton(1)))); + mkMap(mkEntry("test-subtopology", Set.of(0))), + mkMap(mkEntry("test-subtopology", Set.of(1)))); final AssignmentMemberSpec memberSpec21 = createAssignmentMemberSpec("process2", - mkMap(mkEntry("test-subtopology", Collections.singleton(2))), - mkMap(mkEntry("test-subtopology", Collections.singleton(1)))); + mkMap(mkEntry("test-subtopology", Set.of(2))), + mkMap(mkEntry("test-subtopology", Set.of(1)))); final AssignmentMemberSpec memberSpec22 = createAssignmentMemberSpec("process2", - Collections.emptyMap(), Collections.emptyMap()); + Map.of(), Map.of()); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2_1", memberSpec21), mkEntry("member2_2", memberSpec22)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); MemberAssignment testMember1 = result.members().get("member1"); assertNotNull(testMember1); assertEquals(1, testMember1.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(0), testMember1.activeTasks().get("test-subtopology")); + assertEquals(Set.of(0), testMember1.activeTasks().get("test-subtopology")); MemberAssignment testMember21 = result.members().get("member2_1"); assertNotNull(testMember21); assertEquals(1, testMember21.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(2), testMember21.activeTasks().get("test-subtopology")); + assertEquals(Set.of(2), testMember21.activeTasks().get("test-subtopology")); MemberAssignment testMember22 = result.members().get("member2_2"); assertNotNull(testMember22); assertEquals(1, testMember22.activeTasks().get("test-subtopology").size()); - assertEquals(Collections.singleton(1), testMember22.activeTasks().get("test-subtopology")); + assertEquals(Set.of(1), testMember22.activeTasks().get("test-subtopology")); } @Test public void shouldAssignStandbyTasksToDifferentClientThanCorrespondingActiveTaskIsAssignedTo() { final Map> tasks = mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 3))); - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Collections.singleton(0))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Collections.singleton(2))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec4 = createAssignmentMemberSpec("process4", mkMap(mkEntry("test-subtopology", Collections.singleton(3))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Set.of(0))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Set.of(2))), Map.of()); + final AssignmentMemberSpec memberSpec4 = createAssignmentMemberSpec("process4", mkMap(mkEntry("test-subtopology", Set.of(3))), Map.of()); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3), mkEntry("member4", memberSpec4)); @@ -422,7 +421,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(4, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, true, List.of("test-subtopology")) ); final List member1TaskIds = getAllStandbyTaskIds(result, "member1"); @@ -448,9 +447,9 @@ public class StickyTaskAssignorTest { @Test public void shouldAssignMultipleReplicasOfStandbyTask() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Collections.singleton(0))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Collections.singleton(1))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Collections.singleton(2))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Set.of(0))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Set.of(1))), Map.of()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Set.of(2))), Map.of()); Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); @@ -458,7 +457,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "2"))), - new TopologyDescriberImpl(3, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, true, List.of("test-subtopology")) ); assertEquals(Sets.newSet(1, 2), new HashSet<>(getAllStandbyTaskIds(result, "member1"))); @@ -475,7 +474,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(1, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(1, true, List.of("test-subtopology")) ); assertTrue(getAllStandbyTasks(result, "member1").isEmpty()); @@ -493,7 +492,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(3, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, true, List.of("test-subtopology")) ); assertEquals(Sets.newSet(0, 1, 2), new HashSet<>(getAllActiveTaskIds(result))); @@ -513,7 +512,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskIds(result, "member1_1", "member1_2", "member1_3").size()); @@ -535,7 +534,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); assertEquals(3, getAllActiveTaskIds(result, "member1", "member2", "member3", "member4", "member5", "member6").size()); @@ -557,7 +556,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(3, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, true, List.of("test-subtopology")) ); for (String memberId : result.members().keySet()) { @@ -595,7 +594,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(4, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, true, List.of("test-subtopology")) ); for (final String memberId : allMemberIds) { @@ -610,9 +609,9 @@ public class StickyTaskAssignorTest { @Test public void shouldNotHaveSameAssignmentOnAnyTwoHostsWhenThereArePreviousActiveTasks() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(1, 2))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(3))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(1, 2))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(3))), Map.of()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0))), Map.of()); final AssignmentMemberSpec memberSpec4 = createAssignmentMemberSpec("process4"); final List allMemberIds = asList("member1", "member2", "member3", "member4"); Map members = mkMap( @@ -621,7 +620,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(4, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, true, List.of("test-subtopology")) ); for (final String memberId : allMemberIds) { @@ -649,7 +648,7 @@ public class StickyTaskAssignorTest { final GroupAssignment result = assignor.assign( new GroupSpecImpl(members, mkMap(mkEntry(NUM_STANDBY_REPLICAS_CONFIG, "1"))), - new TopologyDescriberImpl(4, true, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, true, List.of("test-subtopology")) ); for (final String memberId : allMemberIds) { @@ -664,7 +663,7 @@ public class StickyTaskAssignorTest { @Test public void shouldReBalanceTasksAcrossAllClientsWhenCapacityAndTaskCountTheSame() { - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 3))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 3))), Map.of()); final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1"); final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); final AssignmentMemberSpec memberSpec4 = createAssignmentMemberSpec("process4"); @@ -673,7 +672,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(4, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskCount(result, "member1")); @@ -684,7 +683,7 @@ public class StickyTaskAssignorTest { @Test public void shouldReBalanceTasksAcrossClientsWhenCapacityLessThanTaskCount() { - final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 3))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 3))), Map.of()); final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1"); final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); Map members = mkMap( @@ -692,7 +691,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(4, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskCount(result, "member1")); @@ -702,7 +701,7 @@ public class StickyTaskAssignorTest { @Test public void shouldRebalanceTasksToClientsBasedOnCapacity() { - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 3, 2))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 3, 2))), Map.of()); final AssignmentMemberSpec memberSpec31 = createAssignmentMemberSpec("process3"); final AssignmentMemberSpec memberSpec32 = createAssignmentMemberSpec("process3"); Map members = mkMap( @@ -710,7 +709,7 @@ public class StickyTaskAssignorTest { GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(3, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(3, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskCount(result, "member2")); @@ -721,15 +720,15 @@ public class StickyTaskAssignorTest { public void shouldMoveMinimalNumberOfTasksWhenPreviouslyAboveCapacityAndNewClientAdded() { final Set p1PrevTasks = Sets.newSet(0, 2); final Set p2PrevTasks = Sets.newSet(1, 3); - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", p1PrevTasks)), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", p2PrevTasks)), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", p1PrevTasks)), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", p2PrevTasks)), Map.of()); final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3"); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(4, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(4, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskCount(result, "member3")); @@ -743,14 +742,14 @@ public class StickyTaskAssignorTest { @Test public void shouldNotMoveAnyTasksWhenNewTasksAdded() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(2, 3))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(2, 3))), Map.of()); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(6, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(6, false, List.of("test-subtopology")) ); final List mem1Tasks = getAllActiveTaskIds(result, "member1"); @@ -763,15 +762,15 @@ public class StickyTaskAssignorTest { @Test public void shouldAssignNewTasksToNewClientWhenPreviousTasksAssignedToOldClients() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(2, 1))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 3))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(2, 1))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 3))), Map.of()); final AssignmentMemberSpec memberSpec3 = createAssignmentMemberSpec("process3"); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(6, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(6, false, List.of("test-subtopology")) ); final List mem1Tasks = getAllActiveTaskIds(result, "member1"); @@ -797,7 +796,7 @@ public class StickyTaskAssignorTest { mkMap(mkEntry("test-subtopology2", Sets.newSet(0, 1, 3))), mkMap(mkEntry("test-subtopology0", Sets.newSet(2)), mkEntry("test-subtopology1", Sets.newSet(2)))); final AssignmentMemberSpec newMemberSpec = createAssignmentMemberSpec("process4", - Collections.emptyMap(), + Map.of(), mkMap(mkEntry("test-subtopology0", Sets.newSet(0, 1, 2, 3)), mkEntry("test-subtopology1", Sets.newSet(0, 1, 2, 3)), mkEntry("test-subtopology2", Sets.newSet(0, 1, 2, 3)))); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("member3", memberSpec3), mkEntry("newMember", newMemberSpec)); @@ -826,10 +825,10 @@ public class StickyTaskAssignorTest { mkMap(mkEntry("test-subtopology0", Sets.newSet(0)), mkEntry("test-subtopology1", Sets.newSet(1)), mkEntry("test-subtopology2", Sets.newSet(2))), mkMap(mkEntry("test-subtopology0", Sets.newSet(1, 2, 3)), mkEntry("test-subtopology1", Sets.newSet(0, 2, 3)), mkEntry("test-subtopology2", Sets.newSet(0, 1, 3)))); final AssignmentMemberSpec bounce1 = createAssignmentMemberSpec("bounce1", - Collections.emptyMap(), + Map.of(), mkMap(mkEntry("test-subtopology2", Sets.newSet(0, 1, 3)))); final AssignmentMemberSpec bounce2 = createAssignmentMemberSpec("bounce2", - Collections.emptyMap(), + Map.of(), mkMap(mkEntry("test-subtopology0", Sets.newSet(2, 3)), mkEntry("test-subtopology1", Sets.newSet(0)))); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("bounce_member1", bounce1), mkEntry("bounce_member2", bounce2)); @@ -851,14 +850,14 @@ public class StickyTaskAssignorTest { @Test public void shouldAssignTasksToNewClient() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(1, 2))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(1, 2))), Map.of()); final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2"); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(2, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(2, false, List.of("test-subtopology")) ); assertEquals(1, getAllActiveTaskCount(result, "member1")); @@ -866,15 +865,15 @@ public class StickyTaskAssignorTest { @Test public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingClients() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(3, 4, 5))), Collections.emptyMap()); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", mkMap(mkEntry("test-subtopology", Sets.newSet(3, 4, 5))), Map.of()); final AssignmentMemberSpec newMemberSpec = createAssignmentMemberSpec("process3"); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("newMember", newMemberSpec)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(6, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(6, false, List.of("test-subtopology")) ); final List mem1Tasks = getAllActiveTaskIds(result, "member1"); @@ -892,15 +891,15 @@ public class StickyTaskAssignorTest { @Test public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExistingAndBouncedClients() { - final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 6))), Collections.emptyMap()); - final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Collections.emptyMap(), mkMap(mkEntry("test-subtopology", Sets.newSet(3, 4, 5)))); + final AssignmentMemberSpec memberSpec1 = createAssignmentMemberSpec("process1", mkMap(mkEntry("test-subtopology", Sets.newSet(0, 1, 2, 6))), Map.of()); + final AssignmentMemberSpec memberSpec2 = createAssignmentMemberSpec("process2", Map.of(), mkMap(mkEntry("test-subtopology", Sets.newSet(3, 4, 5)))); final AssignmentMemberSpec newMemberSpec = createAssignmentMemberSpec("newProcess"); final Map members = mkMap( mkEntry("member1", memberSpec1), mkEntry("member2", memberSpec2), mkEntry("newMember", newMemberSpec)); GroupAssignment result = assignor.assign( new GroupSpecImpl(members, new HashMap<>()), - new TopologyDescriberImpl(7, false, Collections.singletonList("test-subtopology")) + new TopologyDescriberImpl(7, false, List.of("test-subtopology")) ); final List mem1Tasks = getAllActiveTaskIds(result, "member1"); @@ -1054,13 +1053,13 @@ public class StickyTaskAssignorTest { return new AssignmentMemberSpec( Optional.empty(), Optional.empty(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), + Map.of(), + Map.of(), + Map.of(), processId, - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap()); + Map.of(), + Map.of(), + Map.of()); } private AssignmentMemberSpec createAssignmentMemberSpec(final String processId, final Map> prevActiveTasks, @@ -1070,11 +1069,11 @@ public class StickyTaskAssignorTest { Optional.empty(), prevActiveTasks, prevStandbyTasks, - Collections.emptyMap(), + Map.of(), processId, - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap()); + Map.of(), + Map.of(), + Map.of()); } static class TopologyDescriberImpl implements TopologyDescriber { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ChangelogTopicsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ChangelogTopicsTest.java index ab7aec1dce2..511064560ca 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ChangelogTopicsTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ChangelogTopicsTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.To import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.OptionalInt; @@ -46,23 +45,23 @@ public class ChangelogTopicsTest { .setTopicConfigs(List.of(TOPIC_CONFIG)); private static final Subtopology SUBTOPOLOGY_NO_SOURCE = new Subtopology() .setSubtopologyId("SUBTOPOLOGY_NO_SOURCE") - .setSourceTopics(Collections.emptyList()) - .setRepartitionSinkTopics(Collections.singletonList(SINK_TOPIC_NAME)) + .setSourceTopics(List.of()) + .setRepartitionSinkTopics(List.of(SINK_TOPIC_NAME)) .setRepartitionSourceTopics(List.of(REPARTITION_TOPIC_INFO)) - .setStateChangelogTopics(Collections.emptyList()); + .setStateChangelogTopics(List.of()); private static final Subtopology SUBTOPOLOGY_STATELESS = new Subtopology() .setSubtopologyId("SUBTOPOLOGY_STATELESS") - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_NAME)) - .setRepartitionSinkTopics(Collections.singletonList(SINK_TOPIC_NAME)) + .setSourceTopics(List.of(SOURCE_TOPIC_NAME)) + .setRepartitionSinkTopics(List.of(SINK_TOPIC_NAME)) .setRepartitionSourceTopics(List.of(REPARTITION_TOPIC_INFO)) - .setStateChangelogTopics(Collections.emptyList()); + .setStateChangelogTopics(List.of()); private static final TopicInfo SOURCE_CHANGELOG_TOPIC_CONFIG = new TopicInfo() .setName(SOURCE_TOPIC_NAME) .setTopicConfigs(List.of(TOPIC_CONFIG)); private static final Subtopology SUBTOPOLOGY_SOURCE_CHANGELOG = new Subtopology() .setSubtopologyId("SUBTOPOLOGY_SOURCE_CHANGELOG") - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_NAME)) - .setRepartitionSinkTopics(Collections.singletonList(SINK_TOPIC_NAME)) + .setSourceTopics(List.of(SOURCE_TOPIC_NAME)) + .setRepartitionSinkTopics(List.of(SINK_TOPIC_NAME)) .setRepartitionSourceTopics(List.of(REPARTITION_TOPIC_INFO)) .setStateChangelogTopics(List.of(SOURCE_CHANGELOG_TOPIC_CONFIG)); private static final TopicInfo CHANGELOG_TOPIC_CONFIG = new TopicInfo() @@ -70,14 +69,14 @@ public class ChangelogTopicsTest { .setTopicConfigs(List.of(TOPIC_CONFIG)); private static final Subtopology SUBTOPOLOGY_STATEFUL = new Subtopology() .setSubtopologyId("SUBTOPOLOGY_STATEFUL") - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_NAME)) - .setRepartitionSinkTopics(Collections.singletonList(SINK_TOPIC_NAME)) + .setSourceTopics(List.of(SOURCE_TOPIC_NAME)) + .setRepartitionSinkTopics(List.of(SINK_TOPIC_NAME)) .setRepartitionSourceTopics(List.of(REPARTITION_TOPIC_INFO)) .setStateChangelogTopics(List.of(CHANGELOG_TOPIC_CONFIG)); private static final Subtopology SUBTOPOLOGY_BOTH = new Subtopology() .setSubtopologyId("SUBTOPOLOGY_BOTH") - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_NAME)) - .setRepartitionSinkTopics(Collections.singletonList(SINK_TOPIC_NAME)) + .setSourceTopics(List.of(SOURCE_TOPIC_NAME)) + .setRepartitionSinkTopics(List.of(SINK_TOPIC_NAME)) .setRepartitionSourceTopics(List.of(REPARTITION_TOPIC_INFO)) .setStateChangelogTopics(List.of(SOURCE_CHANGELOG_TOPIC_CONFIG, CHANGELOG_TOPIC_CONFIG)); @@ -104,7 +103,7 @@ public class ChangelogTopicsTest { new ChangelogTopics(LOG_CONTEXT, subtopologies, ChangelogTopicsTest::topicPartitionProvider); Map setup = changelogTopics.setup(); - assertEquals(Collections.emptyMap(), setup); + assertEquals(Map.of(), setup); } @Test @@ -126,7 +125,7 @@ public class ChangelogTopicsTest { new ChangelogTopics(LOG_CONTEXT, subtopologies, ChangelogTopicsTest::topicPartitionProvider); Map setup = changelogTopics.setup(); - assertEquals(Collections.emptyMap(), setup); + assertEquals(Map.of(), setup); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java index e1db0f048ac..68815f3b555 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredInternalTopicTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -34,13 +33,13 @@ public class ConfiguredInternalTopicTest { @Test public void testConstructorWithNullName() { assertThrows(NullPointerException.class, - () -> new ConfiguredInternalTopic(null, 1, Optional.empty(), Collections.emptyMap())); + () -> new ConfiguredInternalTopic(null, 1, Optional.empty(), Map.of())); } @Test public void testConstructorWithInvalidName() { assertThrows(InvalidTopicException.class, - () -> new ConfiguredInternalTopic("invalid topic name", 1, Optional.empty(), Collections.emptyMap())); + () -> new ConfiguredInternalTopic("invalid topic name", 1, Optional.empty(), Map.of())); } @Test @@ -52,7 +51,7 @@ public class ConfiguredInternalTopicTest { @Test public void testConstructorWithZeroPartitions() { assertThrows(IllegalArgumentException.class, - () -> new ConfiguredInternalTopic("test-topic", 0, Optional.empty(), Collections.emptyMap())); + () -> new ConfiguredInternalTopic("test-topic", 0, Optional.empty(), Map.of())); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java index d30716c25f7..a7082248857 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredSubtopologyTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -38,9 +37,9 @@ public class ConfiguredSubtopologyTest { assertThrows(NullPointerException.class, () -> new ConfiguredSubtopology( null, - Collections.emptyMap(), - Collections.emptySet(), - Collections.emptyMap() + Map.of(), + Set.of(), + Map.of() ) ); } @@ -49,10 +48,10 @@ public class ConfiguredSubtopologyTest { public void testConstructorWithNullRepartitionSourceTopics() { assertThrows(NullPointerException.class, () -> new ConfiguredSubtopology( - Collections.emptySet(), + Set.of(), null, - Collections.emptySet(), - Collections.emptyMap() + Set.of(), + Map.of() ) ); } @@ -61,10 +60,10 @@ public class ConfiguredSubtopologyTest { public void testConstructorWithNullRepartitionSinkTopics() { assertThrows(NullPointerException.class, () -> new ConfiguredSubtopology( - Collections.emptySet(), - Collections.emptyMap(), + Set.of(), + Map.of(), null, - Collections.emptyMap() + Map.of() ) ); } @@ -73,9 +72,9 @@ public class ConfiguredSubtopologyTest { public void testConstructorWithNullStateChangelogTopics() { assertThrows(NullPointerException.class, () -> new ConfiguredSubtopology( - Collections.emptySet(), - Collections.emptyMap(), - Collections.emptySet(), + Set.of(), + Map.of(), + Set.of(), null ) ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopologyTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopologyTest.java index a909629fa20..b298be7300a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopologyTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/ConfiguredTopologyTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.junit.jupiter.api.Test; import org.mockito.Mockito; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -44,7 +43,7 @@ public class ConfiguredTopologyTest { () -> new ConfiguredTopology( 0, null, - Collections.emptyMap(), + Map.of(), Optional.empty() ) ); @@ -68,7 +67,7 @@ public class ConfiguredTopologyTest { () -> new ConfiguredTopology( 0, Optional.empty(), - Collections.emptyMap(), + Map.of(), null ) ); @@ -80,7 +79,7 @@ public class ConfiguredTopologyTest { () -> new ConfiguredTopology( -1, Optional.of(new TreeMap<>()), - Collections.emptyMap(), + Map.of(), Optional.empty() ) ); @@ -92,7 +91,7 @@ public class ConfiguredTopologyTest { () -> new ConfiguredTopology( 1, Optional.empty(), - Collections.emptyMap(), + Map.of(), Optional.empty() ) ); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcerTest.java index 5529d335de5..1f294f60bb8 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/CopartitionedTopicsEnforcerTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.Map; import java.util.OptionalInt; import java.util.Set; @@ -49,7 +48,7 @@ public class CopartitionedTopicsEnforcerTest { @Test public void shouldThrowIllegalStateExceptionIfNoPartitionsFoundForCoPartitionedTopic() { - final Map topicPartitionCounts = Collections.emptyMap(); + final Map topicPartitionCounts = Map.of(); final CopartitionedTopicsEnforcer enforcer = new CopartitionedTopicsEnforcer(LOG_CONTEXT, topicPartitionProvider(topicPartitionCounts)); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java index 6ec600a0c38..bc013ed875d 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java @@ -29,8 +29,8 @@ import org.apache.kafka.coordinator.group.streams.TopicMetadata; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -94,7 +94,7 @@ class InternalTopicManagerTest { .setReplicationFactor((short) -1) .setConfigs( new CreatableTopicConfigCollection( - Collections.singletonList(new CreatableTopicConfig().setName(CONFIG_KEY).setValue(CONFIG_VALUE)).iterator()) + List.of(new CreatableTopicConfig().setName(CONFIG_KEY).setValue(CONFIG_VALUE)).iterator()) ), internalTopicsToBeCreated.get(STATE_CHANGELOG_TOPIC_1)); @@ -125,7 +125,7 @@ class InternalTopicManagerTest { new ConfiguredInternalTopic(REPARTITION_TOPIC, 2, Optional.of((short) 3), - Collections.emptyMap() + Map.of() ) ), Set.of(), @@ -133,7 +133,7 @@ class InternalTopicManagerTest { new ConfiguredInternalTopic(STATE_CHANGELOG_TOPIC_2, 2, Optional.empty(), - Collections.emptyMap() + Map.of() ))) ) ); @@ -143,12 +143,12 @@ class InternalTopicManagerTest { // Create a subtopology source -> repartition Subtopology subtopology1 = new Subtopology() .setSubtopologyId(SUBTOPOLOGY_1) - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_1)) - .setRepartitionSinkTopics(Collections.singletonList(REPARTITION_TOPIC)) - .setStateChangelogTopics(Collections.singletonList( + .setSourceTopics(List.of(SOURCE_TOPIC_1)) + .setRepartitionSinkTopics(List.of(REPARTITION_TOPIC)) + .setStateChangelogTopics(List.of( new StreamsGroupTopologyValue.TopicInfo() .setName(STATE_CHANGELOG_TOPIC_1) - .setTopicConfigs(Collections.singletonList( + .setTopicConfigs(List.of( new StreamsGroupTopologyValue.TopicConfig() .setKey(CONFIG_KEY) .setValue(CONFIG_VALUE) @@ -157,20 +157,20 @@ class InternalTopicManagerTest { // Create a subtopology repartition/source2 -> sink (copartitioned) Subtopology subtopology2 = new Subtopology() .setSubtopologyId(SUBTOPOLOGY_2) - .setSourceTopics(Collections.singletonList(SOURCE_TOPIC_2)) - .setRepartitionSourceTopics(Collections.singletonList( + .setSourceTopics(List.of(SOURCE_TOPIC_2)) + .setRepartitionSourceTopics(List.of( new StreamsGroupTopologyValue.TopicInfo() .setName(REPARTITION_TOPIC) .setReplicationFactor((short) 3) )) - .setStateChangelogTopics(Collections.singletonList( + .setStateChangelogTopics(List.of( new StreamsGroupTopologyValue.TopicInfo() .setName(STATE_CHANGELOG_TOPIC_2) )) - .setCopartitionGroups(Collections.singletonList( + .setCopartitionGroups(List.of( new StreamsGroupTopologyValue.CopartitionGroup() - .setSourceTopics(Collections.singletonList((short) 0)) - .setRepartitionSourceTopics(Collections.singletonList((short) 0)) + .setSourceTopics(List.of((short) 0)) + .setRepartitionSourceTopics(List.of((short) 0)) )); return new StreamsTopology(3, Map.of(SUBTOPOLOGY_1, subtopology1, SUBTOPOLOGY_2, subtopology2)); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/RepartitionTopicsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/RepartitionTopicsTest.java index decc4d0484f..8d1adb69e20 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/RepartitionTopicsTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/RepartitionTopicsTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.To import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -198,7 +197,7 @@ public class RepartitionTopicsTest { final Map setup = repartitionTopics.setup(); - assertEquals(Collections.emptyMap(), setup); + assertEquals(Map.of(), setup); } } \ No newline at end of file