diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index c644043168438..383fead81a999 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -114,10 +114,9 @@ import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.time.Duration import java.util -import java.util.Arrays.asList import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util.function.Consumer -import java.util.{Collections, Comparator, Optional, OptionalInt, OptionalLong, Properties} +import java.util.{Comparator, Optional, OptionalInt, OptionalLong, Properties} import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ @@ -182,7 +181,7 @@ class KafkaApisTest extends Logging { val apiVersionManager = new SimpleApiVersionManager( ListenerType.BROKER, true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), util.Map.of[String, java.lang.Short], 0)) setupFeatures(featureVersions) @@ -235,14 +234,14 @@ class KafkaApisTest extends Logging { val requestHeader = new RequestHeader(ApiKeys.DESCRIBE_CONFIGS, ApiKeys.DESCRIBE_CONFIGS.latestVersion, clientId, 0) - val expectedActions = Seq( + val expectedActions = util.List.of( new Action(operation, new ResourcePattern(resourceType, resourceName, PatternType.LITERAL), 1, true, true) ) // Verify that authorize is only called once - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions.asJava))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions))) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) val configRepository: ConfigRepository = mock(classOf[ConfigRepository]) val topicConfigs = new Properties() @@ -256,16 +255,16 @@ class KafkaApisTest extends Logging { val describeConfigsRequest = new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData() .setIncludeSynonyms(true) - .setResources(List(new DescribeConfigsRequestData.DescribeConfigsResource() + .setResources(util.List.of(new DescribeConfigsRequestData.DescribeConfigsResource() .setResourceName(resourceName) - .setResourceType(ConfigResource.Type.TOPIC.id)).asJava)) + .setResourceType(ConfigResource.Type.TOPIC.id)))) .build(requestHeader.apiVersion) val request = buildRequest(describeConfigsRequest, requestHeader = Option(requestHeader)) kafkaApis = createKafkaApis(authorizer = Some(authorizer), configRepository = configRepository) kafkaApis.handleDescribeConfigsRequest(request) - verify(authorizer).authorize(any(), ArgumentMatchers.eq(expectedActions.asJava)) + verify(authorizer).authorize(any(), ArgumentMatchers.eq(expectedActions)) val response = verifyNoThrottling[DescribeConfigsResponse](request) val results = response.data.results assertEquals(1, results.size) @@ -319,13 +318,13 @@ class KafkaApisTest extends Logging { val consumerGroupId = "consumer_group_1" val requestHeader = new RequestHeader(ApiKeys.DESCRIBE_CONFIGS, ApiKeys.DESCRIBE_CONFIGS.latestVersion, clientId, 0) - val expectedActions = Seq( + val expectedActions = util.List.of( new Action(operation, new ResourcePattern(resourceType, consumerGroupId, PatternType.LITERAL), 1, true, true) ) - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions.asJava))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions))) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) val configRepository: ConfigRepository = mock(classOf[ConfigRepository]) val cgConfigs = new Properties() @@ -344,9 +343,9 @@ class KafkaApisTest extends Logging { val describeConfigsRequest = new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData() .setIncludeSynonyms(true) - .setResources(List(new DescribeConfigsRequestData.DescribeConfigsResource() + .setResources(util.List.of(new DescribeConfigsRequestData.DescribeConfigsResource() .setResourceName(consumerGroupId) - .setResourceType(ConfigResource.Type.GROUP.id)).asJava)) + .setResourceType(ConfigResource.Type.GROUP.id)))) .build(requestHeader.apiVersion) val request = buildRequest(describeConfigsRequest, requestHeader = Option(requestHeader)) @@ -379,10 +378,10 @@ class KafkaApisTest extends Logging { props.forEach((x, y) => configEntries.add(new AlterConfigsRequest.ConfigEntry(x.asInstanceOf[String], y.asInstanceOf[String]))) - val configs = Map(authorizedResource -> new AlterConfigsRequest.Config(configEntries)) + val configs = util.Map.of(authorizedResource, new AlterConfigsRequest.Config(configEntries)) val requestHeader = new RequestHeader(ApiKeys.ALTER_CONFIGS, ApiKeys.ALTER_CONFIGS.latestVersion, clientId, 0) - val apiRequest = new AlterConfigsRequest.Builder(configs.asJava, false).build(requestHeader.apiVersion) + val apiRequest = new AlterConfigsRequest.Builder(configs, false).build(requestHeader.apiVersion) val request = buildRequest(apiRequest) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.LATEST_PRODUCTION) @@ -435,13 +434,13 @@ class KafkaApisTest extends Logging { val subscriptionName = "client_metric_subscription_1" val requestHeader = new RequestHeader(ApiKeys.DESCRIBE_CONFIGS, ApiKeys.DESCRIBE_CONFIGS.latestVersion, clientId, 0) - val expectedActions = Seq( + val expectedActions = util.List.of( new Action(operation, new ResourcePattern(resourceType, Resource.CLUSTER_NAME, PatternType.LITERAL), 1, true, true) ) - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions.asJava))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(expectedActions))) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) val resource = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, subscriptionName) val configRepository: ConfigRepository = mock(classOf[ConfigRepository]) @@ -453,9 +452,9 @@ class KafkaApisTest extends Logging { val describeConfigsRequest = new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData() .setIncludeSynonyms(true) - .setResources(List(new DescribeConfigsRequestData.DescribeConfigsResource() + .setResources(util.List.of(new DescribeConfigsRequestData.DescribeConfigsResource() .setResourceName(subscriptionName) - .setResourceType(ConfigResource.Type.CLIENT_METRICS.id)).asJava)) + .setResourceType(ConfigResource.Type.CLIENT_METRICS.id)))) .build(requestHeader.apiVersion) val request = buildRequest(describeConfigsRequest, requestHeader = Option(requestHeader)) @@ -546,8 +545,8 @@ class KafkaApisTest extends Logging { new ResourcePattern(resourceType, resourceName, PatternType.LITERAL), 1, logIfAllowed, logIfDenied) - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(Seq(expectedAuthorizedAction).asJava))) - .thenReturn(Seq(result).asJava) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(util.List.of(expectedAuthorizedAction)))) + .thenReturn(util.List.of(result)) } @Test @@ -736,7 +735,7 @@ class KafkaApisTest extends Logging { new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() .setKeyType(coordinatorType.id()) - .setCoordinatorKeys(asList(key))) + .setCoordinatorKeys(util.List.of(key))) } else { new FindCoordinatorRequest.Builder( new FindCoordinatorRequestData() @@ -747,7 +746,7 @@ class KafkaApisTest extends Logging { when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) - val capturedRequest = verifyTopicCreation(topicName, true, true, request) + val capturedRequest = verifyTopicCreation(topicName, enableAutoTopicCreation = true, isInternal = true, request) kafkaApis = createKafkaApis(authorizer = Some(authorizer), overrideProperties = topicConfigOverride) kafkaApis.handleFindCoordinatorRequest(request) @@ -771,7 +770,7 @@ class KafkaApisTest extends Logging { def testFindCoordinatorWithInvalidSharePartitionKey(): Unit = { val request = new FindCoordinatorRequestData() .setKeyType(CoordinatorType.SHARE.id) - .setCoordinatorKeys(asList("")) + .setCoordinatorKeys(util.List.of("")) val requestChannelRequest = buildRequest(new FindCoordinatorRequest.Builder(request).build()) @@ -779,7 +778,7 @@ class KafkaApisTest extends Logging { kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val expectedResponse = new FindCoordinatorResponseData() - .setCoordinators(asList( + .setCoordinators(util.List.of( new FindCoordinatorResponseData.Coordinator() .setKey("") .setErrorCode(Errors.INVALID_REQUEST.code) @@ -798,7 +797,7 @@ class KafkaApisTest extends Logging { val request = new FindCoordinatorRequestData() .setKeyType(CoordinatorType.SHARE.id) - .setCoordinatorKeys(asList(key.asCoordinatorKey)) + .setCoordinatorKeys(util.List.of(key.asCoordinatorKey)) val requestChannelRequest = buildRequest(new FindCoordinatorRequest.Builder(request).build()) @@ -808,7 +807,7 @@ class KafkaApisTest extends Logging { when(shareCoordinator.partitionFor(ArgumentMatchers.eq(key))).thenReturn(10) val expectedResponse = new FindCoordinatorResponseData() - .setCoordinators(asList( + .setCoordinators(util.List.of( new FindCoordinatorResponseData.Coordinator() .setKey(key.asCoordinatorKey) .setNodeId(0) @@ -900,7 +899,7 @@ class KafkaApisTest extends Logging { } val metadataRequest = new MetadataRequest.Builder( - List(topicName).asJava, enableAutoTopicCreation + util.List.of(topicName), enableAutoTopicCreation ).build(requestHeader.apiVersion) val request = buildRequest(metadataRequest) @@ -912,11 +911,11 @@ class KafkaApisTest extends Logging { kafkaApis.handleTopicMetadataRequest(request) val response = verifyNoThrottling[MetadataResponse](request) - val expectedMetadataResponse = util.Collections.singletonList(new TopicMetadata( + val expectedMetadataResponse = util.List.of(new TopicMetadata( expectedError, topicName, isInternal, - util.Collections.emptyList() + util.List.of() )) assertEquals(expectedMetadataResponse, response.topicMetadata()) @@ -969,7 +968,7 @@ class KafkaApisTest extends Logging { val invalidVersions = Set(10, 11) invalidVersions.foreach( version => topics.foreach(topic => { - val metadataRequestData = new MetadataRequestData().setTopics(Collections.singletonList(topic)) + val metadataRequestData = new MetadataRequestData().setTopics(util.List.of(topic)) val request = buildRequest(new MetadataRequest(metadataRequestData, version.toShort)) val kafkaApis = createKafkaApis() try { @@ -999,13 +998,13 @@ class KafkaApisTest extends Logging { val offsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) - .setCommittedOffset(10)).asJava)).asJava) + .setCommittedOffset(10))))) val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) @@ -1023,13 +1022,13 @@ class KafkaApisTest extends Logging { // This is the response returned by the group coordinator. val offsetCommitResponse = new OffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) future.complete(offsetCommitResponse) val response = verifyNoThrottling[OffsetCommitResponse](requestChannelRequest) @@ -1043,13 +1042,13 @@ class KafkaApisTest extends Logging { val offsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) - .setCommittedOffset(10)).asJava)).asJava) + .setCommittedOffset(10))))) val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) @@ -1067,13 +1066,13 @@ class KafkaApisTest extends Logging { ) val expectedOffsetCommitResponse = new OffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) - .setErrorCode(Errors.NOT_COORDINATOR.code)).asJava)).asJava) + .setErrorCode(Errors.NOT_COORDINATOR.code))))) future.completeExceptionally(Errors.NOT_COORDINATOR.exception) val response = verifyNoThrottling[OffsetCommitResponse](requestChannelRequest) @@ -1088,11 +1087,11 @@ class KafkaApisTest extends Logging { val offsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( // foo exists but only has 2 partitions. new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10), @@ -1101,27 +1100,27 @@ class KafkaApisTest extends Logging { .setCommittedOffset(20), new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(2) - .setCommittedOffset(30)).asJava), + .setCommittedOffset(30))), // bar exists. new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(40), new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(50)).asJava), + .setCommittedOffset(50))), // zar does not exist. new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("zar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(60), new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(70)).asJava)).asJava) + .setCommittedOffset(70))))) val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) @@ -1129,26 +1128,26 @@ class KafkaApisTest extends Logging { val expectedOffsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( // foo exists but only has 2 partitions. new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10), new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(20)).asJava), + .setCommittedOffset(20))), new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(40), new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(50)).asJava)).asJava) + .setCommittedOffset(50))))) val future = new CompletableFuture[OffsetCommitResponseData]() when(groupCoordinator.commitOffsets( @@ -1164,31 +1163,31 @@ class KafkaApisTest extends Logging { // This is the response returned by the group coordinator. val offsetCommitResponse = new OffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava), + .setErrorCode(Errors.NONE.code))), new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) val expectedOffsetCommitResponse = new OffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( // foo-2 is first because partitions failing the validation // are put in the response first. new OffsetCommitResponseData.OffsetCommitResponsePartition() @@ -1199,27 +1198,27 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.NONE.code), new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava), + .setErrorCode(Errors.NONE.code))), // zar is before bar because topics failing the validation are // put in the response first. new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("zar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code), new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)).asJava), + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code))), new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) future.complete(offsetCommitResponse) val response = verifyNoThrottling[OffsetCommitResponse](requestChannelRequest) @@ -1237,10 +1236,10 @@ class KafkaApisTest extends Logging { val offsetCommitRequest = new OffsetCommitRequest.Builder( new OffsetCommitRequestData() .setGroupId("groupId") - .setTopics(Collections.singletonList( + .setTopics(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName(topic) - .setPartitions(Collections.singletonList( + .setPartitions(util.List.of( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(invalidPartitionId) .setCommittedOffset(15) @@ -1283,7 +1282,7 @@ class KafkaApisTest extends Logging { "groupId", 15L, 0.toShort, - Map(invalidTopicPartition -> partitionOffsetCommitData).asJava, + util.Map.of(invalidTopicPartition, partitionOffsetCommitData), true ).build() val request = buildRequest(offsetCommitRequest) @@ -1316,13 +1315,13 @@ class KafkaApisTest extends Logging { .setProducerEpoch(30) .setGroupInstanceId("instance-id") .setTransactionalId("transactional-id") - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) - .setCommittedOffset(10)).asJava)).asJava) + .setCommittedOffset(10))))) val requestChannelRequest = buildRequest(new TxnOffsetCommitRequest.Builder(txnOffsetCommitRequest).build()) @@ -1341,13 +1340,13 @@ class KafkaApisTest extends Logging { // This is the response returned by the group coordinator. val txnOffsetCommitResponse = new TxnOffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) future.complete(txnOffsetCommitResponse) val response = verifyNoThrottling[TxnOffsetCommitResponse](requestChannelRequest) @@ -1361,13 +1360,13 @@ class KafkaApisTest extends Logging { val txnOffsetCommitRequest = new TxnOffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) - .setCommittedOffset(10)).asJava)).asJava) + .setCommittedOffset(10))))) val requestChannelRequest = buildRequest(new TxnOffsetCommitRequest.Builder(txnOffsetCommitRequest).build()) @@ -1385,13 +1384,13 @@ class KafkaApisTest extends Logging { ) val expectedTxnOffsetCommitResponse = new TxnOffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) - .setErrorCode(Errors.NOT_COORDINATOR.code)).asJava)).asJava) + .setErrorCode(Errors.NOT_COORDINATOR.code))))) future.completeExceptionally(Errors.NOT_COORDINATOR.exception) val response = verifyNoThrottling[TxnOffsetCommitResponse](requestChannelRequest) @@ -1406,11 +1405,11 @@ class KafkaApisTest extends Logging { val txnOffsetCommitRequest = new TxnOffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( // foo exists but only has 2 partitions. new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10), @@ -1419,27 +1418,27 @@ class KafkaApisTest extends Logging { .setCommittedOffset(20), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(2) - .setCommittedOffset(30)).asJava), + .setCommittedOffset(30))), // bar exists. new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(40), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(50)).asJava), + .setCommittedOffset(50))), // zar does not exist. new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("zar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(60), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(70)).asJava)).asJava) + .setCommittedOffset(70))))) val requestChannelRequest = buildRequest(new TxnOffsetCommitRequest.Builder(txnOffsetCommitRequest).build()) @@ -1447,26 +1446,26 @@ class KafkaApisTest extends Logging { val expectedTxnOffsetCommitRequest = new TxnOffsetCommitRequestData() .setGroupId("group") .setMemberId("member") - .setTopics(List( + .setTopics(util.List.of( // foo exists but only has 2 partitions. new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(20)).asJava), + .setCommittedOffset(20))), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(40), new TxnOffsetCommitRequestData.TxnOffsetCommitRequestPartition() .setPartitionIndex(1) - .setCommittedOffset(50)).asJava)).asJava) + .setCommittedOffset(50))))) val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(expectedTxnOffsetCommitRequest.transactionalId)).thenReturn(0) @@ -1483,31 +1482,31 @@ class KafkaApisTest extends Logging { // This is the response returned by the group coordinator. val txnOffsetCommitResponse = new TxnOffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava), + .setErrorCode(Errors.NONE.code))), new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) val expectedTxnOffsetCommitResponse = new TxnOffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( // foo-2 is first because partitions failing the validation // are put in the response first. new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() @@ -1518,27 +1517,27 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.NONE.code), new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava), + .setErrorCode(Errors.NONE.code))), // zar is before bar because topics failing the validation are // put in the response first. new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("zar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code), new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)).asJava), + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code))), new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(1) - .setErrorCode(Errors.NONE.code)).asJava)).asJava) + .setErrorCode(Errors.NONE.code))))) future.complete(txnOffsetCommitResponse) val response = verifyNoThrottling[TxnOffsetCommitResponse](requestChannelRequest) @@ -1565,7 +1564,7 @@ class KafkaApisTest extends Logging { groupId, producerId, epoch, - Map(topicPartition -> partitionOffsetCommitData).asJava, + util.Map.of(topicPartition, partitionOffsetCommitData), version >= TxnOffsetCommitRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 ).build(version) val request = buildRequest(offsetCommitRequest) @@ -1580,15 +1579,15 @@ class KafkaApisTest extends Logging { )).thenReturn(future) future.complete(new TxnOffsetCommitResponseData() - .setTopics(List( + .setTopics(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic() .setName(topicPartition.topic) - .setPartitions(List( + .setPartitions(util.List.of( new TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition() .setPartitionIndex(topicPartition.partition) .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code) - ).asJava) - ).asJava)) + )) + ))) kafkaApis = createKafkaApis() kafkaApis.handleTxnOffsetCommitRequest(request, requestLocal) @@ -1766,7 +1765,7 @@ class KafkaApisTest extends Logging { transactionalId, producerId, epoch, - Collections.singletonList(topicPartition) + util.List.of(topicPartition) ).build(version.toShort) val request = buildRequest(addPartitionsToTxnRequest) @@ -1792,9 +1791,9 @@ class KafkaApisTest extends Logging { val response = capturedResponse.getValue if (version < 2) { - assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID)) + assertEquals(util.Map.of(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID)) } else { - assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID)) + assertEquals(util.Map.of(topicPartition, Errors.PRODUCER_FENCED), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID)) } } finally { kafkaApis.close() @@ -1824,23 +1823,23 @@ class KafkaApisTest extends Logging { // Allow WRITE but deny TWO_PHASE_COMMIT when(authorizer.authorize( any(), - ArgumentMatchers.eq(Collections.singletonList(new Action( + ArgumentMatchers.eq(util.List.of(new Action( AclOperation.WRITE, new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL), 1, true, true))) - )).thenReturn(Collections.singletonList(AuthorizationResult.ALLOWED)) + )).thenReturn(util.List.of(AuthorizationResult.ALLOWED)) when(authorizer.authorize( any(), - ArgumentMatchers.eq(Collections.singletonList(new Action( + ArgumentMatchers.eq(util.List.of(new Action( AclOperation.TWO_PHASE_COMMIT, new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL), 1, true, true))) - )).thenReturn(Collections.singletonList(AuthorizationResult.DENIED)) + )).thenReturn(util.List.of(AuthorizationResult.DENIED)) val capturedResponse = ArgumentCaptor.forClass(classOf[InitProducerIdResponse]) @@ -1877,23 +1876,23 @@ class KafkaApisTest extends Logging { // Both permissions are allowed when(authorizer.authorize( any(), - ArgumentMatchers.eq(Collections.singletonList(new Action( + ArgumentMatchers.eq(util.List.of(new Action( AclOperation.WRITE, new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL), 1, true, true))) - )).thenReturn(Collections.singletonList(AuthorizationResult.ALLOWED)) + )).thenReturn(util.List.of(AuthorizationResult.ALLOWED)) when(authorizer.authorize( any(), - ArgumentMatchers.eq(Collections.singletonList(new Action( + ArgumentMatchers.eq(util.List.of(new Action( AclOperation.TWO_PHASE_COMMIT, new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL), 1, true, true))) - )).thenReturn(Collections.singletonList(AuthorizationResult.ALLOWED)) + )).thenReturn(util.List.of(AuthorizationResult.ALLOWED)) val responseCallback = ArgumentCaptor.forClass(classOf[InitProducerIdResult => Unit]) @@ -1950,15 +1949,15 @@ class KafkaApisTest extends Logging { val addPartitionsToTxnRequest = AddPartitionsToTxnRequest.Builder.forBroker( new AddPartitionsToTxnTransactionCollection( - List(new AddPartitionsToTxnTransaction() + util.List.of(new AddPartitionsToTxnTransaction() .setTransactionalId(transactionalId1) .setProducerId(producerId) .setProducerEpoch(epoch) .setVerifyOnly(false) .setTopics(new AddPartitionsToTxnTopicCollection( - Collections.singletonList(new AddPartitionsToTxnTopic() + util.List.of(new AddPartitionsToTxnTopic() .setName(tp0.topic) - .setPartitions(Collections.singletonList(tp0.partition)) + .setPartitions(util.List.of(tp0.partition)) ).iterator()) ), new AddPartitionsToTxnTransaction() .setTransactionalId(transactionalId2) @@ -1966,12 +1965,12 @@ class KafkaApisTest extends Logging { .setProducerEpoch(epoch) .setVerifyOnly(true) .setTopics(new AddPartitionsToTxnTopicCollection( - Collections.singletonList(new AddPartitionsToTxnTopic() + util.List.of(new AddPartitionsToTxnTopic() .setName(tp1.topic) - .setPartitions(Collections.singletonList(tp1.partition)) + .setPartitions(util.List.of(tp1.partition)) ).iterator()) ) - ).asJava.iterator() + ).iterator() ) ).build(4.toShort) val request = buildRequest(addPartitionsToTxnRequest) @@ -1993,16 +1992,16 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(epoch), ArgumentMatchers.eq(Set(tp1)), verifyPartitionsCallback.capture(), - )).thenAnswer(_ => verifyPartitionsCallback.getValue.apply(AddPartitionsToTxnResponse.resultForTransaction(transactionalId2, Map(tp1 -> Errors.PRODUCER_FENCED).asJava))) + )).thenAnswer(_ => verifyPartitionsCallback.getValue.apply(AddPartitionsToTxnResponse.resultForTransaction(transactionalId2, util.Map.of(tp1, Errors.PRODUCER_FENCED)))) kafkaApis = createKafkaApis() kafkaApis.handleAddPartitionsToTxnRequest(request, requestLocal) val response = verifyNoThrottling[AddPartitionsToTxnResponse](request) - val expectedErrors = Map( - transactionalId1 -> Collections.singletonMap(tp0, Errors.NONE), - transactionalId2 -> Collections.singletonMap(tp1, Errors.PRODUCER_FENCED) - ).asJava + val expectedErrors = util.Map.of( + transactionalId1, util.Map.of(tp0, Errors.NONE), + transactionalId2, util.Map.of(tp1, Errors.PRODUCER_FENCED) + ) assertEquals(expectedErrors, response.errors()) } @@ -2010,7 +2009,7 @@ class KafkaApisTest extends Logging { @ParameterizedTest @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN) def testHandleAddPartitionsToTxnAuthorizationFailedAndMetrics(version: Short): Unit = { - val requestMetrics = new RequestChannelMetrics(Collections.singleton(ApiKeys.ADD_PARTITIONS_TO_TXN)) + val requestMetrics = new RequestChannelMetrics(util.Set.of(ApiKeys.ADD_PARTITIONS_TO_TXN)) try { val topic = "topic" @@ -2026,27 +2025,27 @@ class KafkaApisTest extends Logging { transactionalId, producerId, epoch, - Collections.singletonList(tp)).build(version) + util.List.of(tp)).build(version) else AddPartitionsToTxnRequest.Builder.forBroker( new AddPartitionsToTxnTransactionCollection( - List(new AddPartitionsToTxnTransaction() + util.List.of(new AddPartitionsToTxnTransaction() .setTransactionalId(transactionalId) .setProducerId(producerId) .setProducerEpoch(epoch) .setVerifyOnly(true) .setTopics(new AddPartitionsToTxnTopicCollection( - Collections.singletonList(new AddPartitionsToTxnTopic() + util.List.of(new AddPartitionsToTxnTopic() .setName(tp.topic) - .setPartitions(Collections.singletonList(tp.partition)) + .setPartitions(util.List.of(tp.partition)) ).iterator())) - ).asJava.iterator())).build(version) + ).iterator())).build(version) val requestChannelRequest = buildRequest(addPartitionsToTxnRequest, requestMetrics = requestMetrics) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handle( requestChannelRequest, @@ -2087,21 +2086,21 @@ class KafkaApisTest extends Logging { transactionalId, producerId, epoch, - List(tp0, tp1).asJava).build(version) + util.List.of(tp0, tp1)).build(version) else AddPartitionsToTxnRequest.Builder.forBroker( new AddPartitionsToTxnTransactionCollection( - List(new AddPartitionsToTxnTransaction() + util.List.of(new AddPartitionsToTxnTransaction() .setTransactionalId(transactionalId) .setProducerId(producerId) .setProducerEpoch(epoch) .setVerifyOnly(true) .setTopics(new AddPartitionsToTxnTopicCollection( - Collections.singletonList(new AddPartitionsToTxnTopic() + util.List.of(new AddPartitionsToTxnTopic() .setName(tp0.topic) - .setPartitions(List[Integer](tp0.partition, tp1.partition()).asJava) + .setPartitions(util.List.of[Integer](tp0.partition, tp1.partition())) ).iterator())) - ).asJava.iterator())).build(version) + ).iterator())).build(version) val requestChannelRequest = buildRequest(addPartitionsToTxnRequest) kafkaApis = createKafkaApis() @@ -2200,8 +2199,8 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + util.List.of(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setPartitionData(util.List.of( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) @@ -2261,8 +2260,8 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + util.List.of(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setPartitionData(util.List.of( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) @@ -2325,8 +2324,8 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + util.List.of(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setPartitionData(util.List.of( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) @@ -2388,8 +2387,8 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + util.List.of(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setPartitionData(util.List.of( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes)))))) @@ -2450,8 +2449,8 @@ class KafkaApisTest extends Logging { val produceRequest = ProduceRequest.builder(new ProduceRequestData() .setTopicData(new ProduceRequestData.TopicProduceDataCollection( - Collections.singletonList(new ProduceRequestData.TopicProduceData() - .setName(tp.topic).setPartitionData(Collections.singletonList( + util.List.of(new ProduceRequestData.TopicProduceData() + .setName(tp.topic).setPartitionData(util.List.of( new ProduceRequestData.PartitionProduceData() .setIndex(tp.partition) .setRecords(MemoryRecords.withTransactionalRecords(Compression.NONE, 0, 0, 0, new SimpleRecord("test".getBytes)))))) @@ -2491,7 +2490,7 @@ class KafkaApisTest extends Logging { val invalidTopicPartition = new TopicPartition(topic, invalidPartitionId) val addPartitionsToTxnRequest = AddPartitionsToTxnRequest.Builder.forClient( - "txnlId", 15L, 0.toShort, List(invalidTopicPartition).asJava + "txnlId", 15L, 0.toShort, util.List.of(invalidTopicPartition) ).build() val request = buildRequest(addPartitionsToTxnRequest) @@ -2515,13 +2514,13 @@ class KafkaApisTest extends Logging { @Test def requiredAclsNotPresentWriteTxnMarkersThrowsAuthorizationException(): Unit = { val topicPartition = new TopicPartition("t", 0) - val (_, request) = createWriteTxnMarkersRequest(asList(topicPartition)) + val (_, request) = createWriteTxnMarkersRequest(util.List.of(topicPartition)) val authorizer: Authorizer = mock(classOf[Authorizer]) val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) - val alterActions = Collections.singletonList(new Action(AclOperation.ALTER, clusterResource, 1, true, false)) - val clusterActions = Collections.singletonList(new Action(AclOperation.CLUSTER_ACTION, clusterResource, 1, true, true)) - val deniedList = Collections.singletonList(AuthorizationResult.DENIED) + val alterActions = util.List.of(new Action(AclOperation.ALTER, clusterResource, 1, true, false)) + val clusterActions = util.List.of(new Action(AclOperation.CLUSTER_ACTION, clusterResource, 1, true, true)) + val deniedList = util.List.of(AuthorizationResult.DENIED) when(authorizer.authorize( request.context, alterActions @@ -2539,8 +2538,8 @@ class KafkaApisTest extends Logging { @Test def shouldRespondWithUnknownTopicWhenPartitionIsNotHosted(): Unit = { val topicPartition = new TopicPartition("t", 0) - val (_, request) = createWriteTxnMarkersRequest(asList(topicPartition)) - val expectedErrors = Map(topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION).asJava + val (_, request) = createWriteTxnMarkersRequest(util.List.of(topicPartition)) + val expectedErrors = util.Map.of(topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION) val capturedResponse: ArgumentCaptor[WriteTxnMarkersResponse] = ArgumentCaptor.forClass(classOf[WriteTxnMarkersResponse]) when(replicaManager.onlinePartition(topicPartition)) @@ -2563,9 +2562,9 @@ class KafkaApisTest extends Logging { // with no records. val topicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder( - asList( - new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, asList(topicPartition)), - new TxnMarkerEntry(2, 1.toShort, 0, TransactionResult.COMMIT, asList(topicPartition)), + util.List.of( + new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, util.List.of(topicPartition)), + new TxnMarkerEntry(2, 1.toShort, 0, TransactionResult.COMMIT, util.List.of(topicPartition)), )).build() val request = buildRequest(writeTxnMarkersRequest) val capturedResponse: ArgumentCaptor[WriteTxnMarkersResponse] = ArgumentCaptor.forClass(classOf[WriteTxnMarkersResponse]) @@ -2597,8 +2596,8 @@ class KafkaApisTest extends Logging { def shouldRespondWithUnknownTopicOrPartitionForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) - val (_, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) - val expectedErrors = Map(tp1 -> Errors.UNKNOWN_TOPIC_OR_PARTITION, tp2 -> Errors.NONE).asJava + val (_, request) = createWriteTxnMarkersRequest(util.List.of(tp1, tp2)) + val expectedErrors = util.Map.of(tp1, Errors.UNKNOWN_TOPIC_OR_PARTITION, tp2, Errors.NONE) val capturedResponse: ArgumentCaptor[WriteTxnMarkersResponse] = ArgumentCaptor.forClass(classOf[WriteTxnMarkersResponse]) val responseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) @@ -2636,7 +2635,7 @@ class KafkaApisTest extends Logging { @ValueSource(strings = Array("ALTER", "CLUSTER_ACTION")) def shouldAppendToLogOnWriteTxnMarkersWhenCorrectMagicVersion(allowedAclOperation: String): Unit = { val topicPartition = new TopicPartition("t", 0) - val request = createWriteTxnMarkersRequest(asList(topicPartition))._2 + val request = createWriteTxnMarkersRequest(util.List.of(topicPartition))._2 when(replicaManager.onlinePartition(topicPartition)) .thenReturn(Some(mock(classOf[Partition]))) @@ -2645,15 +2644,15 @@ class KafkaApisTest extends Logging { // Allowing WriteTxnMarkers API with the help of allowedAclOperation parameter. val authorizer: Authorizer = mock(classOf[Authorizer]) val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) - val allowedAction = Collections.singletonList(new Action( + val allowedAction = util.List.of(new Action( AclOperation.fromString(allowedAclOperation), clusterResource, 1, true, allowedAclOperation.equals("CLUSTER_ACTION") )) - val deniedList = Collections.singletonList(AuthorizationResult.DENIED) - val allowedList = Collections.singletonList(AuthorizationResult.ALLOWED) + val deniedList = util.List.of(AuthorizationResult.DENIED) + val allowedList = util.List.of(AuthorizationResult.ALLOWED) when(authorizer.authorize( ArgumentMatchers.eq(request.context), any() @@ -2692,22 +2691,22 @@ class KafkaApisTest extends Logging { ) val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder( - List( + util.List.of( new TxnMarkerEntry( 1L, 1.toShort, 0, TransactionResult.COMMIT, - List(offset0, foo0).asJava + util.List.of(offset0, foo0) ), new TxnMarkerEntry( 2L, 1.toShort, 0, TransactionResult.ABORT, - List(offset1, foo1).asJava + util.List.of(offset1, foo1) ) - ).asJava + ) ).build() val requestChannelRequest = buildRequest(writeTxnMarkersRequest) @@ -2762,44 +2761,44 @@ class KafkaApisTest extends Logging { kafkaApis.handleWriteTxnMarkersRequest(requestChannelRequest, RequestLocal.noCaching) val expectedResponse = new WriteTxnMarkersResponseData() - .setMarkers(List( + .setMarkers(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerResult() .setProducerId(1L) - .setTopics(List( + .setTopics(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() .setName(Topic.GROUP_METADATA_TOPIC_NAME) - .setPartitions(List( + .setPartitions(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava), + )), new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) - ).asJava), + )) + )), new WriteTxnMarkersResponseData.WritableTxnMarkerResult() .setProducerId(2L) - .setTopics(List( + .setTopics(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() .setName(Topic.GROUP_METADATA_TOPIC_NAME) - .setPartitions(List( + .setPartitions(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava), + )), new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava) - ).asJava) - ).asJava) + )) + )) + )) val response = verifyNoThrottling[WriteTxnMarkersResponse](requestChannelRequest) assertEquals(normalize(expectedResponse), normalize(response.data)) @@ -2816,15 +2815,15 @@ class KafkaApisTest extends Logging { val offset0 = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0) val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder( - List( + util.List.of( new TxnMarkerEntry( 1L, 1.toShort, 0, TransactionResult.COMMIT, - List(offset0).asJava + util.List.of(offset0) ) - ).asJava + ) ).build() val requestChannelRequest = buildRequest(writeTxnMarkersRequest) @@ -2851,19 +2850,19 @@ class KafkaApisTest extends Logging { } val expectedResponse = new WriteTxnMarkersResponseData() - .setMarkers(List( + .setMarkers(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerResult() .setProducerId(1L) - .setTopics(List( + .setTopics(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() .setName(Topic.GROUP_METADATA_TOPIC_NAME) - .setPartitions(List( + .setPartitions(util.List.of( new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() .setPartitionIndex(0) .setErrorCode(expectedError.code) - ).asJava) - ).asJava) - ).asJava) + )) + )) + )) val response = verifyNoThrottling[WriteTxnMarkersResponse](requestChannelRequest) assertEquals(normalize(expectedResponse), normalize(response.data)) @@ -2909,18 +2908,18 @@ class KafkaApisTest extends Logging { @Test def testHandleDeleteGroups(): Unit = { - val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(List( + val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(util.List.of( "group-1", "group-2", "group-3" - ).asJava) + )) val requestChannelRequest = buildRequest(new DeleteGroupsRequest.Builder(deleteGroupsRequest).build()) val future = new CompletableFuture[DeleteGroupsResponseData.DeletableGroupResultCollection]() when(groupCoordinator.deleteGroups( requestChannelRequest.context, - List("group-1", "group-2", "group-3").asJava, + util.List.of("group-1", "group-2", "group-3"), RequestLocal.noCaching.bufferSupplier )).thenReturn(future) kafkaApis = createKafkaApis() @@ -2929,7 +2928,7 @@ class KafkaApisTest extends Logging { RequestLocal.noCaching ) - val results = new DeleteGroupsResponseData.DeletableGroupResultCollection(List( + val results = new DeleteGroupsResponseData.DeletableGroupResultCollection(util.List.of( new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-1") .setErrorCode(Errors.NONE.code), @@ -2939,7 +2938,7 @@ class KafkaApisTest extends Logging { new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-3") .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code), - ).iterator.asJava) + ).iterator) future.complete(results) @@ -2952,18 +2951,18 @@ class KafkaApisTest extends Logging { @Test def testHandleDeleteGroupsFutureFailed(): Unit = { - val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(List( + val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(util.List.of( "group-1", "group-2", "group-3" - ).asJava) + )) val requestChannelRequest = buildRequest(new DeleteGroupsRequest.Builder(deleteGroupsRequest).build()) val future = new CompletableFuture[DeleteGroupsResponseData.DeletableGroupResultCollection]() when(groupCoordinator.deleteGroups( requestChannelRequest.context, - List("group-1", "group-2", "group-3").asJava, + util.List.of("group-1", "group-2", "group-3"), RequestLocal.noCaching.bufferSupplier )).thenReturn(future) kafkaApis = createKafkaApis() @@ -2975,7 +2974,7 @@ class KafkaApisTest extends Logging { future.completeExceptionally(Errors.NOT_CONTROLLER.exception) val expectedDeleteGroupsResponse = new DeleteGroupsResponseData() - .setResults(new DeleteGroupsResponseData.DeletableGroupResultCollection(List( + .setResults(new DeleteGroupsResponseData.DeletableGroupResultCollection(util.List.of( new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-1") .setErrorCode(Errors.NOT_CONTROLLER.code), @@ -2985,7 +2984,7 @@ class KafkaApisTest extends Logging { new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-3") .setErrorCode(Errors.NOT_CONTROLLER.code), - ).iterator.asJava)) + ).iterator)) val response = verifyNoThrottling[DeleteGroupsResponse](requestChannelRequest) assertEquals(expectedDeleteGroupsResponse, response.data) @@ -2993,11 +2992,11 @@ class KafkaApisTest extends Logging { @Test def testHandleDeleteGroupsAuthenticationFailed(): Unit = { - val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(List( + val deleteGroupsRequest = new DeleteGroupsRequestData().setGroupsNames(util.List.of( "group-1", "group-2", "group-3" - ).asJava) + )) val requestChannelRequest = buildRequest(new DeleteGroupsRequest.Builder(deleteGroupsRequest).build()) @@ -3014,15 +3013,15 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val future = new CompletableFuture[DeleteGroupsResponseData.DeletableGroupResultCollection]() when(groupCoordinator.deleteGroups( requestChannelRequest.context, - List("group-2", "group-3").asJava, + util.List.of("group-2", "group-3"), RequestLocal.noCaching.bufferSupplier )).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) @@ -3031,17 +3030,17 @@ class KafkaApisTest extends Logging { RequestLocal.noCaching ) - future.complete(new DeleteGroupsResponseData.DeletableGroupResultCollection(List( + future.complete(new DeleteGroupsResponseData.DeletableGroupResultCollection(util.List.of( new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-2") .setErrorCode(Errors.NONE.code), new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-3") .setErrorCode(Errors.NONE.code) - ).iterator.asJava)) + ).iterator)) val expectedDeleteGroupsResponse = new DeleteGroupsResponseData() - .setResults(new DeleteGroupsResponseData.DeletableGroupResultCollection(List( + .setResults(new DeleteGroupsResponseData.DeletableGroupResultCollection(util.List.of( new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-2") .setErrorCode(Errors.NONE.code), @@ -3050,7 +3049,7 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.NONE.code), new DeleteGroupsResponseData.DeletableGroupResult() .setGroupId("group-1") - .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code)).iterator.asJava)) + .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code)).iterator)) val response = verifyNoThrottling[DeleteGroupsResponse](requestChannelRequest) assertEquals(expectedDeleteGroupsResponse, response.data) @@ -3058,12 +3057,12 @@ class KafkaApisTest extends Logging { @Test def testHandleDescribeGroups(): Unit = { - val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(List( + val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(util.List.of( "group-1", "group-2", "group-3", "group-4" - ).asJava) + )) val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) @@ -3075,15 +3074,15 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) - val groupResults = List( + val groupResults = util.List.of( new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-1") .setProtocolType("consumer") .setProtocolData("range") .setGroupState("Stable") - .setMembers(List( + .setMembers(util.List.of( new DescribeGroupsResponseData.DescribedGroupMember() - .setMemberId("member-1")).asJava), + .setMemberId("member-1"))), new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-2") .setErrorCode(Errors.NOT_COORDINATOR.code), @@ -3095,7 +3094,7 @@ class KafkaApisTest extends Logging { .setGroupState("Dead") .setErrorCode(Errors.GROUP_ID_NOT_FOUND.code) .setErrorMessage("Group group-4 is not a classic group.") - ).asJava + ) future.complete(groupResults) @@ -3106,11 +3105,11 @@ class KafkaApisTest extends Logging { @Test def testHandleDescribeGroupsFutureFailed(): Unit = { - val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(List( + val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(util.List.of( "group-1", "group-2", "group-3" - ).asJava) + )) val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) @@ -3122,7 +3121,7 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) - val expectedDescribeGroupsResponse = new DescribeGroupsResponseData().setGroups(List( + val expectedDescribeGroupsResponse = new DescribeGroupsResponseData().setGroups(util.List.of( new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-1") .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code), @@ -3132,7 +3131,7 @@ class KafkaApisTest extends Logging { new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-3") .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code) - ).asJava) + )) future.completeExceptionally(Errors.UNKNOWN_SERVER_ERROR.exception) @@ -3142,11 +3141,11 @@ class KafkaApisTest extends Logging { @Test def testHandleDescribeGroupsAuthenticationFailed(): Unit = { - val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(List( + val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(util.List.of( "group-1", "group-2", "group-3" - ).asJava) + )) val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) @@ -3163,26 +3162,26 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream(). + map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() when(groupCoordinator.describeGroups( requestChannelRequest.context, - List("group-2").asJava + util.List.of("group-2") )).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) - future.complete(List( + future.complete(util.List.of( new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-2") .setErrorCode(Errors.NOT_COORDINATOR.code) - ).asJava) + )) - val expectedDescribeGroupsResponse = new DescribeGroupsResponseData().setGroups(List( + val expectedDescribeGroupsResponse = new DescribeGroupsResponseData().setGroups(util.List.of( // group-1 and group-3 are first because unauthorized are put first into the response. new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-1") @@ -3193,7 +3192,7 @@ class KafkaApisTest extends Logging { new DescribeGroupsResponseData.DescribedGroup() .setGroupId("group-2") .setErrorCode(Errors.NOT_COORDINATOR.code) - ).asJava) + )) val response = verifyNoThrottling[DescribeGroupsResponse](requestChannelRequest) assertEquals(expectedDescribeGroupsResponse, response.data) @@ -3208,14 +3207,14 @@ class KafkaApisTest extends Logging { val topics = new OffsetDeleteRequestTopicCollection() topics.add(new OffsetDeleteRequestTopic() .setName("topic-1") - .setPartitions(Seq( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), - new OffsetDeleteRequestPartition().setPartitionIndex(1)).asJava)) + new OffsetDeleteRequestPartition().setPartitionIndex(1)))) topics.add(new OffsetDeleteRequestTopic() .setName("topic-2") - .setPartitions(Seq( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), - new OffsetDeleteRequestPartition().setPartitionIndex(1)).asJava)) + new OffsetDeleteRequestPartition().setPartitionIndex(1)))) val offsetDeleteRequest = new OffsetDeleteRequest.Builder( new OffsetDeleteRequestData() @@ -3235,28 +3234,28 @@ class KafkaApisTest extends Logging { kafkaApis.handleOffsetDeleteRequest(request, requestLocal) val offsetDeleteResponseData = new OffsetDeleteResponseData() - .setTopics(new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(List( + .setTopics(new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection(util.List.of( new OffsetDeleteResponseData.OffsetDeleteResponseTopic() .setName("topic-1") - .setPartitions(new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponseData.OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetDeleteResponseData.OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)), + ).iterator)), new OffsetDeleteResponseData.OffsetDeleteResponseTopic() .setName("topic-2") - .setPartitions(new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponseData.OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetDeleteResponseData.OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)) - ).asJava.iterator())) + ).iterator)) + ).iterator())) future.complete(offsetDeleteResponseData) @@ -3272,30 +3271,30 @@ class KafkaApisTest extends Logging { val offsetDeleteRequest = new OffsetDeleteRequestData() .setGroupId(group) - .setTopics(new OffsetDeleteRequestTopicCollection(List( + .setTopics(new OffsetDeleteRequestTopicCollection(util.List.of( // foo exists but has only 2 partitions. new OffsetDeleteRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), new OffsetDeleteRequestPartition().setPartitionIndex(1), new OffsetDeleteRequestPartition().setPartitionIndex(2) - ).asJava), + )), // bar exists. new OffsetDeleteRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), new OffsetDeleteRequestPartition().setPartitionIndex(1) - ).asJava), + )), // zar does not exist. new OffsetDeleteRequestTopic() .setName("zar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), new OffsetDeleteRequestPartition().setPartitionIndex(1) - ).asJava), - ).asJava.iterator)) + )), + ).iterator)) val requestChannelRequest = buildRequest(new OffsetDeleteRequest.Builder(offsetDeleteRequest).build()) @@ -3303,20 +3302,20 @@ class KafkaApisTest extends Logging { // only existing topic-partitions. val expectedOffsetDeleteRequest = new OffsetDeleteRequestData() .setGroupId(group) - .setTopics(new OffsetDeleteRequestTopicCollection(List( + .setTopics(new OffsetDeleteRequestTopicCollection(util.List.of( new OffsetDeleteRequestTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), new OffsetDeleteRequestPartition().setPartitionIndex(1) - ).asJava), + )), new OffsetDeleteRequestTopic() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(0), new OffsetDeleteRequestPartition().setPartitionIndex(1) - ).asJava) - ).asJava.iterator)) + )) + ).iterator)) val future = new CompletableFuture[OffsetDeleteResponseData]() when(groupCoordinator.deleteOffsets( @@ -3332,34 +3331,34 @@ class KafkaApisTest extends Logging { // This is the response returned by the group coordinator. val offsetDeleteResponse = new OffsetDeleteResponseData() - .setTopics(new OffsetDeleteResponseTopicCollection(List( + .setTopics(new OffsetDeleteResponseTopicCollection(util.List.of( new OffsetDeleteResponseTopic() .setName("foo") - .setPartitions(new OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)), + ).iterator)), new OffsetDeleteResponseTopic() .setName("bar") - .setPartitions(new OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)), - ).asJava.iterator)) + ).iterator)), + ).iterator)) val expectedOffsetDeleteResponse = new OffsetDeleteResponseData() - .setTopics(new OffsetDeleteResponseTopicCollection(List( + .setTopics(new OffsetDeleteResponseTopicCollection(util.List.of( new OffsetDeleteResponseTopic() .setName("foo") - .setPartitions(new OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponsePartitionCollection(util.List.of( // foo-2 is first because partitions failing the validation // are put in the response first. new OffsetDeleteResponsePartition() @@ -3371,30 +3370,30 @@ class KafkaApisTest extends Logging { new OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)), + ).iterator)), // zar is before bar because topics failing the validation are // put in the response first. new OffsetDeleteResponseTopic() .setName("zar") - .setPartitions(new OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code), new OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) - ).asJava.iterator)), + ).iterator)), new OffsetDeleteResponseTopic() .setName("bar") - .setPartitions(new OffsetDeleteResponsePartitionCollection(List( + .setPartitions(new OffsetDeleteResponsePartitionCollection(util.List.of( new OffsetDeleteResponsePartition() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), new OffsetDeleteResponsePartition() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava.iterator)), - ).asJava.iterator)) + ).iterator)), + ).iterator)) future.complete(offsetDeleteResponse) val response = verifyNoThrottling[OffsetDeleteResponse](requestChannelRequest) @@ -3413,7 +3412,7 @@ class KafkaApisTest extends Logging { val topics = new OffsetDeleteRequestTopicCollection() topics.add(new OffsetDeleteRequestTopic() .setName(topic) - .setPartitions(Collections.singletonList( + .setPartitions(util.List.of( new OffsetDeleteRequestPartition().setPartitionIndex(invalidPartitionId)))) val offsetDeleteRequest = new OffsetDeleteRequest.Builder( new OffsetDeleteRequestData() @@ -3484,9 +3483,9 @@ class KafkaApisTest extends Logging { val offsetDeleteRequest = new OffsetDeleteRequest.Builder( new OffsetDeleteRequestData() .setGroupId(group) - .setTopics(new OffsetDeleteRequestTopicCollection(Collections.singletonList(new OffsetDeleteRequestTopic() + .setTopics(new OffsetDeleteRequestTopicCollection(util.List.of(new OffsetDeleteRequestTopic() .setName("topic-unknown") - .setPartitions(Collections.singletonList(new OffsetDeleteRequestPartition() + .setPartitions(util.List.of(new OffsetDeleteRequestPartition() .setPartitionIndex(0) )) ).iterator())) @@ -3534,15 +3533,15 @@ class KafkaApisTest extends Logging { .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET) .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) .setPartitionIndex(tp.partition()) - callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(List(partitionResponse).asJava))) + callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(util.List.of(partitionResponse)))) }) - val targetTimes = List(new ListOffsetsTopic() + val targetTimes = util.List.of(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetsPartition() + .setPartitions(util.List.of(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) - .setCurrentLeaderEpoch(currentLeaderEpoch.get)).asJava)).asJava + .setCurrentLeaderEpoch(currentLeaderEpoch.get)))) val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) @@ -3676,20 +3675,19 @@ class KafkaApisTest extends Logging { val unauthorizedTopic = "unauthorized-topic" val authorizedTopic = "authorized-topic" - val expectedActions = Seq( + val expectedActions = util.List.of( new Action(AclOperation.DESCRIBE, new ResourcePattern(ResourceType.TOPIC, unauthorizedTopic, PatternType.LITERAL), 1, true, true), new Action(AclOperation.DESCRIBE, new ResourcePattern(ResourceType.TOPIC, authorizedTopic, PatternType.LITERAL), 1, true, true) ) - when(authorizer.authorize(any[RequestContext], argThat((t: java.util.List[Action]) => t.containsAll(expectedActions.asJava)))) + when(authorizer.authorize(any[RequestContext], argThat((t: java.util.List[Action]) => t.containsAll(expectedActions)))) .thenAnswer { invocation => - val actions = invocation.getArgument(1).asInstanceOf[util.List[Action]].asScala - actions.map { action => - if (action.resourcePattern().name().equals(authorizedTopic)) - AuthorizationResult.ALLOWED - else - AuthorizationResult.DENIED - }.asJava + val actions = invocation.getArgument(1, classOf[util.List[Action]]) + val results = new util.ArrayList[AuthorizationResult]() + actions.forEach { a => + results.add(if (a.resourcePattern.name == authorizedTopic) AuthorizationResult.ALLOWED else AuthorizationResult.DENIED) + } + results } // 3. Set up MetadataCache @@ -3704,15 +3702,15 @@ class KafkaApisTest extends Logging { .setPartitionId(0) .setLeader(0) .setLeaderEpoch(0) - .setReplicas(Collections.singletonList(0)) - .setIsr(Collections.singletonList(0)) + .setReplicas(util.List.of(0)) + .setIsr(util.List.of(0)) } val partitionRecords = Seq(authorizedTopicId, unauthorizedTopicId).map(createDummyPartitionRecord) MetadataCacheTest.updateCache(metadataCache, partitionRecords) // 4. Send TopicMetadataReq using topicId - val metadataReqByTopicId = new MetadataRequest.Builder(util.Arrays.asList(authorizedTopicId, unauthorizedTopicId)).build() + val metadataReqByTopicId = new MetadataRequest.Builder(util.List.of(authorizedTopicId, unauthorizedTopicId)).build() val repByTopicId = buildRequest(metadataReqByTopicId, plaintextListener) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) @@ -3737,7 +3735,7 @@ class KafkaApisTest extends Logging { // 4. Send TopicMetadataReq using topic name reset(clientRequestQuotaManager, requestChannel) - val metadataReqByTopicName = new MetadataRequest.Builder(util.Arrays.asList(authorizedTopic, unauthorizedTopic), false).build() + val metadataReqByTopicName = new MetadataRequest.Builder(util.List.of(authorizedTopic, unauthorizedTopic), false).build() val repByTopicName = buildRequest(metadataReqByTopicName, plaintextListener) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleTopicMetadataRequest(repByTopicName) @@ -3748,7 +3746,7 @@ class KafkaApisTest extends Logging { metadataByTopicName.foreach { case (topicName, metadataResponseTopic) => if (topicName == unauthorizedTopic) { assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code(), metadataResponseTopic.errorCode()) - // Do not return topic Id on unauthorized error + // Do not return topicId on unauthorized error assertEquals(Uuid.ZERO_UUID, metadataResponseTopic.topicId()) } else { assertEquals(Errors.NONE.code(), metadataResponseTopic.errorCode()) @@ -3784,10 +3782,10 @@ class KafkaApisTest extends Logging { Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false))) }) - val fetchData = Map(tidp -> new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, 1000, - Optional.empty())).asJava - val fetchDataBuilder = Map(tp -> new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, 1000, - Optional.empty())).asJava + val fetchData = util.Map.of(tidp, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, 1000, + Optional.empty())) + val fetchDataBuilder = util.Map.of(tp, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, 1000, + Optional.empty())) val fetchMetadata = new JFetchMetadata(0, 0) val fetchContext = new FullFetchContext(time, new FetchSessionCacheShard(1000, 100), fetchMetadata, fetchData, false, false) @@ -3836,10 +3834,10 @@ class KafkaApisTest extends Logging { when(replicaManager.getLogConfig(ArgumentMatchers.eq(unresolvedFoo.topicPartition))).thenReturn(None) // Simulate unknown topic ID in the context - val fetchData = Map(new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition)) -> - new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, Optional.empty())).asJava - val fetchDataBuilder = Map(foo.topicPartition -> new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, - Optional.empty())).asJava + val fetchData = util.Map.of(new TopicIdPartition(foo.topicId, new TopicPartition(null, foo.partition)), + new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, Optional.empty())) + val fetchDataBuilder = util.Map.of(foo.topicPartition, new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, + Optional.empty())) val fetchMetadata = new JFetchMetadata(0, 0) val fetchContext = new FullFetchContext(time, new FetchSessionCacheShard(1000, 100), fetchMetadata, fetchData, true, replicaId >= 0) @@ -3848,8 +3846,8 @@ class KafkaApisTest extends Logging { ApiKeys.FETCH.latestVersion, fetchMetadata, replicaId >= 0, - Collections.singletonMap(foo, new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, Optional.empty())), - Collections.emptyList[TopicIdPartition], + util.Map.of(foo, new FetchRequest.PartitionData(foo.topicId, 0, 0, 1000, Optional.empty())), + util.List.of[TopicIdPartition], metadataCache.topicIdsToNames()) ).thenReturn(fetchContext) @@ -3884,7 +3882,7 @@ class KafkaApisTest extends Logging { addTopicToMetadataCache(tp.topic, numPartitions = 1, numBrokers = 3, topicId) when(replicaManager.getLogConfig(ArgumentMatchers.eq(tp))).thenReturn(Some(LogConfig.fromProps( - Collections.emptyMap(), + util.Map.of(), new Properties() ))) @@ -3907,10 +3905,10 @@ class KafkaApisTest extends Logging { Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false))) }) - val fetchData = Map(tidp -> new FetchRequest.PartitionData(topicId, 0, 0, 1000, - Optional.empty())).asJava - val fetchDataBuilder = Map(tp -> new FetchRequest.PartitionData(topicId, 0, 0, 1000, - Optional.empty())).asJava + val fetchData = util.Map.of(tidp, new FetchRequest.PartitionData(topicId, 0, 0, 1000, + Optional.empty())) + val fetchDataBuilder = util.Map.of(tp, new FetchRequest.PartitionData(topicId, 0, 0, 1000, + Optional.empty())) val fetchMetadata = new JFetchMetadata(0, 0) val fetchContext = new FullFetchContext(time, new FetchSessionCacheShard(1000, 100), fetchMetadata, fetchData, true, false) @@ -3957,19 +3955,19 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -4026,19 +4024,19 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] @@ -4129,19 +4127,19 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -4151,12 +4149,12 @@ class KafkaApisTest extends Logging { ).thenThrow(Errors.INVALID_REQUEST.exception) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4197,19 +4195,19 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(-1). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4280,18 +4278,18 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( @@ -4417,13 +4415,13 @@ class KafkaApisTest extends Logging { val records = MemoryRecords.EMPTY when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.REPLICA_NOT_AVAILABLE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List().asJava)) - ).asJava) + .setAcquiredRecords(new util.ArrayList(util.List.of)) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -4480,18 +4478,18 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, partitionIndex, topicName) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, partitionIndex, topicName), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -4570,18 +4568,18 @@ class KafkaApisTest extends Logging { val records = memoryRecords(10, 0) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -4660,62 +4658,62 @@ class KafkaApisTest extends Logging { val records3 = memoryRecords(10, 20) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records2) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(10) .setLastOffset(19) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records3) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(20) .setLastOffset(29) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] @@ -4787,7 +4785,7 @@ class KafkaApisTest extends Logging { new ShareFetchRequestData.AcknowledgementBatch(). setFirstOffset(0). setLastOffset(9). - setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava))))))) + setAcknowledgeTypes(util.List.of[java.lang.Byte](1.toByte)))))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4824,7 +4822,7 @@ class KafkaApisTest extends Logging { new ShareFetchRequestData.AcknowledgementBatch(). setFirstOffset(10). setLastOffset(19). - setAcknowledgeTypes(List[java.lang.Byte](1.toByte).asJava))))))) + setAcknowledgeTypes(util.List.of[java.lang.Byte](1.toByte)))))))) shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) request = buildRequest(shareFetchRequest) @@ -4884,84 +4882,84 @@ class KafkaApisTest extends Logging { val groupId = "group" when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t1_p1_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)), - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)) -> + ))), + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t1_p2_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(10) .setLastOffset(19) .setDeliveryCount(1) - ).asJava)), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)) -> + ))), + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t2_p1_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(43) .setLastOffset(52) .setDeliveryCount(1) - ).asJava)), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) -> + ))), + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t2_p2_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(17) .setLastOffset(26) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t3_p1_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(54) .setLastOffset(73) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t3_p1_2) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(74) .setLastOffset(93) .setDeliveryCount(1) - ).asJava)), - new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)) -> + ))), + new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setRecords(records_t4_p1_1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(10) .setLastOffset(24) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) val cachedSharePartitions1 = new ImplicitLinkedHashCollection[CachedSharePartition] @@ -5003,45 +5001,45 @@ class KafkaApisTest extends Logging { ).thenReturn(new FinalContext()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)) -> + new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)) -> + new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)) -> + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) -> + new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)) -> + new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)) -> + new TopicIdPartition(topicId4, new TopicPartition(topicName4, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - ).asJava) + )) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -5343,41 +5341,41 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - tp1 -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + tp1, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) .setRecords(records_t1_p1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)), - tp2 -> + ))), + tp2, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) .setRecords(records_t2_p1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(14) .setDeliveryCount(1) - ).asJava)), - tp3 -> + ))), + tp3, new ShareFetchResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) .setRecords(records_t2_p2) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(19) .setDeliveryCount(1) - ).asJava)), - ).asJava) + ))), + )) ) val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() @@ -5493,29 +5491,26 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - tp1 -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + tp1, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) .setRecords(records_t1_p1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) - val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() - erroneousPartitions.put( + val erroneousPartitions = util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( tp2, new ShareFetchResponseData.PartitionData() .setPartitionIndex(1) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) - ) - erroneousPartitions.put( + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code), tp3, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) @@ -5627,26 +5622,26 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - tp1 -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + tp1, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code) .setRecords(emptyRecords) - .setAcquiredRecords(new util.ArrayList(List().asJava)), - tp2 -> + .setAcquiredRecords(new util.ArrayList(util.List.of)), + tp2, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code) .setRecords(emptyRecords) - .setAcquiredRecords(new util.ArrayList(List().asJava)), - tp3 -> + .setAcquiredRecords(new util.ArrayList(util.List.of)), + tp3, new ShareFetchResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code) .setRecords(emptyRecords) - .setAcquiredRecords(new util.ArrayList(List().asJava)) - ).asJava) + .setAcquiredRecords(new util.ArrayList(util.List.of)) + )) ) val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() @@ -5709,7 +5704,7 @@ class KafkaApisTest extends Logging { Errors.UNKNOWN_SERVER_ERROR.code, Errors.NONE.code, emptyRecords, - Collections.emptyList[AcquiredRecords](), + util.List.of[AcquiredRecords](), partitionData1 ) @@ -5722,7 +5717,7 @@ class KafkaApisTest extends Logging { Errors.UNKNOWN_SERVER_ERROR.code, Errors.NONE.code, emptyRecords, - Collections.emptyList[AcquiredRecords](), + util.List.of[AcquiredRecords](), partitionData2 ) @@ -5735,7 +5730,7 @@ class KafkaApisTest extends Logging { Errors.UNKNOWN_SERVER_ERROR.code, Errors.NONE.code, emptyRecords, - Collections.emptyList[AcquiredRecords](), + util.List.of[AcquiredRecords](), partitionData3 ) } @@ -5767,30 +5762,30 @@ class KafkaApisTest extends Logging { val tp4 = new TopicIdPartition(topicId3, new TopicPartition(topicName3, 0)) when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - tp2 -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + tp2, new ShareFetchResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) .setRecords(records1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)), - tp3 -> + ))), + tp3, new ShareFetchResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) .setRecords(records2) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(19) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) val erroneousPartitions: util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = new util.HashMap() @@ -5859,7 +5854,7 @@ class KafkaApisTest extends Logging { Errors.TOPIC_AUTHORIZATION_FAILED.code, Errors.NONE.code, MemoryRecords.EMPTY, - Collections.emptyList[AcquiredRecords](), + util.List.of[AcquiredRecords](), partitionData1 ) @@ -5898,7 +5893,7 @@ class KafkaApisTest extends Logging { Errors.UNKNOWN_TOPIC_OR_PARTITION.code, Errors.NONE.code, MemoryRecords.EMPTY, - Collections.emptyList[AcquiredRecords](), + util.List.of[AcquiredRecords](), partitionData4 ) } @@ -5940,33 +5935,33 @@ class KafkaApisTest extends Logging { val groupId = "group" when(sharePartitionManager.fetchMessages(any(), any(), any(), anyInt(), anyInt(), anyInt(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records1) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(0) .setLastOffset(9) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareFetchResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareFetchResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareFetchResponseData.PartitionData() .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(Errors.NONE.code) .setRecords(records2) - .setAcquiredRecords(new util.ArrayList(List( + .setAcquiredRecords(new util.ArrayList(util.List.of( new ShareFetchResponseData.AcquiredRecords() .setFirstOffset(10) .setLastOffset(19) .setDeliveryCount(1) - ).asJava)) - ).asJava) + ))) + )) ) val cachedSharePartitions = new ImplicitLinkedHashCollection[CachedSharePartition] @@ -5986,12 +5981,12 @@ class KafkaApisTest extends Logging { any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - ).asJava) + )) ) var shareFetchRequestData = new ShareFetchRequestData(). @@ -6134,9 +6129,9 @@ class KafkaApisTest extends Logging { )) val authorizer: Authorizer = mock(classOf[Authorizer]) - when(authorizer.authorize(any(), any())).thenReturn(List[AuthorizationResult]( + when(authorizer.authorize(any(), any())).thenReturn(util.List.of[AuthorizationResult]( AuthorizationResult.DENIED - ).asJava) + )) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6167,12 +6162,12 @@ class KafkaApisTest extends Logging { val groupId = "group" when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - ).asJava) + )) ) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( @@ -6190,18 +6185,18 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(ShareRequestMetadata.FINAL_EPOCH). - setTopics(List(new ShareFetchRequestData.FetchTopic(). + setTopics(util.List.of(new ShareFetchRequestData.FetchTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareFetchRequestData.FetchPartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava)).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) @@ -6241,12 +6236,12 @@ class KafkaApisTest extends Logging { any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) doNothing().when(sharePartitionManager).acknowledgeSessionUpdate(any(), any()) @@ -6255,19 +6250,18 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6302,19 +6296,18 @@ class KafkaApisTest extends Logging { .setGroupId(groupId) .setMemberId(memberId.toString) .setShareSessionEpoch(1) - .setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) @@ -6348,24 +6341,23 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val authorizer: Authorizer = mock(classOf[Authorizer]) - when(authorizer.authorize(any(), any())).thenReturn(List[AuthorizationResult]( + when(authorizer.authorize(any(), any())).thenReturn(util.List.of[AuthorizationResult]( AuthorizationResult.DENIED - ).asJava) + )) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6404,19 +6396,18 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6455,19 +6446,18 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(0). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6505,19 +6495,18 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(4) // end offset is less than base offset - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6566,12 +6555,12 @@ class KafkaApisTest extends Logging { any(), any(), any() - )).thenReturn(CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) -> + )).thenReturn(CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) - ).asJava)) + ))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -6580,19 +6569,18 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(20) - .setAcknowledgeTypes(util.Arrays.asList(1.toByte,1.toByte,0.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte,1.toByte,0.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte,1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6643,19 +6631,18 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6687,42 +6674,41 @@ class KafkaApisTest extends Logging { any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) doNothing().when(sharePartitionManager).acknowledgeSessionUpdate(any(), any()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData(). setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(ShareRequestMetadata.FINAL_EPOCH). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6760,12 +6746,12 @@ class KafkaApisTest extends Logging { any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) when(sharePartitionManager.acknowledge(any(), any(), any())).thenReturn( - CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId, new TopicPartition(topicName, 0)) -> + CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId, new TopicPartition(topicName, 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava) + )) ) doNothing().when(sharePartitionManager).acknowledgeSessionUpdate(any(), any()) @@ -6778,19 +6764,18 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(ShareRequestMetadata.FINAL_EPOCH). - setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic(). + setTopics(util.List.of(new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(partitionIndex) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )))) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -6971,44 +6956,44 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(Uuid.randomUuid().toString). setShareSessionEpoch(0). - setTopics(List( + setTopics(util.List.of( new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)), + .setAcknowledgeTypes(util.List.of(1.toByte)), new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(10) .setLastOffset(17) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)) + )), new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(1) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(2.toByte)) - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of(2.toByte)) + )) + )), new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(24) .setLastOffset(65) - .setAcknowledgeTypes(Collections.singletonList(3.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(3.toByte)) + )) + )) + )) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val topicNames = new util.HashMap[Uuid, String] @@ -7043,40 +7028,40 @@ class KafkaApisTest extends Logging { setGroupId("group"). setMemberId(Uuid.randomUuid().toString). setShareSessionEpoch(0). - setTopics(List( + setTopics(util.List.of( new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(7.toByte)) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) - ).asJava), + .setAcknowledgeTypes(util.List.of(7.toByte)) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) + )), new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(1) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.emptyList()) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of) // wrong acknowledgement type here (can only be 0, 1, 2 or 3) + )) + )), new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(24) .setLastOffset(65) - .setAcknowledgeTypes(Collections.singletonList(3.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(3.toByte)) + )) + )) + )) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val topicIdNames = new util.HashMap[Uuid, String] @@ -7123,32 +7108,32 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.acknowledge(any(), any(), any())) - .thenReturn(CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - tp1 -> + .thenReturn(CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + tp1, new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - tp2 -> + tp2, new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - tp3 -> + tp3, new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava)) + ))) val acknowledgementData = mutable.Map[TopicIdPartition, util.List[ShareAcknowledgementBatch]]() - acknowledgementData += (tp1 -> util.Arrays.asList( - new ShareAcknowledgementBatch(0, 9, Collections.singletonList(1.toByte)), - new ShareAcknowledgementBatch(10, 19, Collections.singletonList(2.toByte)) + acknowledgementData += (tp1 -> util.List.of( + new ShareAcknowledgementBatch(0, 9, util.List.of(1.toByte)), + new ShareAcknowledgementBatch(10, 19, util.List.of(2.toByte)) )) - acknowledgementData += (tp2 -> util.Arrays.asList( - new ShareAcknowledgementBatch(5, 19, Collections.singletonList(2.toByte)) + acknowledgementData += (tp2 -> util.List.of( + new ShareAcknowledgementBatch(5, 19, util.List.of(2.toByte)) )) - acknowledgementData += (tp3 -> util.Arrays.asList( - new ShareAcknowledgementBatch(34, 56, Collections.singletonList(1.toByte)) + acknowledgementData += (tp3 -> util.List.of( + new ShareAcknowledgementBatch(34, 56, util.List.of(1.toByte)) )) val authorizedTopics: Set[String] = Set(topicName1, topicName2) @@ -7202,32 +7187,32 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.acknowledge(any(), any(), any())) - .thenReturn(CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId1, new TopicPartition("foo1", 0)) -> + .thenReturn(CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId1, new TopicPartition("foo1", 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition("foo2", 0)) -> + new TopicIdPartition(topicId2, new TopicPartition("foo2", 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition("foo2", 1)) -> + new TopicIdPartition(topicId2, new TopicPartition("foo2", 1)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava)) + ))) val acknowledgementData = mutable.Map[TopicIdPartition, util.List[ShareAcknowledgementBatch]]() - acknowledgementData += (tp1 -> util.Arrays.asList( - new ShareAcknowledgementBatch(39, 24, Collections.singletonList(1.toByte)), // this is an invalid batch because last offset is less than base offset - new ShareAcknowledgementBatch(43, 56, Collections.singletonList(2.toByte)) + acknowledgementData += (tp1 -> util.List.of( + new ShareAcknowledgementBatch(39, 24, util.List.of(1.toByte)), // this is an invalid batch because last offset is less than base offset + new ShareAcknowledgementBatch(43, 56, util.List.of(2.toByte)) )) - acknowledgementData += (tp2 -> util.Arrays.asList( - new ShareAcknowledgementBatch(5, 19, util.Arrays.asList(0.toByte, 2.toByte)) + acknowledgementData += (tp2 -> util.List.of( + new ShareAcknowledgementBatch(5, 19, util.List.of(0.toByte, 2.toByte)) )) - acknowledgementData += (tp3 -> util.Arrays.asList( - new ShareAcknowledgementBatch(34, 56, Collections.singletonList(1.toByte)), - new ShareAcknowledgementBatch(10, 19, Collections.singletonList(1.toByte)) // this is an invalid batch because start is offset is less than previous end offset + acknowledgementData += (tp3 -> util.List.of( + new ShareAcknowledgementBatch(34, 56, util.List.of(1.toByte)), + new ShareAcknowledgementBatch(10, 19, util.List.of(1.toByte)) // this is an invalid batch because start is offset is less than previous end offset )) val authorizedTopics: Set[String] = Set(topicName1, topicName2) @@ -7281,33 +7266,33 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.acknowledge(any(), any(), any())) - .thenReturn(CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - new TopicIdPartition(topicId1, new TopicPartition("foo1", 0)) -> + .thenReturn(CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + new TopicIdPartition(topicId1, new TopicPartition("foo1", 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition("foo2", 0)) -> + new TopicIdPartition(topicId2, new TopicPartition("foo2", 0)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - new TopicIdPartition(topicId2, new TopicPartition("foo2", 1)) -> + new TopicIdPartition(topicId2, new TopicPartition("foo2", 1)), new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(1) .setErrorCode(Errors.NONE.code) - ).asJava)) + ))) val acknowledgementData = mutable.Map[TopicIdPartition, util.List[ShareAcknowledgementBatch]]() - acknowledgementData += (tp1 -> util.Arrays.asList( - new ShareAcknowledgementBatch(24, 39, Collections.singletonList(1.toByte)), - new ShareAcknowledgementBatch(43, 56, Collections.singletonList(2.toByte)) + acknowledgementData += (tp1 -> util.List.of( + new ShareAcknowledgementBatch(24, 39, util.List.of(1.toByte)), + new ShareAcknowledgementBatch(43, 56, util.List.of(2.toByte)) )) - acknowledgementData += (tp2 -> util.Arrays.asList( - new ShareAcknowledgementBatch(5, 19, Collections.singletonList(2.toByte)) + acknowledgementData += (tp2 -> util.List.of( + new ShareAcknowledgementBatch(5, 19, util.List.of(2.toByte)) )) - acknowledgementData += (tp3 -> util.Arrays.asList( - new ShareAcknowledgementBatch(34, 56, Collections.singletonList(1.toByte)), - new ShareAcknowledgementBatch(67, 87, Collections.singletonList(1.toByte)) + acknowledgementData += (tp3 -> util.List.of( + new ShareAcknowledgementBatch(34, 56, util.List.of(1.toByte)), + new ShareAcknowledgementBatch(67, 87, util.List.of(1.toByte)) )) val authorizedTopics: Set[String] = Set(topicName1) // Topic with topicId2 is not authorized @@ -7361,25 +7346,25 @@ class KafkaApisTest extends Logging { val tp3 = new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) when(sharePartitionManager.acknowledge(any(), any(), any())) - .thenReturn(CompletableFuture.completedFuture(Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( - tp1 -> + .thenReturn(CompletableFuture.completedFuture(util.Map.of[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]( + tp1, new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code), - tp2 -> + tp2, new ShareAcknowledgeResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(Errors.NONE.code) - ).asJava)) + ))) val acknowledgementData = mutable.Map[TopicIdPartition, util.List[ShareAcknowledgementBatch]]() - acknowledgementData += (tp1 -> util.Arrays.asList( - new ShareAcknowledgementBatch(0, 9, Collections.singletonList(1.toByte)), - new ShareAcknowledgementBatch(10, 19, Collections.singletonList(2.toByte)) + acknowledgementData += (tp1 -> util.List.of( + new ShareAcknowledgementBatch(0, 9, util.List.of(1.toByte)), + new ShareAcknowledgementBatch(10, 19, util.List.of(2.toByte)) )) - acknowledgementData += (tp2 -> util.Arrays.asList( - new ShareAcknowledgementBatch(5, 19, Collections.singletonList(2.toByte)) + acknowledgementData += (tp2 -> util.List.of( + new ShareAcknowledgementBatch(5, 19, util.List.of(2.toByte)) )) val authorizedTopics: Set[String] = Set(topicName1, topicName2) @@ -7439,48 +7424,48 @@ class KafkaApisTest extends Logging { setGroupId(groupId). setMemberId(memberId.toString). setShareSessionEpoch(1). - setTopics(List( + setTopics(util.List.of( new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId1). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)) + )), new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(1) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )), new ShareAcknowledgeRequestData.AcknowledgeTopic(). setTopicId(topicId2). - setPartitions(List( + setPartitions(util.List.of( new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(0) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava), + .setAcknowledgeTypes(util.List.of(1.toByte)) + )), new ShareAcknowledgeRequestData.AcknowledgePartition() .setPartitionIndex(1) - .setAcknowledgementBatches(List( + .setAcknowledgementBatches(util.List.of( new ShareAcknowledgeRequestData.AcknowledgementBatch() .setFirstOffset(0) .setLastOffset(9) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) + .setAcknowledgeTypes(util.List.of(1.toByte)) + )) + )) + )) val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) @@ -7694,7 +7679,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7825,7 +7810,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7943,7 +7928,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleHeartbeatRequest( requestChannelRequest @@ -7959,15 +7944,14 @@ class KafkaApisTest extends Logging { def makeRequest(version: Short): RequestChannel.Request = { buildRequest(new LeaveGroupRequest.Builder( "group", - List( + util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1"), new MemberIdentity() .setMemberId("member-2") .setGroupInstanceId("instance-2") - ).asJava - ).build(version)) + )).build(version)) } if (version < 3) { @@ -7978,14 +7962,14 @@ class KafkaApisTest extends Logging { val expectedLeaveGroupRequest = new LeaveGroupRequestData() .setGroupId("group") - .setMembers(List( + .setMembers(util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1"), new MemberIdentity() .setMemberId("member-2") .setGroupInstanceId("instance-2") - ).asJava) + )) val future = new CompletableFuture[LeaveGroupResponseData]() when(groupCoordinator.leaveGroup( @@ -7997,14 +7981,14 @@ class KafkaApisTest extends Logging { val expectedLeaveResponse = new LeaveGroupResponseData() .setErrorCode(Errors.NONE.code) - .setMembers(List( + .setMembers(util.List.of( new LeaveGroupResponseData.MemberResponse() .setMemberId("member-1") .setGroupInstanceId("instance-1"), new LeaveGroupResponseData.MemberResponse() .setMemberId("member-2") .setGroupInstanceId("instance-2"), - ).asJava) + )) future.complete(expectedLeaveResponse) val response = verifyNoThrottling[LeaveGroupResponse](requestChannelRequest) @@ -8017,20 +8001,20 @@ class KafkaApisTest extends Logging { def testHandleLeaveGroupWithSingleMember(version: Short): Unit = { val requestChannelRequest = buildRequest(new LeaveGroupRequest.Builder( "group", - List( + util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava + ) ).build(version)) val expectedLeaveGroupRequest = new LeaveGroupRequestData() .setGroupId("group") - .setMembers(List( + .setMembers(util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId(if (version >= 3) "instance-1" else null) - ).asJava) + )) val future = new CompletableFuture[LeaveGroupResponseData]() when(groupCoordinator.leaveGroup( @@ -8042,20 +8026,20 @@ class KafkaApisTest extends Logging { val leaveGroupResponse = new LeaveGroupResponseData() .setErrorCode(Errors.NONE.code) - .setMembers(List( + .setMembers(util.List.of( new LeaveGroupResponseData.MemberResponse() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava) + )) val expectedLeaveResponse = if (version >= 3) { new LeaveGroupResponseData() .setErrorCode(Errors.NONE.code) - .setMembers(List( + .setMembers(util.List.of( new LeaveGroupResponseData.MemberResponse() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava) + )) } else { new LeaveGroupResponseData() .setErrorCode(Errors.NONE.code) @@ -8070,20 +8054,20 @@ class KafkaApisTest extends Logging { def testHandleLeaveGroupFutureFailed(): Unit = { val requestChannelRequest = buildRequest(new LeaveGroupRequest.Builder( "group", - List( + util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava + ) ).build(ApiKeys.LEAVE_GROUP.latestVersion)) val expectedLeaveGroupRequest = new LeaveGroupRequestData() .setGroupId("group") - .setMembers(List( + .setMembers(util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava) + )) val future = new CompletableFuture[LeaveGroupResponseData]() when(groupCoordinator.leaveGroup( @@ -8102,20 +8086,20 @@ class KafkaApisTest extends Logging { def testHandleLeaveGroupAuthenticationFailed(): Unit = { val requestChannelRequest = buildRequest(new LeaveGroupRequest.Builder( "group", - List( + util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava + ) ).build(ApiKeys.LEAVE_GROUP.latestVersion)) val expectedLeaveGroupRequest = new LeaveGroupRequestData() .setGroupId("group") - .setMembers(List( + .setMembers(util.List.of( new MemberIdentity() .setMemberId("member-1") .setGroupInstanceId("instance-1") - ).asJava) + )) val future = new CompletableFuture[LeaveGroupResponseData]() when(groupCoordinator.leaveGroup( @@ -8125,7 +8109,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -8138,10 +8122,10 @@ class KafkaApisTest extends Logging { def testHandleOffsetFetchWithMultipleGroups(version: Short): Unit = { def makeRequest(version: Short): RequestChannel.Request = { val groups = Map( - "group-1" -> List( + "group-1" -> util.List.of( new TopicPartition("foo", 0), new TopicPartition("foo", 1) - ).asJava, + ), "group-2" -> null, "group-3" -> null, "group-4" -> null, @@ -8160,10 +8144,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), + .setPartitionIndexes(util.List.of[Integer](0, 1)))), false )).thenReturn(group1Future) @@ -8198,10 +8182,10 @@ class KafkaApisTest extends Logging { val group1Response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8210,15 +8194,15 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(200) .setCommittedLeaderEpoch(2) - ).asJava) - ).asJava) + )) + )) val group2Response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-2") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8231,8 +8215,8 @@ class KafkaApisTest extends Logging { .setPartitionIndex(2) .setCommittedOffset(300) .setCommittedLeaderEpoch(3) - ).asJava) - ).asJava) + )) + )) val group3Response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-3") @@ -8261,10 +8245,10 @@ class KafkaApisTest extends Logging { buildRequest(new OffsetFetchRequest.Builder( "group-1", false, - List( + util.List.of( new TopicPartition("foo", 0), new TopicPartition("foo", 1) - ).asJava, + ), false ).build(version)) } @@ -8276,9 +8260,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(util.List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), + .setPartitionIndexes(util.List.of[Integer](0, 1)))), false )).thenReturn(future) kafkaApis = createKafkaApis() @@ -8286,10 +8270,10 @@ class KafkaApisTest extends Logging { val group1Response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8298,18 +8282,18 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(200) .setCommittedLeaderEpoch(2) - ).asJava) - ).asJava) + )) + )) val expectedOffsetFetchResponse = if (version >= 8) { new OffsetFetchResponseData() - .setGroups(List(group1Response).asJava) + .setGroups(util.List.of(group1Response)) } else { new OffsetFetchResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartition() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8318,8 +8302,8 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(200) .setCommittedLeaderEpoch(if (version >= 5) 2 else -1) - ).asJava) - ).asJava) + )) + )) } future.complete(group1Response) @@ -8359,10 +8343,10 @@ class KafkaApisTest extends Logging { val group1Response = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8371,18 +8355,18 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(200) .setCommittedLeaderEpoch(2) - ).asJava) - ).asJava) + )) + )) val expectedOffsetFetchResponse = if (version >= 8) { new OffsetFetchResponseData() - .setGroups(List(group1Response).asJava) + .setGroups(util.List.of(group1Response)) } else { new OffsetFetchResponseData() - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopic() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartition() .setPartitionIndex(0) .setCommittedOffset(100) @@ -8391,8 +8375,8 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(200) .setCommittedLeaderEpoch(if (version >= 5) 2 else -1) - ).asJava) - ).asJava) + )) + )) } future.complete(group1Response) @@ -8405,14 +8389,14 @@ class KafkaApisTest extends Logging { def testHandleOffsetFetchAuthorization(): Unit = { def makeRequest(version: Short): RequestChannel.Request = { val groups = Map( - "group-1" -> List( + "group-1" -> util.List.of( new TopicPartition("foo", 0), new TopicPartition("bar", 0) - ).asJava, - "group-2" -> List( + ), + "group-2" -> util.List.of( new TopicPartition("foo", 0), new TopicPartition("bar", 0) - ).asJava, + ), "group-3" -> null, "group-4" -> null, ).asJava @@ -8437,9 +8421,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } // group-1 is allowed and bar is allowed. @@ -8448,9 +8432,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(util.List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), + .setPartitionIndexes(util.List.of[Integer](0)))), false )).thenReturn(group1Future) @@ -8468,62 +8452,62 @@ class KafkaApisTest extends Logging { val group1ResponseFromCoordinator = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) .setCommittedLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) val group3ResponseFromCoordinator = new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-3") - .setTopics(List( + .setTopics(util.List.of( // foo should be filtered out. new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) .setCommittedLeaderEpoch(1) - ).asJava), + )), new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) .setCommittedLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) val expectedOffsetFetchResponse = new OffsetFetchResponseData() - .setGroups(List( + .setGroups(util.List.of( // group-1 is authorized but foo is not. new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) .setCommittedLeaderEpoch(1) - ).asJava), + )), new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("foo") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) .setCommittedOffset(-1) - ).asJava) - ).asJava), + )) + )), // group-2 is not authorized. new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-2") @@ -8531,21 +8515,21 @@ class KafkaApisTest extends Logging { // group-3 is authorized but foo is not. new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-3") - .setTopics(List( + .setTopics(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseTopics() .setName("bar") - .setPartitions(List( + .setPartitions(util.List.of( new OffsetFetchResponseData.OffsetFetchResponsePartitions() .setPartitionIndex(0) .setCommittedOffset(100) .setCommittedLeaderEpoch(1) - ).asJava) - ).asJava), + )) + )), // group-4 is not authorized. new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-4") .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code), - ).asJava) + )) group1Future.complete(group1ResponseFromCoordinator) group3Future.complete(group3ResponseFromCoordinator) @@ -8557,16 +8541,16 @@ class KafkaApisTest extends Logging { @Test def testHandleOffsetFetchWithUnauthorizedTopicAndTopLevelError(): Unit = { def makeRequest(version: Short): RequestChannel.Request = { - val groups = Map( - "group-1" -> List( + val groups = util.Map.of( + "group-1", util.List.of( new TopicPartition("foo", 0), new TopicPartition("bar", 0) - ).asJava, - "group-2" -> List( + ), + "group-2", util.List.of( new TopicPartition("foo", 0), new TopicPartition("bar", 0) - ).asJava - ).asJava + ) + ) buildRequest(new OffsetFetchRequest.Builder(groups, false, false).build(version)) } @@ -8586,9 +8570,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } // group-1 and group-2 are allowed and bar is allowed. @@ -8597,9 +8581,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(util.List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), + .setPartitionIndexes(util.List.of[Integer](0)))), false )).thenReturn(group1Future) @@ -8608,9 +8592,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() .setGroupId("group-2") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setTopics(util.List.of(new OffsetFetchRequestData.OffsetFetchRequestTopics() .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), + .setPartitionIndexes(util.List.of[Integer](0)))), false )).thenReturn(group1Future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) @@ -8624,12 +8608,12 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code) val expectedOffsetFetchResponse = new OffsetFetchResponseData() - .setGroups(List( + .setGroups(util.List.of( new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId("group-1") .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code), group2ResponseFromCoordinator - ).asJava) + )) group1Future.completeExceptionally(Errors.COORDINATOR_NOT_AVAILABLE.exception) group2Future.complete(group2ResponseFromCoordinator) @@ -8657,8 +8641,8 @@ class KafkaApisTest extends Logging { setupBasicMetadataCache(tp0.topic, numPartitions = 1, 1, topicId) val hw = 3 - val fetchDataBuilder = Collections.singletonMap(tp0, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, Int.MaxValue, Optional.of(leaderEpoch))) - val fetchData = Collections.singletonMap(tidp0, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, Int.MaxValue, Optional.of(leaderEpoch))) + val fetchDataBuilder = util.Map.of(tp0, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, Int.MaxValue, Optional.of(leaderEpoch))) + val fetchData = util.Map.of(tidp0, new FetchRequest.PartitionData(Uuid.ZERO_UUID, 0, 0, Int.MaxValue, Optional.of(leaderEpoch))) val fetchFromFollower = buildRequest(new FetchRequest.Builder( ApiKeys.FETCH.oldestVersion(), ApiKeys.FETCH.latestVersion(), 1, 1, 1000, 0, fetchDataBuilder).build()) @@ -8703,14 +8687,14 @@ class KafkaApisTest extends Logging { @ApiKeyVersionsSource(apiKey = ApiKeys.LIST_GROUPS) def testListGroupsRequest(version: Short): Unit = { val listGroupsRequest = new ListGroupsRequestData() - .setStatesFilter(if (version >= 4) List("Stable", "Empty").asJava else List.empty.asJava) - .setTypesFilter(if (version >= 5) List("classic", "consumer").asJava else List.empty.asJava) + .setStatesFilter(if (version >= 4) util.List.of("Stable", "Empty") else util.List.of) + .setTypesFilter(if (version >= 5) util.List.of("classic", "consumer") else util.List.of) val requestChannelRequest = buildRequest(new ListGroupsRequest.Builder(listGroupsRequest).build(version)) val expectedListGroupsRequest = new ListGroupsRequestData() - .setStatesFilter(if (version >= 4) List("Stable", "Empty").asJava else List.empty.asJava) - .setTypesFilter(if (version >= 5) List("classic", "consumer").asJava else List.empty.asJava) + .setStatesFilter(if (version >= 4) util.List.of("Stable", "Empty") else util.List.of) + .setTypesFilter(if (version >= 5) util.List.of("classic", "consumer") else util.List.of) val future = new CompletableFuture[ListGroupsResponseData]() when(groupCoordinator.listGroups( @@ -8721,7 +8705,7 @@ class KafkaApisTest extends Logging { kafkaApis.handleListGroupsRequest(requestChannelRequest) val expectedListGroupsResponse = new ListGroupsResponseData() - .setGroups(List( + .setGroups(util.List.of( new ListGroupsResponseData.ListedGroup() .setGroupId("group1") .setProtocolType("protocol1") @@ -8737,7 +8721,7 @@ class KafkaApisTest extends Logging { .setProtocolType("protocol3") .setGroupState(if (version >= 4) "Stable" else "") .setGroupType(if (version >= 5) "classic" else ""), - ).asJava) + )) future.complete(expectedListGroupsResponse) val response = verifyNoThrottling[ListGroupsResponse](requestChannelRequest) @@ -8747,14 +8731,14 @@ class KafkaApisTest extends Logging { @Test def testListGroupsRequestFutureFailed(): Unit = { val listGroupsRequest = new ListGroupsRequestData() - .setStatesFilter(List("Stable", "Empty").asJava) - .setTypesFilter(List("classic", "consumer").asJava) + .setStatesFilter(util.List.of("Stable", "Empty")) + .setTypesFilter(util.List.of("classic", "consumer")) val requestChannelRequest = buildRequest(new ListGroupsRequest.Builder(listGroupsRequest).build()) val expectedListGroupsRequest = new ListGroupsRequestData() - .setStatesFilter(List("Stable", "Empty").asJava) - .setTypesFilter(List("classic", "consumer").asJava) + .setStatesFilter(util.List.of("Stable", "Empty")) + .setTypesFilter(util.List.of("classic", "consumer")) val future = new CompletableFuture[ListGroupsResponseData]() when(groupCoordinator.listGroups( @@ -8917,7 +8901,7 @@ class KafkaApisTest extends Logging { assertEquals(clusterId, describeClusterResponse.data.clusterId) assertEquals(8096, describeClusterResponse.data.clusterAuthorizedOperations) assertEquals(util.Set.copyOf(metadataCache.getAliveBrokerNodes(plaintextListener)), - describeClusterResponse.nodes.asScala.values.toSet.asJava) + new util.HashSet(describeClusterResponse.nodes.values)) } /** @@ -8971,11 +8955,11 @@ class KafkaApisTest extends Logging { private def testConsumerListOffsetWithUnsupportedVersion(timestamp: Long, version: Short): Unit = { val tp = new TopicPartition("foo", 0) - val targetTimes = List(new ListOffsetsTopic() + val targetTimes = util.List.of(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetsPartition() + .setPartitions(util.List.of(new ListOffsetsPartition() .setPartitionIndex(tp.partition) - .setTimestamp(timestamp)).asJava)).asJava + .setTimestamp(timestamp)))) when(replicaManager.fetchOffset( ArgumentMatchers.any[Seq[ListOffsetsTopic]](), @@ -9000,7 +8984,7 @@ class KafkaApisTest extends Logging { .setOffset(ListOffsetsResponse.UNKNOWN_OFFSET) .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) .setPartitionIndex(tp.partition()) - callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(List(partitionResponse).asJava))) + callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(util.List.of(partitionResponse)))) }) val data = new ListOffsetsRequestData().setTopics(targetTimes).setReplicaId(ListOffsetsRequest.CONSUMER_REPLICA_ID) @@ -9023,11 +9007,11 @@ class KafkaApisTest extends Logging { val tp = new TopicPartition("foo", 0) val latestOffset = 15L - val targetTimes = List(new ListOffsetsTopic() + val targetTimes = util.List.of(new ListOffsetsTopic() .setName(tp.topic) - .setPartitions(List(new ListOffsetsPartition() + .setPartitions(util.List.of(new ListOffsetsPartition() .setPartitionIndex(tp.partition) - .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)).asJava)).asJava + .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)))) when(replicaManager.fetchOffset( ArgumentMatchers.eq(targetTimes.asScala.toSeq), @@ -9047,7 +9031,7 @@ class KafkaApisTest extends Logging { .setOffset(latestOffset) .setTimestamp(ListOffsetsResponse.UNKNOWN_TIMESTAMP) .setPartitionIndex(tp.partition()) - callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(List(partitionResponse).asJava))) + callback.accept(util.List.of(new ListOffsetsTopicResponse().setName(tp.topic()).setPartitions(util.List.of(partitionResponse)))) }) val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) @@ -9069,7 +9053,7 @@ class KafkaApisTest extends Logging { private def createWriteTxnMarkersRequest(partitions: util.List[TopicPartition]) = { val writeTxnMarkersRequest = new WriteTxnMarkersRequest.Builder( - asList(new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, partitions))).build() + util.List.of(new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, partitions))).build() (writeTxnMarkersRequest, buildRequest(writeTxnMarkersRequest)) } @@ -9156,7 +9140,7 @@ class KafkaApisTest extends Logging { val topicRecord = new TopicRecord().setName(topic).setTopicId(topicId) results += topicRecord - val replicas = List(0.asInstanceOf[Integer]).asJava + val replicas = util.List.of(0.asInstanceOf[Integer]) def createPartitionRecord(partition: Int) = new PartitionRecord() .setTopicId(topicId) @@ -9211,7 +9195,7 @@ class KafkaApisTest extends Logging { val data = new AlterReplicaLogDirsRequestData() val dir = new AlterReplicaLogDirsRequestData.AlterReplicaLogDir() .setPath("/foo") - dir.topics().add(new AlterReplicaLogDirsRequestData.AlterReplicaLogDirTopic().setName("t0").setPartitions(asList(0, 1, 2))) + dir.topics().add(new AlterReplicaLogDirsRequestData.AlterReplicaLogDirTopic().setName("t0").setPartitions(util.List.of(0, 1, 2))) data.dirs().add(dir) val alterReplicaLogDirsRequest = new AlterReplicaLogDirsRequest.Builder( data @@ -9243,9 +9227,9 @@ class KafkaApisTest extends Logging { new TopicPartition(tr.topicName, pr.partitionIndex) -> Errors.forCode(pr.errorCode) } }.toMap) - assertEquals(Map(Errors.NONE -> 1, - Errors.LOG_DIR_NOT_FOUND -> 1, - Errors.INVALID_TOPIC_EXCEPTION -> 1).asJava, response.errorCounts) + assertEquals(util.Map.of(Errors.NONE, 1, + Errors.LOG_DIR_NOT_FOUND, 1, + Errors.INVALID_TOPIC_EXCEPTION, 1), response.errorCounts) } @Test @@ -9293,45 +9277,45 @@ class KafkaApisTest extends Logging { val tp4 = new TopicPartition("invalid;topic", 1) val authorizer: Authorizer = mock(classOf[Authorizer]) - val data = new DescribeProducersRequestData().setTopics(List( + val data = new DescribeProducersRequestData().setTopics(util.List.of( new DescribeProducersRequestData.TopicRequest() .setName(tp1.topic) - .setPartitionIndexes(List(Int.box(tp1.partition)).asJava), + .setPartitionIndexes(util.List.of(Int.box(tp1.partition))), new DescribeProducersRequestData.TopicRequest() .setName(tp2.topic) - .setPartitionIndexes(List(Int.box(tp2.partition)).asJava), + .setPartitionIndexes(util.List.of(Int.box(tp2.partition))), new DescribeProducersRequestData.TopicRequest() .setName(tp3.topic) - .setPartitionIndexes(List(Int.box(tp3.partition)).asJava), + .setPartitionIndexes(util.List.of(Int.box(tp3.partition))), new DescribeProducersRequestData.TopicRequest() .setName(tp4.topic) - .setPartitionIndexes(List(Int.box(tp4.partition)).asJava) - ).asJava) + .setPartitionIndexes(util.List.of(Int.box(tp4.partition))) + )) def buildExpectedActions(topic: String): util.List[Action] = { val pattern = new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL) val action = new Action(AclOperation.READ, pattern, 1, true, true) - Collections.singletonList(action) + util.List.of(action) } // Topic `foo` is authorized and present in the metadata addTopicToMetadataCache(tp1.topic, 4) // We will only access the first topic when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions(tp1.topic)))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) // Topic `bar` is not authorized when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions(tp2.topic)))) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) // Topic `baz` is authorized, but not present in the metadata when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions(tp3.topic)))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) when(replicaManager.activeProducerState(tp1)) .thenReturn(new DescribeProducersResponseData.PartitionResponse() .setErrorCode(Errors.NONE.code) .setPartitionIndex(tp1.partition) - .setActiveProducers(List( + .setActiveProducers(util.List.of( new DescribeProducersResponseData.ProducerState() .setProducerId(12345L) .setProducerEpoch(15) @@ -9339,7 +9323,7 @@ class KafkaApisTest extends Logging { .setLastTimestamp(time.milliseconds()) .setCurrentTxnStartOffset(-1) .setCoordinatorEpoch(200) - ).asJava)) + ))) val describeProducersRequest = new DescribeProducersRequest.Builder(data).build() val request = buildRequest(describeProducersRequest) @@ -9381,7 +9365,7 @@ class KafkaApisTest extends Logging { def testDescribeTransactions(): Unit = { val authorizer: Authorizer = mock(classOf[Authorizer]) val data = new DescribeTransactionsRequestData() - .setTransactionalIds(List("foo", "bar").asJava) + .setTransactionalIds(util.List.of("foo", "bar")) val describeTransactionsRequest = new DescribeTransactionsRequest.Builder(data).build() val request = buildRequest(describeTransactionsRequest) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), @@ -9390,7 +9374,7 @@ class KafkaApisTest extends Logging { def buildExpectedActions(transactionalId: String): util.List[Action] = { val pattern = new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL) val action = new Action(AclOperation.DESCRIBE, pattern, 1, true, true) - Collections.singletonList(action) + util.List.of(action) } when(txnCoordinator.handleDescribeTransactions("foo")) @@ -9404,10 +9388,10 @@ class KafkaApisTest extends Logging { .setTransactionTimeoutMs(10000)) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("foo")))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("bar")))) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDescribeTransactionsRequest(request) @@ -9432,7 +9416,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) val transactionalId = "foo" val data = new DescribeTransactionsRequestData() - .setTransactionalIds(List(transactionalId).asJava) + .setTransactionalIds(util.List.of(transactionalId)) val describeTransactionsRequest = new DescribeTransactionsRequest.Builder(data).build() val request = buildRequest(describeTransactionsRequest) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), @@ -9445,10 +9429,10 @@ class KafkaApisTest extends Logging { ): Unit = { val pattern = new ResourcePattern(resourceType, transactionalId, PatternType.LITERAL) val action = new Action(AclOperation.DESCRIBE, pattern, 1, true, true) - val actions = Collections.singletonList(action) + val actions = util.List.of(action) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(actions))) - .thenReturn(Seq(result).asJava) + .thenReturn(util.List.of(result)) } // Principal is authorized to one of the two topics. The second topic should be @@ -9542,14 +9526,14 @@ class KafkaApisTest extends Logging { def buildExpectedActions(transactionalId: String): util.List[Action] = { val pattern = new ResourcePattern(ResourceType.TRANSACTIONAL_ID, transactionalId, PatternType.LITERAL) val action = new Action(AclOperation.DESCRIBE, pattern, 1, true, true) - Collections.singletonList(action) + util.List.of(action) } when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("foo")))) - .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("bar")))) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleListTransactionsRequest(request) @@ -9577,11 +9561,11 @@ class KafkaApisTest extends Logging { def testInvalidLegacyAlterConfigsRequestWithKRaft(): Unit = { val request = buildRequest(new AlterConfigsRequest(new AlterConfigsRequestData(). setValidateOnly(true). - setResources(new LAlterConfigsResourceCollection(asList( + setResources(new LAlterConfigsResourceCollection(util.List.of( new LAlterConfigsResource(). setResourceName(brokerId.toString). setResourceType(BROKER.id()). - setConfigs(new LAlterableConfigCollection(asList(new LAlterableConfig(). + setConfigs(new LAlterableConfigCollection(util.List.of(new LAlterableConfig(). setName("foo"). setValue(null)).iterator()))).iterator())), 1.toShort)) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) @@ -9590,7 +9574,7 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis() kafkaApis.handleAlterConfigsRequest(request) val response = verifyNoThrottling[AlterConfigsResponse](request) - assertEquals(new AlterConfigsResponseData().setResponses(asList( + assertEquals(new AlterConfigsResponseData().setResponses(util.List.of( new LAlterConfigsResourceResponse(). setErrorCode(Errors.INVALID_REQUEST.code()). setErrorMessage("Null value not supported for : foo"). @@ -9615,10 +9599,10 @@ class KafkaApisTest extends Logging { def testLog4jIncrementalAlterConfigsRequestWithKRaft(): Unit = { val request = buildRequest(new IncrementalAlterConfigsRequest(new IncrementalAlterConfigsRequestData(). setValidateOnly(true). - setResources(new IAlterConfigsResourceCollection(asList(new IAlterConfigsResource(). + setResources(new IAlterConfigsResourceCollection(util.List.of(new IAlterConfigsResource(). setResourceName(brokerId.toString). setResourceType(BROKER_LOGGER.id()). - setConfigs(new IAlterableConfigCollection(asList(new IAlterableConfig(). + setConfigs(new IAlterableConfigCollection(util.List.of(new IAlterableConfig(). setName(LoggingController.ROOT_LOGGER). setValue("TRACE")).iterator()))).iterator())), 1.toShort)) @@ -9628,7 +9612,7 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis() kafkaApis.handleIncrementalAlterConfigsRequest(request) val response = verifyNoThrottling[IncrementalAlterConfigsResponse](request) - assertEquals(new IncrementalAlterConfigsResponseData().setResponses(asList( + assertEquals(new IncrementalAlterConfigsResponseData().setResponses(util.List.of( new IAlterConfigsResourceResponse(). setErrorCode(0.toShort). setErrorMessage(null). @@ -9644,7 +9628,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) metadataCache = { val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); + val delta = new MetadataDelta(MetadataImage.EMPTY) delta.replay(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) @@ -9720,7 +9704,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis( authorizer = Some(authorizer), featureVersions = Seq(GroupVersion.GV_1) @@ -9741,7 +9725,7 @@ class KafkaApisTest extends Logging { val consumerGroupHeartbeatRequest = new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) - .setSubscribedTopicNames(List(fooTopicName, barTopicName, zarTopicName).asJava) + .setSubscribedTopicNames(util.List.of(fooTopicName, barTopicName, zarTopicName)) val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) @@ -9756,9 +9740,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } kafkaApis = createKafkaApis( @@ -9778,7 +9762,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) metadataCache = { val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); + val delta = new MetadataDelta(MetadataImage.EMPTY) delta.replay(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) @@ -9816,7 +9800,7 @@ class KafkaApisTest extends Logging { val streamsGroupHeartbeatResponse = new StreamsGroupHeartbeatResponseData() .setMemberId("member") - future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, Collections.emptyMap())) + future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, util.Map.of())) val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest) assertEquals(streamsGroupHeartbeatResponse, response.data) } @@ -9837,12 +9821,12 @@ class KafkaApisTest extends Logging { .setSubtopologies( util.List.of( new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology1") - .setSourceTopics(Collections.singletonList(fooTopicName)) - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) - .setStateChangelogTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(tarTopicName))), + .setSourceTopics(util.List.of(fooTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) + .setStateChangelogTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(tarTopicName))), new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology2") - .setSourceTopics(Collections.singletonList(zarTopicName)) - .setRepartitionSourceTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(barTopicName))) + .setSourceTopics(util.List.of(zarTopicName)) + .setRepartitionSourceTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(barTopicName))) ) ) ) @@ -9863,9 +9847,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val future = new CompletableFuture[StreamsGroupHeartbeatResult]() @@ -9882,7 +9866,7 @@ class KafkaApisTest extends Logging { val streamsGroupHeartbeatResponse = new StreamsGroupHeartbeatResponseData() .setMemberId("member") - future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, Collections.emptyMap())) + future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, util.Map.of())) val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest) assertEquals(streamsGroupHeartbeatResponse, response.data) } @@ -9920,7 +9904,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -9945,10 +9929,10 @@ class KafkaApisTest extends Logging { .setEpoch(3) .setSubtopologies( util.List.of(new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology") - .setSourceTopics(Collections.singletonList(fooTopicName)) - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) - .setRepartitionSourceTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(zarTopicName))) - .setStateChangelogTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(tarTopicName))) + .setSourceTopics(util.List.of(fooTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) + .setRepartitionSourceTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(zarTopicName))) + .setStateChangelogTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName(tarTopicName))) ) ) ) @@ -9968,9 +9952,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } kafkaApis = createKafkaApis( @@ -10008,11 +9992,11 @@ class KafkaApisTest extends Logging { new StreamsGroupHeartbeatRequestData.Topology() .setEpoch(3) .setSubtopologies( - Collections.singletonList(new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology") - .setSourceTopics(Collections.singletonList("a ")) - .setRepartitionSinkTopics(Collections.singletonList("b?")) - .setRepartitionSourceTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("c!"))) - .setStateChangelogTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("d/"))) + util.List.of(new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology") + .setSourceTopics(util.List.of("a ")) + .setRepartitionSinkTopics(util.List.of("b?")) + .setRepartitionSourceTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("c!"))) + .setStateChangelogTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("d/"))) ) ) ) @@ -10037,10 +10021,10 @@ class KafkaApisTest extends Logging { new StreamsGroupHeartbeatRequestData.Topology() .setEpoch(3) .setSubtopologies( - Collections.singletonList(new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology") - .setSourceTopics(Collections.singletonList("__consumer_offsets")) - .setRepartitionSinkTopics(Collections.singletonList("__transaction_state")) - .setRepartitionSourceTopics(Collections.singletonList(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("__share_group_state"))) + util.List.of(new StreamsGroupHeartbeatRequestData.Subtopology().setSubtopologyId("subtopology") + .setSourceTopics(util.List.of("__consumer_offsets")) + .setRepartitionSinkTopics(util.List.of("__transaction_state")) + .setRepartitionSourceTopics(util.List.of(new StreamsGroupHeartbeatRequestData.TopicInfo().setName("__share_group_state"))) ) ) ) @@ -10061,7 +10045,7 @@ class KafkaApisTest extends Logging { def testStreamsGroupHeartbeatRequestWithInternalTopicsToCreate(): Unit = { metadataCache = mock(classOf[KRaftMetadataCache]) - val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group"); + val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) @@ -10090,7 +10074,7 @@ class KafkaApisTest extends Logging { def testStreamsGroupHeartbeatRequestWithInternalTopicsToCreateMissingCreateACL(): Unit = { metadataCache = mock(classOf[KRaftMetadataCache]) - val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group"); + val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) @@ -10103,15 +10087,18 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], isNotNull[util.List[Action]])).thenAnswer(invocation => { val actions = invocation.getArgument(1).asInstanceOf[util.List[Action]] - actions.asScala.map { action => - if (action.resourcePattern.name.equals("test") && action.operation() == AclOperation.CREATE && action.resourcePattern().resourceType() == ResourceType.TOPIC) { + val results: util.List[AuthorizationResult] = new util.ArrayList[AuthorizationResult](actions.size()) + actions.forEach { action => + val result = if (action.resourcePattern.name == "test" && action.operation == AclOperation.CREATE && action.resourcePattern.resourceType == ResourceType.TOPIC) { AuthorizationResult.DENIED - } else if (action.operation() == AclOperation.CREATE && action.resourcePattern().resourceType() == ResourceType.CLUSTER) { + } else if (action.operation == AclOperation.CREATE && action.resourcePattern.resourceType == ResourceType.CLUSTER) { AuthorizationResult.DENIED } else { AuthorizationResult.ALLOWED } - }.asJava + results.add(result) + } + results }) kafkaApis = createKafkaApis( authorizer = Some(authorizer), @@ -10144,7 +10131,7 @@ class KafkaApisTest extends Logging { val barTopicName = "bar" metadataCache = mock(classOf[KRaftMetadataCache]) - val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava + val groupIds = util.List.of("group-id-0", "group-id-1", "group-id-2") val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() .setIncludeAuthorizedOperations(includeAuthorizedOperations) consumerGroupDescribeRequestData.groupIds.addAll(groupIds) @@ -10163,42 +10150,42 @@ class KafkaApisTest extends Logging { val member0 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member0") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) val member1 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member1") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( + .setTopicPartitions(util.List.of( new TopicPartitions().setTopicName(fooTopicName), - new TopicPartitions().setTopicName(barTopicName)).asJava)) + new TopicPartitions().setTopicName(barTopicName)))) val member2 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member2") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(barTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(barTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) - future.complete(List( + future.complete(util.List.of( new DescribedGroup() .setGroupId(groupIds.get(0)) - .setMembers(List(member0).asJava), + .setMembers(util.List.of(member0)), new DescribedGroup() .setGroupId(groupIds.get(1)) - .setMembers(List(member0, member1).asJava), + .setMembers(util.List.of(member0, member1)), new DescribedGroup() .setGroupId(groupIds.get(2)) - .setMembers(List(member2).asJava) - ).asJava) + .setMembers(util.List.of(member2)) + )) var authorizedOperationsInt = Int.MinValue if (includeAuthorizedOperations) { @@ -10211,13 +10198,13 @@ class KafkaApisTest extends Logging { val describedGroups = List( new DescribedGroup() .setGroupId(groupIds.get(0)) - .setMembers(List(member0).asJava), + .setMembers(util.List.of(member0)), new DescribedGroup() .setGroupId(groupIds.get(1)) - .setMembers(List(member0, member1).asJava), + .setMembers(util.List.of(member0, member1)), new DescribedGroup() .setGroupId(groupIds.get(2)) - .setMembers(List(member2).asJava) + .setMembers(util.List.of(member2)) ).map(group => group.setAuthorizedOperations(authorizedOperationsInt)) val expectedConsumerGroupDescribeResponseData = new ConsumerGroupDescribeResponseData() .setGroups(describedGroups.asJava) @@ -10240,7 +10227,7 @@ class KafkaApisTest extends Logging { expectedResponse.groups.add(expectedDescribedGroup) metadataCache = { val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); + val delta = new MetadataDelta(MetadataImage.EMPTY) delta.replay(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) @@ -10265,14 +10252,14 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() when(groupCoordinator.consumerGroupDescribe( any[RequestContext], any[util.List[String]] )).thenReturn(future) - future.complete(List().asJava) + future.complete(util.List.of) kafkaApis = createKafkaApis( authorizer = Some(authorizer), featureVersions = Seq(GroupVersion.GV_1) @@ -10313,7 +10300,7 @@ class KafkaApisTest extends Logging { val fooTopicName = "foo" val barTopicName = "bar" - val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava + val groupIds = util.List.of("group-id-0", "group-id-1", "group-id-2") val streamsGroupDescribeRequestData = new StreamsGroupDescribeRequestData() .setIncludeAuthorizedOperations(includeAuthorizedOperations) streamsGroupDescribeRequestData.groupIds.addAll(groupIds) @@ -10331,33 +10318,33 @@ class KafkaApisTest extends Logging { val subtopology0 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology0") - .setSourceTopics(Collections.singletonList(fooTopicName)) + .setSourceTopics(util.List.of(fooTopicName)) val subtopology1 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology1") - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) val subtopology2 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology2") - .setSourceTopics(Collections.singletonList(fooTopicName)) - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) + .setSourceTopics(util.List.of(fooTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) - future.complete(List( + future.complete(util.List.of( new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(0)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology0))), + .setSubtopologies(util.List.of(subtopology0))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(1)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology1))), + .setSubtopologies(util.List.of(subtopology1))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(2)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology2))) - ).asJava) + .setSubtopologies(util.List.of(subtopology2))) + )) - var authorizedOperationsInt = Int.MinValue; + var authorizedOperationsInt = Int.MinValue if (includeAuthorizedOperations) { authorizedOperationsInt = Utils.to32BitField( AclEntry.supportedOperations(ResourceType.GROUP).asScala @@ -10369,15 +10356,15 @@ class KafkaApisTest extends Logging { new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(0)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology0))), + .setSubtopologies(util.List.of(subtopology0))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(1)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology1))), + .setSubtopologies(util.List.of(subtopology1))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(2)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology2))) + .setSubtopologies(util.List.of(subtopology2))) ).map(group => group.setAuthorizedOperations(authorizedOperationsInt)) val expectedStreamsGroupDescribeResponseData = new StreamsGroupDescribeResponseData() .setGroups(describedGroups.asJava) @@ -10400,7 +10387,7 @@ class KafkaApisTest extends Logging { expectedResponse.groups.add(expectedDescribedGroup) metadataCache = { val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); + val delta = new MetadataDelta(MetadataImage.EMPTY) delta.replay(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) @@ -10425,14 +10412,14 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() when(groupCoordinator.streamsGroupDescribe( any[RequestContext], any[util.List[String]] )).thenReturn(future) - future.complete(List().asJava) + future.complete(util.List.of) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -10475,7 +10462,7 @@ class KafkaApisTest extends Logging { metadataCache = mock(classOf[KRaftMetadataCache]) - val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava + val groupIds = util.List.of("group-id-0", "group-id-1", "group-id-2") val streamsGroupDescribeRequestData = new StreamsGroupDescribeRequestData() .setIncludeAuthorizedOperations(includeAuthorizedOperations) streamsGroupDescribeRequestData.groupIds.addAll(groupIds) @@ -10494,9 +10481,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() @@ -10512,31 +10499,31 @@ class KafkaApisTest extends Logging { val subtopology0 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology0") - .setSourceTopics(Collections.singletonList(fooTopicName)) + .setSourceTopics(util.List.of(fooTopicName)) val subtopology1 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology1") - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) val subtopology2 = new StreamsGroupDescribeResponseData.Subtopology() .setSubtopologyId("subtopology2") - .setSourceTopics(Collections.singletonList(fooTopicName)) - .setRepartitionSinkTopics(Collections.singletonList(barTopicName)) + .setSourceTopics(util.List.of(fooTopicName)) + .setRepartitionSinkTopics(util.List.of(barTopicName)) - future.complete(List( + future.complete(util.List.of( new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(0)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology0))), + .setSubtopologies(util.List.of(subtopology0))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(1)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology1))), + .setSubtopologies(util.List.of(subtopology1))), new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(2)) .setTopology(new StreamsGroupDescribeResponseData.Topology() - .setSubtopologies(Collections.singletonList(subtopology2))) - ).asJava) + .setSubtopologies(util.List.of(subtopology2))) + )) val response = verifyNoThrottling[StreamsGroupDescribeResponse](requestChannelRequest) assertNotNull(response.data) @@ -10556,7 +10543,7 @@ class KafkaApisTest extends Logging { metadataCache = mock(classOf[KRaftMetadataCache]) - val groupIds = List("group-id-0", "group-id-1", "group-id-2").asJava + val groupIds = util.List.of("group-id-0", "group-id-1", "group-id-2") val consumerGroupDescribeRequestData = new ConsumerGroupDescribeRequestData() .setGroupIds(groupIds) val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) @@ -10574,9 +10561,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() @@ -10593,48 +10580,48 @@ class KafkaApisTest extends Logging { val member0 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member0") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) val member1 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member1") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( + .setTopicPartitions(util.List.of( new TopicPartitions().setTopicName(fooTopicName), - new TopicPartitions().setTopicName(barTopicName)).asJava)) + new TopicPartitions().setTopicName(barTopicName)))) val member2 = new ConsumerGroupDescribeResponseData.Member() .setMemberId("member2") .setAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(barTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(barTopicName)))) .setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new TopicPartitions().setTopicName(fooTopicName)))) - future.complete(List( + future.complete(util.List.of( new DescribedGroup() .setGroupId(groupIds.get(0)) - .setMembers(List(member0).asJava), + .setMembers(util.List.of(member0)), new DescribedGroup() .setGroupId(groupIds.get(1)) - .setMembers(List(member0, member1).asJava), + .setMembers(util.List.of(member0, member1)), new DescribedGroup() .setGroupId(groupIds.get(2)) - .setMembers(List(member2).asJava) - ).asJava) + .setMembers(util.List.of(member2)) + )) val expectedConsumerGroupDescribeResponseData = new ConsumerGroupDescribeResponseData() - .setGroups(List( + .setGroups(util.List.of( new DescribedGroup() .setGroupId(groupIds.get(0)) - .setMembers(List(member0).asJava), + .setMembers(util.List.of(member0)), new DescribedGroup() .setGroupId(groupIds.get(1)) .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) @@ -10643,7 +10630,7 @@ class KafkaApisTest extends Logging { .setGroupId(groupIds.get(2)) .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) .setErrorMessage(errorMessage) - ).asJava) + )) val response = verifyNoThrottling[ConsumerGroupDescribeResponse](requestChannelRequest) @@ -10745,8 +10732,8 @@ class KafkaApisTest extends Logging { val request = buildRequest(new ListClientMetricsResourcesRequest.Builder(new ListClientMetricsResourcesRequestData()).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) - val resources = new mutable.HashSet[String] - when(clientMetricsManager.listClientMetricsResources).thenReturn(resources.asJava) + val resources = new util.HashSet[String] + when(clientMetricsManager.listClientMetricsResources).thenReturn(resources) kafkaApis = createKafkaApis() kafkaApis.handle(request, RequestLocal.noCaching) val response = verifyNoThrottling[ListClientMetricsResourcesResponse](request) @@ -10816,7 +10803,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -10838,7 +10825,7 @@ class KafkaApisTest extends Logging { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData() .setGroupId(groupId) - .setSubscribedTopicNames(List(fooTopicName, barTopicName, zarTopicName).asJava) + .setSubscribedTopicNames(util.List.of(fooTopicName, barTopicName, zarTopicName)) val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) @@ -10853,9 +10840,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } kafkaApis = createKafkaApis( @@ -10895,44 +10882,44 @@ class KafkaApisTest extends Logging { val fooTopicName = "foo" val barTopicName = "bar" - val groupIds = List("share-group-id-0", "share-group-id-1", "share-group_id-2").asJava + val groupIds = util.List.of("share-group-id-0", "share-group-id-1", "share-group_id-2") val member0 = new ShareGroupDescribeResponseData.Member() .setMemberId("member0") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName)))) val member1 = new ShareGroupDescribeResponseData.Member() .setMemberId("member1") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( + .setTopicPartitions(util.List.of( new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName), - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)).asJava)) + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)))) val member2 = new ShareGroupDescribeResponseData.Member() .setMemberId("member2") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)).asJava)) - - val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = List( - new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(0)).setMembers(List(member0).asJava), - new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)).setMembers(List(member1).asJava), - new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(2)).setMembers(List(member2).asJava) - ).asJava + .setTopicPartitions(util.List.of( + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)))) + + val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = util.List.of( + new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(0)).setMembers(util.List.of(member0)), + new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)).setMembers(util.List.of(member1)), + new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(2)).setMembers(util.List.of(member2)) + ) getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , true, null, describedGroups) + , verifyNoErr = true, null, describedGroups) } @Test def testShareGroupDescribeReturnsUnsupportedVersion(): Unit = { - val groupIds = List("share-group-id-0", "share-group-id-1").asJava - val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = List( + val groupIds = util.List.of("share-group-id-0", "share-group-id-1") + val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = util.List.of( new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(0)), new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)) - ).asJava - val response = getShareGroupDescribeResponse(groupIds, Map.empty, false, null, describedGroups) + ) + val response = getShareGroupDescribeResponse(groupIds, Map.empty, verifyNoErr = false, null, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.UNSUPPORTED_VERSION.code(), group.errorCode())) @@ -10940,13 +10927,13 @@ class KafkaApisTest extends Logging { @Test def testShareGroupDescribeRequestAuthorizationFailed(): Unit = { - val groupIds = List("share-group-id-0", "share-group-id-1").asJava - val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = List().asJava + val groupIds = util.List.of("share-group-id-0", "share-group-id-1") + val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = util.List.of val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED)) val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + , verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.GROUP_AUTHORIZATION_FAILED.code(), group.errorCode())) @@ -10954,17 +10941,17 @@ class KafkaApisTest extends Logging { @Test def testShareGroupDescribeRequestAuthorizationFailedForOneGroup(): Unit = { - val groupIds = List("share-group-id-fail-0", "share-group-id-1").asJava - val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = List( + val groupIds = util.List.of("share-group-id-fail-0", "share-group-id-1") + val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = util.List.of( new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)) - ).asJava + ) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + , verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) @@ -10978,7 +10965,7 @@ class KafkaApisTest extends Logging { val barTopicName = "bar" val errorMessage = "The group has described topic(s) that the client is not authorized to describe." - val groupIds = List("share-group-id-0", "share-group-id-1", "share-group_id-2").asJava + val groupIds = util.List.of("share-group-id-0", "share-group-id-1", "share-group_id-2") val authorizer: Authorizer = mock(classOf[Authorizer]) val acls = Map( @@ -10993,42 +10980,42 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } val member0 = new ShareGroupDescribeResponseData.Member() .setMemberId("member0") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName)))) val member1 = new ShareGroupDescribeResponseData.Member() .setMemberId("member1") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( + .setTopicPartitions(util.List.of( new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(fooTopicName), - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)).asJava)) + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)))) val member2 = new ShareGroupDescribeResponseData.Member() .setMemberId("member2") .setAssignment(new ShareGroupDescribeResponseData.Assignment() - .setTopicPartitions(List( - new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)).asJava)) + .setTopicPartitions(util.List.of( + new ShareGroupDescribeResponseData.TopicPartitions().setTopicName(barTopicName)))) - val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = List( + val describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup] = util.List.of( new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(0)) - .setMembers(List(member0).asJava), + .setMembers(util.List.of(member0)), new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(1)) - .setMembers(List(member1).asJava), + .setMembers(util.List.of(member1)), new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId(groupIds.get(2)) - .setMembers(List(member2).asJava)).asJava + .setMembers(util.List.of(member2))) val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + , verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(3, response.data.groups.size) @@ -11041,38 +11028,38 @@ class KafkaApisTest extends Logging { @Test def testReadShareGroupStateSuccess(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val readRequestData = new ReadShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new ReadShareGroupStateRequestData.ReadStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) - val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = List( + val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = util.List.of( new ReadShareGroupStateResponseData.ReadStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) .setStateEpoch(1) .setStartOffset(10) - .setStateBatches(List( + .setStateBatches(util.List.of( new ReadShareGroupStateResponseData.StateBatch() .setFirstOffset(11) .setLastOffset(15) .setDeliveryState(0) .setDeliveryCount(1) - ).asJava) - ).asJava) - ).asJava + )) + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -11092,42 +11079,42 @@ class KafkaApisTest extends Logging { @Test def testReadShareGroupStateAuthorizationFailed(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val readRequestData = new ReadShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new ReadShareGroupStateRequestData.ReadStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) - val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = List( + val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = util.List.of( new ReadShareGroupStateResponseData.ReadStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) .setStateEpoch(1) .setStartOffset(10) - .setStateBatches(List( + .setStateBatches(util.List.of( new ReadShareGroupStateResponseData.StateBatch() .setFirstOffset(11) .setLastOffset(15) .setDeliveryState(0) .setDeliveryCount(1) - ).asJava) - ).asJava) - ).asJava + )) + )) + ) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -11151,31 +11138,31 @@ class KafkaApisTest extends Logging { @Test def testReadShareGroupStateSummarySuccess(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val readSummaryRequestData = new ReadShareGroupStateSummaryRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateSummaryRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) - val readStateSummaryResultData: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult] = List( + val readStateSummaryResultData: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult] = util.List.of( new ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateSummaryResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) .setStateEpoch(1) .setStartOffset(10) - ).asJava) - ).asJava + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -11195,35 +11182,35 @@ class KafkaApisTest extends Logging { @Test def testReadShareGroupStateSummaryAuthorizationFailed(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val readSummaryRequestData = new ReadShareGroupStateSummaryRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateSummaryRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) - ).asJava) - ).asJava) + )) + )) - val readStateSummaryResultData: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult] = List( + val readStateSummaryResultData: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult] = util.List.of( new ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new ReadShareGroupStateSummaryResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) .setStateEpoch(1) .setStartOffset(10) - ).asJava) - ).asJava + )) + ) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -11500,9 +11487,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11662,9 +11649,9 @@ class KafkaApisTest extends Logging { any[util.List[Action]] )).thenAnswer { invocation => val actions = invocation.getArgument(1, classOf[util.List[Action]]) - actions.asScala.map { action => - acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED) - }.asJava + actions.stream() + .map(action => acls.getOrElse(action.resourcePattern.name, AuthorizationResult.DENIED)) + .toList } kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11980,13 +11967,13 @@ class KafkaApisTest extends Logging { def buildExpectedActionsTopic(topic: String): util.List[Action] = { val pattern = new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL) val action = new Action(AclOperation.READ, pattern, 1, true, true) - Collections.singletonList(action) + util.List.of(action) } def buildExpectedActionsGroup(topic: String): util.List[Action] = { val pattern = new ResourcePattern(ResourceType.GROUP, topic, PatternType.LITERAL) val action = new Action(AclOperation.DELETE, pattern, 1, true, true) - Collections.singletonList(action) + util.List.of(action) } val groupId = "group" @@ -12291,38 +12278,38 @@ class KafkaApisTest extends Logging { @Test def testWriteShareGroupStateSuccess(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val writeRequestData = new WriteShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new WriteShareGroupStateRequestData.WriteStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new WriteShareGroupStateRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) .setStateEpoch(2) .setStartOffset(10) - .setStateBatches(List( + .setStateBatches(util.List.of( new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(11) .setLastOffset(15) .setDeliveryCount(1) .setDeliveryState(0) - ).asJava) - ).asJava) - ).asJava) + )) + )) + )) - val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = List( + val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = util.List.of( new WriteShareGroupStateResponseData.WriteStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new WriteShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -12342,38 +12329,38 @@ class KafkaApisTest extends Logging { @Test def testWriteShareGroupStateAuthorizationFailed(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val writeRequestData = new WriteShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new WriteShareGroupStateRequestData.WriteStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new WriteShareGroupStateRequestData.PartitionData() .setPartition(1) .setLeaderEpoch(1) .setStateEpoch(2) .setStartOffset(10) - .setStateBatches(List( + .setStateBatches(util.List.of( new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(11) .setLastOffset(15) .setDeliveryCount(1) .setDeliveryState(0) - ).asJava) - ).asJava) - ).asJava) + )) + )) + )) - val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = List( + val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = util.List.of( new WriteShareGroupStateResponseData.WriteStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new WriteShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -12381,7 +12368,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val response = getWriteShareGroupResponse( writeRequestData, @@ -12401,28 +12388,28 @@ class KafkaApisTest extends Logging { @Test def testDeleteShareGroupStateSuccess(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val deleteRequestData = new DeleteShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new DeleteShareGroupStateRequestData.DeleteStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new DeleteShareGroupStateRequestData.PartitionData() .setPartition(1) - ).asJava) - ).asJava) + )) + )) - val deleteStateResultData: util.List[DeleteShareGroupStateResponseData.DeleteStateResult] = List( + val deleteStateResultData: util.List[DeleteShareGroupStateResponseData.DeleteStateResult] = util.List.of( new DeleteShareGroupStateResponseData.DeleteStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new DeleteShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -12442,32 +12429,32 @@ class KafkaApisTest extends Logging { @Test def testDeleteShareGroupStateAuthorizationFailed(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val deleteRequestData = new DeleteShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new DeleteShareGroupStateRequestData.DeleteStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new DeleteShareGroupStateRequestData.PartitionData() .setPartition(1) - ).asJava) - ).asJava) + )) + )) - val deleteStateResultData: util.List[DeleteShareGroupStateResponseData.DeleteStateResult] = List( + val deleteStateResultData: util.List[DeleteShareGroupStateResponseData.DeleteStateResult] = util.List.of( new DeleteShareGroupStateResponseData.DeleteStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new DeleteShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -12491,29 +12478,29 @@ class KafkaApisTest extends Logging { @Test def testInitializeShareGroupStateSuccess(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val initRequestData = new InitializeShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new InitializeShareGroupStateRequestData.InitializeStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new InitializeShareGroupStateRequestData.PartitionData() .setPartition(1) .setStateEpoch(0) - ).asJava) - ).asJava) + )) + )) - val initStateResultData: util.List[InitializeShareGroupStateResponseData.InitializeStateResult] = List( + val initStateResultData: util.List[InitializeShareGroupStateResponseData.InitializeStateResult] = util.List.of( new InitializeShareGroupStateResponseData.InitializeStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new InitializeShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", @@ -12533,33 +12520,33 @@ class KafkaApisTest extends Logging { @Test def testInitializeShareGroupStateAuthorizationFailed(): Unit = { - val topicId = Uuid.randomUuid(); + val topicId = Uuid.randomUuid() val initRequestData = new InitializeShareGroupStateRequestData() .setGroupId("group1") - .setTopics(List( + .setTopics(util.List.of( new InitializeShareGroupStateRequestData.InitializeStateData() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new InitializeShareGroupStateRequestData.PartitionData() .setPartition(1) .setStateEpoch(0) - ).asJava) - ).asJava) + )) + )) - val initStateResultData: util.List[InitializeShareGroupStateResponseData.InitializeStateResult] = List( + val initStateResultData: util.List[InitializeShareGroupStateResponseData.InitializeStateResult] = util.List.of( new InitializeShareGroupStateResponseData.InitializeStateResult() .setTopicId(topicId) - .setPartitions(List( + .setPartitions(util.List.of( new InitializeShareGroupStateResponseData.PartitionResult() .setPartition(1) .setErrorCode(Errors.NONE.code()) .setErrorMessage(null) - ).asJava) - ).asJava + )) + ) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) - .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + .thenReturn(util.List.of(AuthorizationResult.DENIED), util.List.of(AuthorizationResult.ALLOWED)) val config = Map( ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true",