From 516837162e3a95ac40ac8c03fda996ab6ff984b5 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 16 Apr 2025 08:34:37 +0200 Subject: [PATCH 1/5] MINOR: Update GroupCoordinator interface to use AuthorizableRequestContext instead of RequestContext --- .../common/requests/JoinGroupRequest.java | 6 +- .../common/runtime/CoordinatorRuntime.java | 2 +- .../common/runtime/PartitionWriter.java | 2 +- .../runtime/InMemoryPartitionWriter.java | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 125 +---- .../unit/kafka/server/KafkaApisTest.scala | 479 ++++-------------- .../apache/kafka/coordinator/group/Group.java | 2 +- .../coordinator/group/GroupCoordinator.java | 44 +- .../group/GroupCoordinatorService.java | 90 ++-- .../group/GroupCoordinatorShard.java | 26 +- .../group/GroupMetadataManager.java | 72 +-- .../group/OffsetMetadataManager.java | 18 +- .../group/classic/ClassicGroup.java | 2 +- .../group/modern/consumer/ConsumerGroup.java | 2 +- .../group/modern/share/ShareGroup.java | 2 +- .../group/streams/StreamsGroup.java | 2 +- .../group/GroupCoordinatorServiceTest.java | 6 +- 17 files changed, 277 insertions(+), 605 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index ed3625068aa4e..9d75d383bab13 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -96,7 +96,7 @@ public static String maybeTruncateReason(final String reason) { * * @return whether a known member id is required or not. */ - public static boolean requiresKnownMemberId(short apiVersion) { + public static boolean requiresKnownMemberId(int apiVersion) { return apiVersion >= 4; } @@ -117,7 +117,7 @@ public static boolean requiresKnownMemberId(short apiVersion) { */ public static boolean requiresKnownMemberId( JoinGroupRequestData request, - short apiVersion + int apiVersion ) { return request.groupInstanceId() == null && request.memberId().equals(UNKNOWN_MEMBER_ID) @@ -150,7 +150,7 @@ public static boolean requiresKnownMemberId( * @return whether the version supports skipping assignment. */ - public static boolean supportsSkippingAssignment(short apiVersion) { + public static boolean supportsSkippingAssignment(int apiVersion) { return apiVersion >= 9; } diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index 88b0eac353b75..19567344f2d36 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -2236,7 +2236,7 @@ public CompletableFuture scheduleTransactionalWriteOperation( short producerEpoch, Duration timeout, CoordinatorWriteOperation op, - Short apiVersion + int apiVersion ) { throwIfNotRunning(); log.debug("Scheduled execution of transactional write operation {}.", name); diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/PartitionWriter.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/PartitionWriter.java index cb8bec3f71c94..cc76cfd64605b 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/PartitionWriter.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/PartitionWriter.java @@ -105,7 +105,7 @@ CompletableFuture maybeStartTransactionVerification( String transactionalId, long producerId, short producerEpoch, - short apiVersion + int apiVersion ) throws KafkaException; /** diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/InMemoryPartitionWriter.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/InMemoryPartitionWriter.java index 66cfbbe8b101d..7809c46cb1001 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/InMemoryPartitionWriter.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/InMemoryPartitionWriter.java @@ -129,7 +129,7 @@ public CompletableFuture maybeStartTransactionVerification( String transactionalId, long producerId, short producerEpoch, - short apiVersion + int apiVersion ) throws KafkaException { return CompletableFuture.completedFuture(new VerificationGuard()); } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6a22963ac7d6a..c754c67568939 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -349,11 +349,7 @@ class KafkaApis(val requestChannel: RequestChannel, .setGroupInstanceId(offsetCommitRequest.data.groupInstanceId) .setTopics(authorizedTopicsRequest.asJava) - groupCoordinator.commitOffsets( - request.context, - offsetCommitRequestData, - requestLocal.bufferSupplier - ).handle[Unit] { (results, exception) => + groupCoordinator.commitOffsets(request.context, offsetCommitRequestData, requestLocal.bufferSupplier).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(exception)) } else { @@ -1027,11 +1023,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup, requireStable: Boolean ): CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup] = { - groupCoordinator.fetchAllOffsets( - requestContext, - offsetFetchRequest, - requireStable - ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => + groupCoordinator.fetchAllOffsets(requestContext, offsetFetchRequest, requireStable).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => if (exception != null) { new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId(offsetFetchRequest.groupId) @@ -1064,15 +1056,11 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest.topics.asScala )(_.name) - groupCoordinator.fetchOffsets( - requestContext, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId(offsetFetchRequest.groupId) - .setMemberId(offsetFetchRequest.memberId) - .setMemberEpoch(offsetFetchRequest.memberEpoch) - .setTopics(authorizedTopics.asJava), - requireStable - ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => + groupCoordinator.fetchOffsets(requestContext, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(offsetFetchRequest.groupId) + .setMemberId(offsetFetchRequest.memberId) + .setMemberEpoch(offsetFetchRequest.memberEpoch) + .setTopics(authorizedTopics.asJava), requireStable).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => if (exception != null) { new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId(offsetFetchRequest.groupId) @@ -1234,10 +1222,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.describeGroups( - request.context, - authorizedGroups.asJava - ).handle[Unit] { (results, exception) => + groupCoordinator.describeGroups(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, describeRequest.getErrorResponse(exception)) } else { @@ -1269,10 +1254,7 @@ class KafkaApis(val requestChannel: RequestChannel, val listGroupsRequest = request.body[ListGroupsRequest] val hasClusterDescribe = authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME, logIfDenied = false) - groupCoordinator.listGroups( - request.context, - listGroupsRequest.data - ).handle[Unit] { (response, exception) => + groupCoordinator.listGroups(request.context, listGroupsRequest.data).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, listGroupsRequest.getErrorResponse(exception)) } else { @@ -1301,11 +1283,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.joinGroup( - request.context, - joinGroupRequest.data, - requestLocal.bufferSupplier - ).handle[Unit] { (response, exception) => + groupCoordinator.joinGroup(request.context, joinGroupRequest.data, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(exception)) } else { @@ -1329,11 +1307,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.syncGroup( - request.context, - syncGroupRequest.data, - requestLocal.bufferSupplier - ).handle[Unit] { (response, exception) => + groupCoordinator.syncGroup(request.context, syncGroupRequest.data, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(exception)) } else { @@ -1353,11 +1327,7 @@ class KafkaApis(val requestChannel: RequestChannel, val (authorizedGroups, unauthorizedGroups) = authHelper.partitionSeqByAuthorized(request.context, DELETE, GROUP, groups)(identity) - groupCoordinator.deleteGroups( - request.context, - authorizedGroups.toList.asJava, - requestLocal.bufferSupplier - ).handle[Unit] { (results, exception) => + groupCoordinator.deleteGroups(request.context, authorizedGroups.toList.asJava, requestLocal.bufferSupplier).handle[Unit] { (results, exception) => val response = new DeleteGroupsResponseData() if (exception != null) { @@ -1388,10 +1358,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.heartbeat( - request.context, - heartbeatRequest.data - ).handle[Unit] { (response, exception) => + groupCoordinator.heartbeat(request.context, heartbeatRequest.data).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(exception)) } else { @@ -1408,10 +1375,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, leaveGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.leaveGroup( - request.context, - leaveGroupRequest.normalizedData() - ).handle[Unit] { (response, exception) => + groupCoordinator.leaveGroup(request.context, leaveGroupRequest.normalizedData()).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, leaveGroupRequest.getErrorResponse(exception)) } else { @@ -2013,11 +1977,7 @@ class KafkaApis(val requestChannel: RequestChannel, .setTransactionalId(txnOffsetCommitRequest.data.transactionalId) .setTopics(authorizedTopicCommittedOffsets.asJava) - groupCoordinator.commitTransactionalOffsets( - request.context, - txnOffsetCommitRequestData, - requestLocal.bufferSupplier - ).handle[Unit] { (response, exception) => + groupCoordinator.commitTransactionalOffsets(request.context, txnOffsetCommitRequestData, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => if (exception != null) { sendResponse(txnOffsetCommitRequest.getErrorResponse(exception)) } else { @@ -2324,11 +2284,7 @@ class KafkaApis(val requestChannel: RequestChannel, .setGroupId(offsetDeleteRequest.data.groupId) .setTopics(authorizedTopicPartitions) - groupCoordinator.deleteOffsets( - request.context, - offsetDeleteRequestData, - requestLocal.bufferSupplier - ).handle[Unit] { (response, exception) => + groupCoordinator.deleteOffsets(request.context, offsetDeleteRequestData, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, offsetDeleteRequest.getErrorResponse(exception)) } else { @@ -2534,10 +2490,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.consumerGroupHeartbeat( - request.context, - consumerGroupHeartbeatRequest.data - ).handle[Unit] { (response, exception) => + groupCoordinator.consumerGroupHeartbeat(request.context, consumerGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, consumerGroupHeartbeatRequest.getErrorResponse(exception)) } else { @@ -2571,10 +2524,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.consumerGroupDescribe( - request.context, - authorizedGroups.asJava - ).handle[Unit] { (results, exception) => + groupCoordinator.consumerGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, consumerGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -2693,10 +2643,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.streamsGroupHeartbeat( - request.context, - streamsGroupHeartbeatRequest.data - ).handle[Unit] { (response, exception) => + groupCoordinator.streamsGroupHeartbeat(request.context, streamsGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, streamsGroupHeartbeatRequest.getErrorResponse(exception)) } else { @@ -2752,10 +2699,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.streamsGroupDescribe( - request.context, - authorizedGroups.asJava - ).handle[Unit] { (results, exception) => + groupCoordinator.streamsGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, streamsGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -2887,10 +2831,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.shareGroupHeartbeat( - request.context, - shareGroupHeartbeatRequest.data - ).handle[Unit] { (response, exception) => + groupCoordinator.shareGroupHeartbeat(request.context, shareGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, shareGroupHeartbeatRequest.getErrorResponse(exception)) @@ -2923,10 +2864,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.shareGroupDescribe( - request.context, - authorizedGroups.asJava - ).handle[Unit] { (results, exception) => + groupCoordinator.shareGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, shareGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -3552,10 +3490,7 @@ class KafkaApis(val requestChannel: RequestChannel, private def describeShareGroupAllOffsetsForGroup(requestContext: RequestContext, groupDescribeOffsetsRequest: DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup ): CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] = { - groupCoordinator.describeShareGroupAllOffsets( - requestContext, - groupDescribeOffsetsRequest - ).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => + groupCoordinator.describeShareGroupAllOffsets(requestContext, groupDescribeOffsetsRequest).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => if (exception != null) { val error = Errors.forException(exception) new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup() @@ -3586,12 +3521,9 @@ class KafkaApis(val requestChannel: RequestChannel, groupDescribeOffsetsRequest.topics.asScala )(_.topicName) - groupCoordinator.describeShareGroupOffsets( - requestContext, - new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() - .setGroupId(groupDescribeOffsetsRequest.groupId) - .setTopics(authorizedTopics.asJava) - ).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => + groupCoordinator.describeShareGroupOffsets(requestContext, new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() + .setGroupId(groupDescribeOffsetsRequest.groupId) + .setTopics(authorizedTopics.asJava)).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => if (exception != null) { val error = Errors.forException(exception) new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup() @@ -3669,10 +3601,7 @@ class KafkaApis(val requestChannel: RequestChannel, return } - groupCoordinator.deleteShareGroupOffsets( - request.context, - new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics) - ).handle[Unit] {(responseData, exception) => { + groupCoordinator.deleteShareGroupOffsets(request.context, new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics)).handle[Unit] { (responseData, exception) => { if (exception != null) { requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception)) } else if (responseData.errorCode() != Errors.NONE.code) { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index c644043168438..b3d7254067a83 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -1010,11 +1010,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets( - requestChannelRequest.context, - offsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitOffsets(requestChannelRequest.context, offsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1054,11 +1050,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets( - requestChannelRequest.context, - offsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitOffsets(requestChannelRequest.context, offsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( @@ -1151,11 +1143,7 @@ class KafkaApisTest extends Logging { .setCommittedOffset(50)).asJava)).asJava) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets( - requestChannelRequest.context, - expectedOffsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitOffsets(requestChannelRequest.context, expectedOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1328,11 +1316,7 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(txnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets( - requestChannelRequest.context, - txnOffsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, txnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1373,11 +1357,7 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(txnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets( - requestChannelRequest.context, - txnOffsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, txnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1470,11 +1450,7 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(expectedTxnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets( - requestChannelRequest.context, - expectedTxnOffsetCommitRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, expectedTxnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1573,11 +1549,7 @@ class KafkaApisTest extends Logging { val requestLocal = RequestLocal.withThreadConfinedCaching val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(offsetCommitRequest.data.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets( - request.context, - offsetCommitRequest.data, - requestLocal.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets(request.context, offsetCommitRequest.data, requestLocal.bufferSupplier)).thenReturn(future) future.complete(new TxnOffsetCommitResponseData() .setTopics(List( @@ -2918,11 +2890,7 @@ class KafkaApisTest extends Logging { 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, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteGroups(requestChannelRequest.context, List("group-1", "group-2", "group-3").asJava, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -2961,11 +2929,7 @@ class KafkaApisTest extends Logging { 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, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteGroups(requestChannelRequest.context, List("group-1", "group-2", "group-3").asJava, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -3020,11 +2984,7 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[DeleteGroupsResponseData.DeletableGroupResultCollection]() - when(groupCoordinator.deleteGroups( - requestChannelRequest.context, - List("group-2", "group-3").asJava, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteGroups(requestChannelRequest.context, List("group-2", "group-3").asJava, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -3068,10 +3028,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups( - requestChannelRequest.context, - describeGroupsRequest.groups - )).thenReturn(future) + when(groupCoordinator.describeGroups(requestChannelRequest.context, describeGroupsRequest.groups)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3115,10 +3072,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups( - requestChannelRequest.context, - describeGroupsRequest.groups - )).thenReturn(future) + when(groupCoordinator.describeGroups(requestChannelRequest.context, describeGroupsRequest.groups)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3169,10 +3123,7 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups( - requestChannelRequest.context, - List("group-2").asJava - )).thenReturn(future) + when(groupCoordinator.describeGroups(requestChannelRequest.context, List("group-2").asJava)).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3226,11 +3177,7 @@ class KafkaApisTest extends Logging { val requestLocal = RequestLocal.withThreadConfinedCaching val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets( - request.context, - offsetDeleteRequest.data, - requestLocal.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteOffsets(request.context, offsetDeleteRequest.data, requestLocal.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, requestLocal) @@ -3319,11 +3266,7 @@ class KafkaApisTest extends Logging { ).asJava.iterator)) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets( - requestChannelRequest.context, - expectedOffsetDeleteRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteOffsets(requestChannelRequest.context, expectedOffsetDeleteRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -3424,11 +3367,7 @@ class KafkaApisTest extends Logging { // The group coordinator is called even if there are no // topic-partitions left after the validation. - when(groupCoordinator.deleteOffsets( - request.context, - new OffsetDeleteRequestData().setGroupId(group), - RequestLocal.noCaching.bufferSupplier - )).thenReturn(CompletableFuture.completedFuture( + when(groupCoordinator.deleteOffsets(request.context, new OffsetDeleteRequestData().setGroupId(group), RequestLocal.noCaching.bufferSupplier)).thenReturn(CompletableFuture.completedFuture( new OffsetDeleteResponseData() )) val kafkaApis = createKafkaApis() @@ -3460,11 +3399,7 @@ class KafkaApisTest extends Logging { val request = buildRequest(offsetDeleteRequest) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets( - request.context, - offsetDeleteRequest.data, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteOffsets(request.context, offsetDeleteRequest.data, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, RequestLocal.noCaching) @@ -3494,11 +3429,7 @@ class KafkaApisTest extends Logging { val request = buildRequest(offsetDeleteRequest) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets( - request.context, - new OffsetDeleteRequestData().setGroupId(group), // Nonexistent topics won't be passed to groupCoordinator. - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.deleteOffsets(request.context, new OffsetDeleteRequestData().setGroupId(group), RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, RequestLocal.noCaching) @@ -7584,11 +7515,7 @@ class KafkaApisTest extends Logging { .setSessionTimeoutMs(joinGroupRequest.sessionTimeoutMs) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup( - requestChannelRequest.context, - expectedJoinGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.joinGroup(requestChannelRequest.context, expectedJoinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7627,11 +7554,7 @@ class KafkaApisTest extends Logging { .setSessionTimeoutMs(joinGroupRequest.sessionTimeoutMs) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup( - requestChannelRequest.context, - expectedJoinGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.joinGroup(requestChannelRequest.context, expectedJoinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7665,11 +7588,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new JoinGroupRequest.Builder(joinGroupRequest).build()) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup( - requestChannelRequest.context, - joinGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.joinGroup(requestChannelRequest.context, joinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7717,11 +7636,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new JoinGroupRequest.Builder(joinGroupRequest).build()) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup( - requestChannelRequest.context, - joinGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.joinGroup(requestChannelRequest.context, joinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) var response: JoinGroupResponse = null when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ => @@ -7760,11 +7675,7 @@ class KafkaApisTest extends Logging { .setProtocolName(if (version >= 5) "range" else null) val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup( - requestChannelRequest.context, - expectedSyncGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7797,11 +7708,7 @@ class KafkaApisTest extends Logging { .setProtocolName("range") val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup( - requestChannelRequest.context, - expectedSyncGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7850,11 +7757,7 @@ class KafkaApisTest extends Logging { .setMemberId("member") val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup( - requestChannelRequest.context, - expectedSyncGroupRequest, - RequestLocal.noCaching.bufferSupplier - )).thenReturn(future) + when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7892,10 +7795,7 @@ class KafkaApisTest extends Logging { .setGenerationId(0) val future = new CompletableFuture[HeartbeatResponseData]() - when(groupCoordinator.heartbeat( - requestChannelRequest.context, - expectedHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.heartbeat(requestChannelRequest.context, expectedHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleHeartbeatRequest(requestChannelRequest) @@ -7920,10 +7820,7 @@ class KafkaApisTest extends Logging { .setGenerationId(0) val future = new CompletableFuture[HeartbeatResponseData]() - when(groupCoordinator.heartbeat( - requestChannelRequest.context, - expectedHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.heartbeat(requestChannelRequest.context, expectedHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleHeartbeatRequest(requestChannelRequest) @@ -7988,10 +7885,7 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup( - requestChannelRequest.context, - expectedLeaveGroupRequest - )).thenReturn(future) + when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -8033,10 +7927,7 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup( - requestChannelRequest.context, - expectedLeaveGroupRequest - )).thenReturn(future) + when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -8086,10 +7977,7 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup( - requestChannelRequest.context, - expectedLeaveGroupRequest - )).thenReturn(future) + when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -8118,10 +8006,7 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup( - requestChannelRequest.context, - expectedLeaveGroupRequest - )).thenReturn(future) + when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -8156,43 +8041,27 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List( - new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), - false - )).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("foo") + .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), false)).thenReturn(group1Future) val group2Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-2") - .setTopics(null), - false - )).thenReturn(group2Future) + when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-2") + .setTopics(null), false)).thenReturn(group2Future) val group3Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-3") - .setTopics(null), - false - )).thenReturn(group3Future) + when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-3") + .setTopics(null), false)).thenReturn(group3Future) val group4Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-4") - .setTopics(null), - false - )).thenReturn(group4Future) + when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-4") + .setTopics(null), false)).thenReturn(group4Future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8272,15 +8141,11 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), - false - )).thenReturn(future) + when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("foo") + .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), false)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8347,13 +8212,9 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(null), - false - )).thenReturn(future) + when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(null), false)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8444,25 +8305,17 @@ class KafkaApisTest extends Logging { // group-1 is allowed and bar is allowed. val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), - false - )).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) // group-3 is allowed and bar is allowed. val group3Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-3") - .setTopics(null), - false - )).thenReturn(group3Future) + when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-3") + .setTopics(null), false)).thenReturn(group3Future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -8593,26 +8446,18 @@ class KafkaApisTest extends Logging { // group-1 and group-2 are allowed and bar is allowed. val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), - false - )).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) val group2Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets( - requestChannelRequest.context, - new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-2") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), - false - )).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-2") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -8713,10 +8558,7 @@ class KafkaApisTest extends Logging { .setTypesFilter(if (version >= 5) List("classic", "consumer").asJava else List.empty.asJava) val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups( - requestChannelRequest.context, - expectedListGroupsRequest - )).thenReturn(future) + when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -8757,10 +8599,7 @@ class KafkaApisTest extends Logging { .setTypesFilter(List("classic", "consumer").asJava) val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups( - requestChannelRequest.context, - expectedListGroupsRequest - )).thenReturn(future) + when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -8853,10 +8692,7 @@ class KafkaApisTest extends Logging { val expectedListGroupsRequest = new ListGroupsRequestData() val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups( - requestChannelRequest.context, - expectedListGroupsRequest - )).thenReturn(future) + when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -9670,10 +9506,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) val future = new CompletableFuture[ConsumerGroupHeartbeatResponseData]() - when(groupCoordinator.consumerGroupHeartbeat( - requestChannelRequest.context, - consumerGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.consumerGroupHeartbeat(requestChannelRequest.context, consumerGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -9696,10 +9529,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) val future = new CompletableFuture[ConsumerGroupHeartbeatResponseData]() - when(groupCoordinator.consumerGroupHeartbeat( - requestChannelRequest.context, - consumerGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.consumerGroupHeartbeat(requestChannelRequest.context, consumerGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -9804,10 +9634,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat( - requestChannelRequest.context, - streamsGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -9869,10 +9696,7 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat( - requestChannelRequest.context, - streamsGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -9896,10 +9720,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat( - requestChannelRequest.context, - streamsGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -10066,10 +9887,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat( - requestChannelRequest.context, - streamsGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -10095,10 +9913,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat( - requestChannelRequest.context, - streamsGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], isNotNull[util.List[Action]])).thenAnswer(invocation => { @@ -10151,10 +9966,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -10268,10 +10080,7 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), @@ -10292,10 +10101,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -10320,10 +10126,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupDescribeRequest.Builder(streamsGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -10428,10 +10231,7 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), @@ -10452,10 +10252,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupDescribeRequest.Builder(streamsGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -10500,10 +10297,7 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -10580,10 +10374,7 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), featureVersions = Seq(GroupVersion.GV_1) @@ -10790,10 +10581,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() - when(groupCoordinator.shareGroupHeartbeat( - requestChannelRequest.context, - shareGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.shareGroupHeartbeat(requestChannelRequest.context, shareGroupHeartbeatRequest)).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -10875,10 +10663,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() - when(groupCoordinator.shareGroupHeartbeat( - requestChannelRequest.context, - shareGroupHeartbeatRequest - )).thenReturn(future) + when(groupCoordinator.shareGroupHeartbeat(requestChannelRequest.context, shareGroupHeartbeatRequest)).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11352,15 +11137,9 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets( - requestChannelRequest.context, - describeShareGroupOffsetsRequestGroup1 - )).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup1)).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets( - requestChannelRequest.context, - describeShareGroupOffsetsRequestGroup2 - )).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2)).thenReturn(futureGroup2) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -11469,23 +11248,17 @@ class KafkaApisTest extends Logging { // The group coordinator will only be asked for information about topics which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets( - requestChannelRequest.context, - new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics( - util.List.of( - new DescribeShareGroupOffsetsRequestTopic().setTopicName(topicName1).setPartitions(util.List.of(1, 2, 3)), - ) + when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics( + util.List.of( + new DescribeShareGroupOffsetsRequestTopic().setTopicName(topicName1).setPartitions(util.List.of(1, 2, 3)), ) - )).thenReturn(futureGroup1) + ))).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets( - requestChannelRequest.context, - new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics( - util.List.of( - ) + when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics( + util.List.of( ) - )).thenReturn(futureGroup2) + ))).thenReturn(futureGroup2) val authorizer: Authorizer = mock(classOf[Authorizer]) val acls = Map( @@ -11638,16 +11411,10 @@ class KafkaApisTest extends Logging { // The group coordinator is being asked for information about all topics, not just those which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets( - requestChannelRequest.context, - new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics(null) - )).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics(null))).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets( - requestChannelRequest.context, - new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics(null) - )).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics(null))).thenReturn(futureGroup2) val authorizer: Authorizer = mock(classOf[Authorizer]) val acls = Map( @@ -11799,15 +11566,9 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets( - requestChannelRequest.context, - describeShareGroupOffsetsRequestGroup1 - )).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup1)).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets( - requestChannelRequest.context, - describeShareGroupOffsetsRequestGroup2 - )).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2)).thenReturn(futureGroup2) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -11917,10 +11678,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets( - requestChannelRequest.context, - describeShareGroupOffsetsRequestGroup - )).thenReturn(future) + when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup)).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -12018,10 +11776,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - when(groupCoordinator.deleteShareGroupOffsets( - requestChannelRequest.context, - deleteShareGroupOffsetsGroupCoordinatorRequestData - )).thenReturn(resultFuture) + when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsGroupCoordinatorRequestData)).thenReturn(resultFuture) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActionsGroup(groupId)))) @@ -12119,10 +11874,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - when(groupCoordinator.deleteShareGroupOffsets( - requestChannelRequest.context, - deleteShareGroupOffsetsRequestData - )).thenReturn(resultFuture) + when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(resultFuture) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -12203,10 +11955,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) - when(groupCoordinator.deleteShareGroupOffsets( - requestChannelRequest.context, - deleteShareGroupOffsetsRequestData - )).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) + when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -12249,10 +11998,7 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) .setErrorMessage(Errors.UNKNOWN_SERVER_ERROR.message()) - when(groupCoordinator.deleteShareGroupOffsets( - requestChannelRequest.context, - deleteShareGroupOffsetsRequestData - )).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) + when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -12589,10 +12335,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ShareGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.shareGroupDescribe( - any[RequestContext], - any[util.List[String]] - )).thenReturn(future) + when(groupCoordinator.shareGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java index 410064008db29..54d7e98d4b7be 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java @@ -117,7 +117,7 @@ void validateOffsetCommit( String groupInstanceId, int generationIdOrMemberEpoch, boolean isTransactional, - short apiVersion + int apiVersion ) throws KafkaException; diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java index 98d33e9f254cb..0bc37c82b1450 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java @@ -49,12 +49,12 @@ import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.message.TxnOffsetCommitRequestData; import org.apache.kafka.common.message.TxnOffsetCommitResponseData; -import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import java.time.Duration; import java.util.List; @@ -80,7 +80,7 @@ public interface GroupCoordinator { * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture consumerGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ConsumerGroupHeartbeatRequestData request ); @@ -94,7 +94,7 @@ CompletableFuture consumerGroupHeartbeat( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture streamsGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, StreamsGroupHeartbeatRequestData request ); @@ -108,7 +108,7 @@ CompletableFuture streamsGroupHeartbeat( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture shareGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ShareGroupHeartbeatRequestData request ); @@ -123,7 +123,7 @@ CompletableFuture shareGroupHeartbeat( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture joinGroup( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, BufferSupplier bufferSupplier ); @@ -139,7 +139,7 @@ CompletableFuture joinGroup( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture syncGroup( - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, BufferSupplier bufferSupplier ); @@ -154,7 +154,7 @@ CompletableFuture syncGroup( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture heartbeat( - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ); @@ -168,7 +168,7 @@ CompletableFuture heartbeat( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture leaveGroup( - RequestContext context, + AuthorizableRequestContext context, LeaveGroupRequestData request ); @@ -182,7 +182,7 @@ CompletableFuture leaveGroup( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture listGroups( - RequestContext context, + AuthorizableRequestContext context, ListGroupsRequestData request ); @@ -196,7 +196,7 @@ CompletableFuture listGroups( * The error codes of the results are set to indicate the errors occurred during the execution. */ CompletableFuture> describeGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds ); @@ -209,7 +209,7 @@ CompletableFuture> describeGroup * @return A future yielding the results or an exception. */ CompletableFuture> consumerGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ); @@ -222,7 +222,7 @@ CompletableFuture> consum * @return A future yielding the results or an exception. */ CompletableFuture> streamsGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ); @@ -235,7 +235,7 @@ CompletableFuture> streams * @return A future yielding the results or an exception. */ CompletableFuture> shareGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ); @@ -250,7 +250,7 @@ CompletableFuture> shareGrou * The error codes of the results are set to indicate the errors occurred during the execution. */ CompletableFuture deleteGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds, BufferSupplier bufferSupplier ); @@ -265,7 +265,7 @@ CompletableFuture delet * The error codes of the results are set to indicate the errors occurred during the execution. */ CompletableFuture fetchOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetFetchRequestData.OffsetFetchRequestGroup request, boolean requireStable ); @@ -280,7 +280,7 @@ CompletableFuture fetchOffsets * The error codes of the results are set to indicate the errors occurred during the execution. */ CompletableFuture fetchAllOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetFetchRequestData.OffsetFetchRequestGroup request, boolean requireStable ); @@ -295,7 +295,7 @@ CompletableFuture fetchAllOffs * The error codes of the response are set to indicate the errors occurred during the execution. */ CompletableFuture describeShareGroupOffsets( - RequestContext context, + AuthorizableRequestContext context, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup request ); @@ -309,7 +309,7 @@ CompletableFuture describeShareGroupAllOffsets( - RequestContext context, + AuthorizableRequestContext context, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup request ); @@ -323,7 +323,7 @@ CompletableFuture deleteShareGroupOffsets( - RequestContext context, + AuthorizableRequestContext context, DeleteShareGroupOffsetsRequestData request ); @@ -338,7 +338,7 @@ CompletableFuture deleteShareGroupOffsets( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture commitOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetCommitRequestData request, BufferSupplier bufferSupplier ); @@ -354,7 +354,7 @@ CompletableFuture commitOffsets( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture commitTransactionalOffsets( - RequestContext context, + AuthorizableRequestContext context, TxnOffsetCommitRequestData request, BufferSupplier bufferSupplier ); @@ -370,7 +370,7 @@ CompletableFuture commitTransactionalOffsets( * The error code(s) of the response are set to indicate the error(s) occurred during the execution. */ CompletableFuture deleteOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetDeleteRequestData request, BufferSupplier bufferSupplier ); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index de9f12337e156..46f7e513def95 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -66,7 +66,6 @@ import org.apache.kafka.common.requests.DescribeGroupsRequest; import org.apache.kafka.common.requests.DescribeShareGroupOffsetsRequest; import org.apache.kafka.common.requests.OffsetCommitRequest; -import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.ShareGroupDescribeRequest; import org.apache.kafka.common.requests.StreamsGroupDescribeRequest; import org.apache.kafka.common.requests.TransactionResult; @@ -88,6 +87,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.record.BrokerCompressionType; import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters; @@ -364,11 +364,11 @@ public int partitionFor( } /** - * See {@link GroupCoordinator#consumerGroupHeartbeat(RequestContext, ConsumerGroupHeartbeatRequestData)}. + * See {@link GroupCoordinator#consumerGroupHeartbeat(AuthorizableRequestContext, ConsumerGroupHeartbeatRequestData)}. */ @Override public CompletableFuture consumerGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ConsumerGroupHeartbeatRequestData request ) { if (!isActive.get()) { @@ -395,11 +395,11 @@ public CompletableFuture consumerGroupHeartb /** * See - * {@link GroupCoordinator#streamsGroupHeartbeat(RequestContext, org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData)}. + * {@link GroupCoordinator#streamsGroupHeartbeat(AuthorizableRequestContext, StreamsGroupHeartbeatRequestData)}. */ @Override public CompletableFuture streamsGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, StreamsGroupHeartbeatRequestData request ) { if (!isActive.get()) { @@ -432,11 +432,11 @@ public CompletableFuture streamsGroupHeartbeat( } /** - * See {@link GroupCoordinator#shareGroupHeartbeat(RequestContext, ShareGroupHeartbeatRequestData)}. + * See {@link GroupCoordinator#shareGroupHeartbeat(AuthorizableRequestContext, ShareGroupHeartbeatRequestData)}. */ @Override public CompletableFuture shareGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ShareGroupHeartbeatRequestData request ) { if (!isActive.get()) { @@ -592,11 +592,11 @@ CompletableFuture reconcileShareGroupStateInitializingState() { } /** - * See {@link GroupCoordinator#joinGroup(RequestContext, JoinGroupRequestData, BufferSupplier)}. + * See {@link GroupCoordinator#joinGroup(AuthorizableRequestContext, JoinGroupRequestData, BufferSupplier)}. */ @Override public CompletableFuture joinGroup( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, BufferSupplier bufferSupplier ) { @@ -646,11 +646,11 @@ public CompletableFuture joinGroup( } /** - * See {@link GroupCoordinator#syncGroup(RequestContext, SyncGroupRequestData, BufferSupplier)}. + * See {@link GroupCoordinator#syncGroup(AuthorizableRequestContext, SyncGroupRequestData, BufferSupplier)}. */ @Override public CompletableFuture syncGroup( - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, BufferSupplier bufferSupplier ) { @@ -690,11 +690,11 @@ public CompletableFuture syncGroup( } /** - * See {@link GroupCoordinator#heartbeat(RequestContext, HeartbeatRequestData)}. + * See {@link GroupCoordinator#heartbeat(AuthorizableRequestContext, HeartbeatRequestData)}. */ @Override public CompletableFuture heartbeat( - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ) { if (!isActive.get()) { @@ -733,11 +733,11 @@ public CompletableFuture heartbeat( } /** - * See {@link GroupCoordinator#leaveGroup(RequestContext, LeaveGroupRequestData)}. + * See {@link GroupCoordinator#leaveGroup(AuthorizableRequestContext, LeaveGroupRequestData)}. */ @Override public CompletableFuture leaveGroup( - RequestContext context, + AuthorizableRequestContext context, LeaveGroupRequestData request ) { if (!isActive.get()) { @@ -782,11 +782,11 @@ public CompletableFuture leaveGroup( } /** - * See {@link GroupCoordinator#listGroups(RequestContext, ListGroupsRequestData)}. + * See {@link GroupCoordinator#listGroups(AuthorizableRequestContext, ListGroupsRequestData)}. */ @Override public CompletableFuture listGroups( - RequestContext context, + AuthorizableRequestContext context, ListGroupsRequestData request ) { if (!isActive.get()) { @@ -827,11 +827,11 @@ public CompletableFuture listGroups( } /** - * See {@link GroupCoordinator#consumerGroupDescribe(RequestContext, List)}. + * See {@link GroupCoordinator#consumerGroupDescribe(AuthorizableRequestContext, List)}. */ @Override public CompletableFuture> consumerGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ) { if (!isActive.get()) { @@ -879,11 +879,11 @@ public CompletableFuture> } /** - * See {@link GroupCoordinator#streamsGroupDescribe(RequestContext, List)}. + * See {@link GroupCoordinator#streamsGroupDescribe(AuthorizableRequestContext, List)}. */ @Override public CompletableFuture> streamsGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ) { if (!isActive.get()) { @@ -931,11 +931,11 @@ public CompletableFuture> } /** - * See {@link GroupCoordinator#shareGroupDescribe(RequestContext, List)}. + * See {@link GroupCoordinator#shareGroupDescribe(AuthorizableRequestContext, List)}. */ @Override public CompletableFuture> shareGroupDescribe( - RequestContext context, + AuthorizableRequestContext context, List groupIds ) { if (!isActive.get()) { @@ -983,11 +983,11 @@ public CompletableFuture> shareGroupDescribe( } /** - * See {@link GroupCoordinator#describeGroups(RequestContext, List)}. + * See {@link GroupCoordinator#describeGroups(AuthorizableRequestContext, List)}. */ @Override public CompletableFuture> describeGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds ) { if (!isActive.get()) { @@ -1037,11 +1037,11 @@ public CompletableFuture> descri } /** - * See {@link GroupCoordinator#deleteGroups(RequestContext, List, BufferSupplier)}. + * See {@link GroupCoordinator#deleteGroups(AuthorizableRequestContext, List, BufferSupplier)}. */ @Override public CompletableFuture deleteGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds, BufferSupplier bufferSupplier ) { @@ -1141,7 +1141,7 @@ private List updateResponseAndGetNonErrorGroupList( } private CompletableFuture handleDeleteGroups( - RequestContext context, + AuthorizableRequestContext context, TopicPartition topicPartition, List groupIds ) { @@ -1296,11 +1296,11 @@ private CompletableFuture persistDeleteShar } /** - * See {@link GroupCoordinator#fetchOffsets(RequestContext, OffsetFetchRequestData.OffsetFetchRequestGroup, boolean)}. + * See {@link GroupCoordinator#fetchOffsets(AuthorizableRequestContext, OffsetFetchRequestData.OffsetFetchRequestGroup, boolean)}. */ @Override public CompletableFuture fetchOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetFetchRequestData.OffsetFetchRequestGroup request, boolean requireStable ) { @@ -1351,11 +1351,11 @@ public CompletableFuture fetch } /** - * See {@link GroupCoordinator#fetchAllOffsets(RequestContext, OffsetFetchRequestData.OffsetFetchRequestGroup, boolean)}. + * See {@link GroupCoordinator#fetchAllOffsets(AuthorizableRequestContext, OffsetFetchRequestData.OffsetFetchRequestGroup, boolean)}. */ @Override public CompletableFuture fetchAllOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetFetchRequestData.OffsetFetchRequestGroup request, boolean requireStable ) { @@ -1406,11 +1406,11 @@ public CompletableFuture fetch } /** - * See {@link GroupCoordinator#describeShareGroupOffsets(RequestContext, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup)}. + * See {@link GroupCoordinator#describeShareGroupOffsets(AuthorizableRequestContext, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup)}. */ @Override public CompletableFuture describeShareGroupOffsets( - RequestContext context, + AuthorizableRequestContext context, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup requestData ) { if (!isActive.get()) { @@ -1468,11 +1468,11 @@ public CompletableFuture describeShareGroupAllOffsets( - RequestContext context, + AuthorizableRequestContext context, DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup requestData ) { if (!isActive.get()) { @@ -1561,11 +1561,11 @@ private CompletableFuture deleteShareGroupOffsets( - RequestContext context, + AuthorizableRequestContext context, DeleteShareGroupOffsetsRequestData requestData ) { if (!isActive.get()) { @@ -1635,11 +1635,11 @@ public CompletableFuture deleteShareGroupOf } /** - * See {@link GroupCoordinator#commitOffsets(RequestContext, OffsetCommitRequestData, BufferSupplier)}. + * See {@link GroupCoordinator#commitOffsets(AuthorizableRequestContext, OffsetCommitRequestData, BufferSupplier)}. */ @Override public CompletableFuture commitOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetCommitRequestData request, BufferSupplier bufferSupplier ) { @@ -1673,11 +1673,11 @@ public CompletableFuture commitOffsets( } /** - * See {@link GroupCoordinator#commitTransactionalOffsets(RequestContext, TxnOffsetCommitRequestData, BufferSupplier)}. + * See {@link GroupCoordinator#commitTransactionalOffsets(AuthorizableRequestContext, TxnOffsetCommitRequestData, BufferSupplier)}. */ @Override public CompletableFuture commitTransactionalOffsets( - RequestContext context, + AuthorizableRequestContext context, TxnOffsetCommitRequestData request, BufferSupplier bufferSupplier ) { @@ -1703,7 +1703,7 @@ public CompletableFuture commitTransactionalOffsets request.producerEpoch(), Duration.ofMillis(config.offsetCommitTimeoutMs()), coordinator -> coordinator.commitTransactionalOffset(context, request), - context.apiVersion() + context.requestVersion() ).exceptionally(exception -> handleOperationException( "txn-commit-offset", request, @@ -1714,11 +1714,11 @@ public CompletableFuture commitTransactionalOffsets } /** - * See {@link GroupCoordinator#deleteOffsets(RequestContext, OffsetDeleteRequestData, BufferSupplier)}. + * See {@link GroupCoordinator#deleteOffsets(AuthorizableRequestContext, OffsetDeleteRequestData, BufferSupplier)}. */ @Override public CompletableFuture deleteOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetDeleteRequestData request, BufferSupplier bufferSupplier ) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java index a9d69e2c68f63..3af48c5396f15 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java @@ -55,7 +55,6 @@ import org.apache.kafka.common.message.TxnOffsetCommitResponseData; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -122,6 +121,7 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters; @@ -456,7 +456,7 @@ public int hashCode() { * a list of records to update the state machine. */ public CoordinatorResult consumerGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ConsumerGroupHeartbeatRequestData request ) { return groupMetadataManager.consumerGroupHeartbeat(context, request); @@ -472,7 +472,7 @@ public CoordinatorResult * a list of records to update the state machine. */ public CoordinatorResult streamsGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, StreamsGroupHeartbeatRequestData request ) { return groupMetadataManager.streamsGroupHeartbeat(context, request); @@ -488,7 +488,7 @@ public CoordinatorResult streams * and a list of records to update the state machine. */ public CoordinatorResult>, CoordinatorRecord> shareGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ShareGroupHeartbeatRequestData request ) { return groupMetadataManager.shareGroupHeartbeat(context, request); @@ -557,7 +557,7 @@ public Map> initializedShareGroupPartitions( * a list of records to update the state machine. */ public CoordinatorResult classicGroupJoin( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, CompletableFuture responseFuture ) { @@ -578,7 +578,7 @@ public CoordinatorResult classicGroupJoin( * a list of records to update the state machine. */ public CoordinatorResult classicGroupSync( - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, CompletableFuture responseFuture ) { @@ -599,7 +599,7 @@ public CoordinatorResult classicGroupSync( * a list of records to update the state machine. */ public CoordinatorResult classicGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ) { return groupMetadataManager.classicGroupHeartbeat( @@ -617,7 +617,7 @@ public CoordinatorResult classicGroupH * a list of records to update the state machine. */ public CoordinatorResult deleteGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds ) throws ApiException { final DeleteGroupsResponseData.DeletableGroupResultCollection resultCollection = @@ -774,7 +774,7 @@ public OffsetFetchResponseData.OffsetFetchResponseGroup fetchAllOffsets( * a list of records to update the state machine. */ public CoordinatorResult commitOffset( - RequestContext context, + AuthorizableRequestContext context, OffsetCommitRequestData request ) throws ApiException { return offsetMetadataManager.commitOffset(context, request); @@ -790,7 +790,7 @@ public CoordinatorResult commitOffs * a list of records to update the state machine. */ public CoordinatorResult commitTransactionalOffset( - RequestContext context, + AuthorizableRequestContext context, TxnOffsetCommitRequestData request ) throws ApiException { return offsetMetadataManager.commitTransactionalOffset(context, request); @@ -875,7 +875,7 @@ public List shareGroupDescribe( * @return A list containing the DescribeGroupsResponseData.DescribedGroup. */ public List describeGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds, long committedOffset ) { @@ -892,7 +892,7 @@ public List describeGroups( * a list of records to update the state machine. */ public CoordinatorResult classicGroupLeave( - RequestContext context, + AuthorizableRequestContext context, LeaveGroupRequestData request ) throws ApiException { return groupMetadataManager.classicGroupLeave(context, request); @@ -908,7 +908,7 @@ public CoordinatorResult classicGroup * a list of records to update the state machine. */ public CoordinatorResult deleteOffsets( - RequestContext context, + AuthorizableRequestContext context, OffsetDeleteRequestData request ) throws ApiException { return offsetMetadataManager.deleteOffsets(request); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 8bd3dfd5abcf0..2c03b2a30293c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -73,7 +73,6 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.requests.JoinGroupRequest; -import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; import org.apache.kafka.common.resource.ResourcePattern; @@ -167,6 +166,7 @@ import org.apache.kafka.image.TopicsDelta; import org.apache.kafka.image.TopicsImage; import org.apache.kafka.server.authorizer.Action; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.server.authorizer.AuthorizationResult; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters; @@ -743,7 +743,7 @@ public List streamsGroupDescrib * @return A list containing the DescribeGroupsResponseData.DescribedGroup. */ public List describeGroups( - RequestContext context, + AuthorizableRequestContext context, List groupIds, long committedOffset ) { @@ -779,7 +779,7 @@ public List describeGroups( ); } } catch (GroupIdNotFoundException exception) { - if (context.header.apiVersion() >= 6) { + if (context.requestVersion() >= 6) { describedGroups.add(new DescribeGroupsResponseData.DescribedGroup() .setGroupId(groupId) .setGroupState(DEAD.toString()) @@ -1502,7 +1502,7 @@ private static void throwIfNull( */ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( ConsumerGroupHeartbeatRequestData request, - short apiVersion + int apiVersion ) throws InvalidRequestException, UnsupportedAssignorException { if (apiVersion >= CONSUMER_GENERATED_MEMBER_ID_REQUIRED_VERSION || request.memberEpoch() > 0 || @@ -2369,7 +2369,7 @@ private List createStreamsGroupHeartb * a list of records to update the state machine. */ private CoordinatorResult consumerGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, String groupId, String memberId, int memberEpoch, @@ -2427,7 +2427,7 @@ private CoordinatorResult .maybeUpdateSubscribedTopicNames(Optional.ofNullable(subscribedTopicNames)) .maybeUpdateSubscribedTopicRegex(Optional.ofNullable(subscribedTopicRegex)) .setClientId(context.clientId()) - .setClientHost(context.clientAddress.toString()) + .setClientHost(context.clientAddress().toString()) .setClassicMemberMetadata(null) .build(); @@ -2538,7 +2538,7 @@ private CoordinatorResult */ private CoordinatorResult classicGroupJoinToConsumerGroup( ConsumerGroup group, - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, CompletableFuture responseFuture ) throws ApiException { @@ -2556,7 +2556,7 @@ private CoordinatorResult classicGroupJoinToConsumerGro throwIfConsumerGroupIsFull(group, memberId); throwIfClassicProtocolIsNotSupported(group, memberId, request.protocolType(), protocols); - if (JoinGroupRequest.requiresKnownMemberId(request, context.apiVersion())) { + if (JoinGroupRequest.requiresKnownMemberId(request, context.requestVersion())) { // A dynamic member requiring a member id joins the group. Send back a response to call for another // join group request with allocated member id. responseFuture.complete(new JoinGroupResponseData() @@ -2609,7 +2609,7 @@ private CoordinatorResult classicGroupJoinToConsumerGro .maybeUpdateServerAssignorName(Optional.empty()) .maybeUpdateSubscribedTopicNames(Optional.ofNullable(subscription.topics())) .setClientId(context.clientId()) - .setClientHost(context.clientAddress.toString()) + .setClientHost(context.clientAddress().toString()) .setClassicMemberMetadata( new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata() .setSessionTimeoutMs(sessionTimeoutMs) @@ -3228,7 +3228,7 @@ private static boolean isNotEmpty(String value) { * @return Whether a rebalance must be triggered. */ private boolean maybeUpdateRegularExpressions( - RequestContext context, + AuthorizableRequestContext context, ConsumerGroup group, ConsumerGroupMember member, ConsumerGroupMember updatedMember, @@ -3339,7 +3339,7 @@ private boolean maybeUpdateRegularExpressions( * public for benchmarks. */ public static Map refreshRegularExpressions( - RequestContext context, + AuthorizableRequestContext context, String groupId, Logger log, Time time, @@ -3404,7 +3404,7 @@ public static Map refreshRegularExpressions( * @param resolvedRegexes The map of the regex pattern and its set of matched topics. */ private static void filterTopicDescribeAuthorizedTopics( - RequestContext context, + AuthorizableRequestContext context, Optional> authorizerPlugin, Map> resolvedRegexes ) { @@ -4792,10 +4792,10 @@ private void cancelConsumerGroupSyncTimeout( * a list of records to update the state machine. */ public CoordinatorResult consumerGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ConsumerGroupHeartbeatRequestData request ) throws ApiException { - throwIfConsumerGroupHeartbeatRequestIsInvalid(request, context.apiVersion()); + throwIfConsumerGroupHeartbeatRequestIsInvalid(request, context.requestVersion()); if (request.memberEpoch() == LEAVE_GROUP_MEMBER_EPOCH || request.memberEpoch() == LEAVE_GROUP_STATIC_MEMBER_EPOCH) { // -1 means that the member wants to leave the group. @@ -4834,7 +4834,7 @@ public CoordinatorResult * a list of records to update the state machine. */ public CoordinatorResult streamsGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, StreamsGroupHeartbeatRequestData request ) throws ApiException { throwIfStreamsGroupHeartbeatRequestIsInvalid(request); @@ -4857,7 +4857,7 @@ public CoordinatorResult streams request.rackId(), request.rebalanceTimeoutMs(), context.clientId(), - context.clientAddress.toString(), + context.clientAddress().toString(), request.topology(), request.activeTasks(), request.standbyTasks(), @@ -4917,7 +4917,7 @@ public void replay( * and a list of records to update the state machine. */ public CoordinatorResult>, CoordinatorRecord> shareGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, ShareGroupHeartbeatRequestData request ) throws ApiException { throwIfShareGroupHeartbeatRequestIsInvalid(request); @@ -4941,7 +4941,7 @@ public CoordinatorResult classicGroupJoin( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, CompletableFuture responseFuture ) { @@ -6244,7 +6244,7 @@ public CoordinatorResult classicGroupJoin( * @return The result that contains records to append if the join group phase completes. */ CoordinatorResult classicGroupJoinToClassicGroup( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, CompletableFuture responseFuture ) { @@ -6360,7 +6360,7 @@ private CoordinatorResult maybeCompleteJoinPhase(Classi * @return The coordinator result that will be appended to the log. */ private CoordinatorResult classicGroupJoinNewMember( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, CompletableFuture responseFuture @@ -6418,7 +6418,7 @@ private CoordinatorResult classicGroupJoinNewMember( * @return The coordinator result that will be appended to the log. */ private CoordinatorResult classicGroupJoinNewStaticMember( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, String newMemberId, @@ -6464,13 +6464,13 @@ private CoordinatorResult classicGroupJoinNewStaticMemb * @return The coordinator result that will be appended to the log. */ private CoordinatorResult classicGroupJoinNewDynamicMember( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, String newMemberId, CompletableFuture responseFuture ) { - if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) { + if (JoinGroupRequest.requiresKnownMemberId(context.requestVersion())) { // If member id required, register the member in the pending member list and send // back a response to call for another join group request with allocated member id. log.info("Dynamic member with unknown member id joins group {} in {} state. " + @@ -6514,7 +6514,7 @@ private CoordinatorResult classicGroupJoinNewDynamicMem * @return The coordinator result that will be appended to the log. */ private CoordinatorResult classicGroupJoinExistingMember( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, CompletableFuture responseFuture @@ -6937,7 +6937,7 @@ private CoordinatorResult updateMemberThenRebalanceOrCo * @return The coordinator result that will be appended to the log. */ private CoordinatorResult addMemberThenRebalanceOrCompleteJoin( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, String memberId, @@ -7318,7 +7318,7 @@ private boolean acceptJoiningMember(ClassicGroup group, String memberId) { * @return The coordinator result that will be appended to the log. */ private CoordinatorResult updateStaticMemberThenRebalanceOrCompleteJoin( - RequestContext context, + AuthorizableRequestContext context, JoinGroupRequestData request, ClassicGroup group, String oldMemberId, @@ -7377,7 +7377,7 @@ private CoordinatorResult updateStaticMemberThenRebalan .setSkipAssignment(false) .setErrorCode(appendGroupMetadataErrorToResponseError(Errors.forException(t)).code())); - } else if (JoinGroupRequest.supportsSkippingAssignment(context.apiVersion())) { + } else if (JoinGroupRequest.supportsSkippingAssignment(context.requestVersion())) { boolean isLeader = group.isLeader(newMemberId); group.completeJoinFuture(newMember, new JoinGroupResponseData() @@ -7442,7 +7442,7 @@ private CoordinatorResult updateStaticMemberThenRebalan * @return The result that contains records to append. */ public CoordinatorResult classicGroupSync( - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, CompletableFuture responseFuture ) throws UnknownMemberIdException { @@ -7484,7 +7484,7 @@ public CoordinatorResult classicGroupSync( */ private CoordinatorResult classicGroupSyncToClassicGroup( ClassicGroup group, - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, CompletableFuture responseFuture ) throws IllegalStateException { @@ -7582,7 +7582,7 @@ private CoordinatorResult classicGroupSyncToClassicGrou */ private CoordinatorResult classicGroupSyncToConsumerGroup( ConsumerGroup group, - RequestContext context, + AuthorizableRequestContext context, SyncGroupRequestData request, CompletableFuture responseFuture ) throws UnknownMemberIdException, FencedInstanceIdException, IllegalGenerationException, @@ -7721,7 +7721,7 @@ private void removePendingSyncMember( * @return The coordinator result that contains the heartbeat response. */ public CoordinatorResult classicGroupHeartbeat( - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ) { Group group; @@ -7755,7 +7755,7 @@ public CoordinatorResult classicGroupH */ private CoordinatorResult classicGroupHeartbeatToClassicGroup( ClassicGroup group, - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ) { validateClassicGroupHeartbeat(group, request.memberId(), request.groupInstanceId(), request.generationId()); @@ -7838,7 +7838,7 @@ private void validateClassicGroupHeartbeat( */ private CoordinatorResult classicGroupHeartbeatToConsumerGroup( ConsumerGroup group, - RequestContext context, + AuthorizableRequestContext context, HeartbeatRequestData request ) throws UnknownMemberIdException, FencedInstanceIdException, IllegalGenerationException { String groupId = request.groupId(); @@ -7908,7 +7908,7 @@ private ConsumerGroupMember validateConsumerGroupMember( * @return The LeaveGroup response and the records to append. */ public CoordinatorResult classicGroupLeave( - RequestContext context, + AuthorizableRequestContext context, LeaveGroupRequestData request ) throws UnknownMemberIdException { Group group; @@ -8135,7 +8135,7 @@ private void removeCurrentMemberFromClassicGroup( /** * Handles a DeleteGroups request. * Populates the record list passed in with record to update the state machine. - * Validations are done in {@link GroupCoordinatorShard#deleteGroups(RequestContext, List)} by + * Validations are done in {@link GroupCoordinatorShard#deleteGroups(AuthorizableRequestContext, List)} by * calling {@link GroupMetadataManager#validateDeleteGroup(String)}. * * @param groupId The id of the group to be deleted. It has been checked in {@link GroupMetadataManager#validateDeleteGroup}. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 9f601a74917f5..2b50071a7f771 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -35,7 +35,6 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.OffsetCommitRequest; -import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -48,6 +47,7 @@ import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics; import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard; import org.apache.kafka.image.MetadataImage; +import org.apache.kafka.server.authorizer.AuthorizableRequestContext; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; import org.apache.kafka.timeline.TimelineHashSet; @@ -290,7 +290,7 @@ private OffsetAndMetadata remove( * @param request The actual request. */ private Group validateOffsetCommit( - RequestContext context, + AuthorizableRequestContext context, OffsetCommitRequestData request ) throws ApiException { Group group; @@ -305,7 +305,7 @@ private Group validateOffsetCommit( log.info("[GroupId {}] Creating a simple consumer group via manual offset commit.", request.groupId()); group = groupMetadataManager.getOrMaybeCreateClassicGroup(request.groupId(), true); } else { - if (context.header.apiVersion() >= 9) { + if (context.requestVersion() >= 9) { // Starting from version 9 of the OffsetCommit API, we return GROUP_ID_NOT_FOUND // if the group does not exist. This error works for both the old and the new // protocol for clients using this version of the API. @@ -323,7 +323,7 @@ private Group validateOffsetCommit( request.groupInstanceId(), request.generationIdOrMemberEpoch(), false, - context.apiVersion() + context.requestVersion() ); return group; @@ -336,7 +336,7 @@ private Group validateOffsetCommit( * @param request The actual request. */ private Group validateTransactionalOffsetCommit( - RequestContext context, + AuthorizableRequestContext context, TxnOffsetCommitRequestData request ) throws ApiException { Group group; @@ -360,7 +360,7 @@ private Group validateTransactionalOffsetCommit( request.groupInstanceId(), request.generationId(), true, - context.apiVersion() + context.requestVersion() ); } catch (StaleMemberEpochException ex) { throw Errors.ILLEGAL_GENERATION.exception(); @@ -438,7 +438,7 @@ private static OptionalLong expireTimestampMs( * a list of records to update the state machine. */ public CoordinatorResult commitOffset( - RequestContext context, + AuthorizableRequestContext context, OffsetCommitRequestData request ) throws ApiException { Group group = validateOffsetCommit(context, request); @@ -511,7 +511,7 @@ public CoordinatorResult commitOffs * a list of records to update the state machine. */ public CoordinatorResult commitTransactionalOffset( - RequestContext context, + AuthorizableRequestContext context, TxnOffsetCommitRequestData request ) throws ApiException { validateTransactionalOffsetCommit(context, request); @@ -623,7 +623,7 @@ public CoordinatorResult deleteOffs /** * Deletes offsets as part of a DeleteGroups request. * Populates the record list passed in with records to update the state machine. - * Validations are done in {@link GroupCoordinatorShard#deleteGroups(RequestContext, List)} + * Validations are done in {@link GroupCoordinatorShard#deleteGroups(AuthorizableRequestContext, List)} * * @param groupId The id of the given group. * @param records The record list to populate. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java index 39a05be844d3c..0e14d4e63db00 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java @@ -831,7 +831,7 @@ public void validateOffsetCommit( String groupInstanceId, int generationId, boolean isTransactional, - short apiVersion + int apiVersion ) throws CoordinatorNotAvailableException, UnknownMemberIdException, IllegalGenerationException, FencedInstanceIdException { if (isInState(DEAD)) { throw Errors.COORDINATOR_NOT_AVAILABLE.exception(); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java index ef95b123ff983..aea5bb4250026 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java @@ -644,7 +644,7 @@ public void validateOffsetCommit( String groupInstanceId, int memberEpoch, boolean isTransactional, - short apiVersion + int apiVersion ) throws UnknownMemberIdException, StaleMemberEpochException, IllegalGenerationException { // When the member epoch is -1, the request comes from either the admin client // or a consumer which does not use the group management facility. In this case, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java index b63100744c6b8..eacbab7dbfff9 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java @@ -186,7 +186,7 @@ public void validateOffsetCommit( String groupInstanceId, int memberEpoch, boolean isTransactional, - short apiVersion + int apiVersion ) { throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId)); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index 3ace0d1c5b69e..549eefe618a70 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -683,7 +683,7 @@ public void validateOffsetCommit( String groupInstanceId, int memberEpoch, boolean isTransactional, - short apiVersion + int apiVersion ) throws UnknownMemberIdException, StaleMemberEpochException { // When the member epoch is -1, the request comes from either the admin client // or a consumer which does not use the group management facility. In this case, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 01e74ea558adb..003d864b93762 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -2396,7 +2396,7 @@ public void testCommitTransactionalOffsetsWithInvalidGroupId(String groupId) thr @ParameterizedTest @ValueSource(shorts = {4, 5}) - public void testCommitTransactionalOffsets(Short txnOffsetCommitVersion) throws ExecutionException, InterruptedException { + public void testCommitTransactionalOffsets(short txnOffsetCommitVersion) throws ExecutionException, InterruptedException { CoordinatorRuntime runtime = mockRuntime(); GroupCoordinatorService service = new GroupCoordinatorServiceBuilder() .setConfig(createConfig()) @@ -2431,7 +2431,7 @@ public void testCommitTransactionalOffsets(Short txnOffsetCommitVersion) throws ArgumentMatchers.eq((short) 5), ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any(), - ArgumentMatchers.any() + ArgumentMatchers.eq((int) txnOffsetCommitVersion) )).thenReturn(CompletableFuture.completedFuture(response)); CompletableFuture future = service.commitTransactionalOffsets( @@ -2486,7 +2486,7 @@ public void testCommitTransactionalOffsetsWithWrappedError( ArgumentMatchers.eq((short) 5), ArgumentMatchers.eq(Duration.ofMillis(5000)), ArgumentMatchers.any(), - ArgumentMatchers.any() + ArgumentMatchers.eq((int) ApiKeys.TXN_OFFSET_COMMIT.latestVersion()) )).thenReturn(FutureUtils.failedFuture(new CompletionException(error.exception()))); CompletableFuture future = service.commitTransactionalOffsets( From 1daa5b6cf2460acf6f9245a1f3877b6bd9d745ed Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 16 Apr 2025 08:42:34 +0200 Subject: [PATCH 2/5] update CoordinatorPartitionWriter --- .../kafka/coordinator/group/CoordinatorPartitionWriter.scala | 2 +- .../src/main/scala/kafka/server/AddPartitionsToTxnManager.scala | 2 +- .../coordinator/group/CoordinatorPartitionWriterTest.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala index 64f23c32f5228..d6ea17fe7946b 100644 --- a/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala +++ b/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala @@ -107,7 +107,7 @@ class CoordinatorPartitionWriter( transactionalId: String, producerId: Long, producerEpoch: Short, - apiVersion: Short + apiVersion: Int ): CompletableFuture[VerificationGuard] = { val transactionSupportedOperation = AddPartitionsToTxnManager.txnOffsetCommitRequestVersionToTransactionSupportedOperation(apiVersion) val future = new CompletableFuture[VerificationGuard]() diff --git a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala index 050f48cd4e487..b7e3bd36d8478 100644 --- a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala +++ b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala @@ -51,7 +51,7 @@ object AddPartitionsToTxnManager { } } - def txnOffsetCommitRequestVersionToTransactionSupportedOperation(version: Short): TransactionSupportedOperation = { + def txnOffsetCommitRequestVersionToTransactionSupportedOperation(version: Int): TransactionSupportedOperation = { if (version > 4) { addPartition } else if (version > 3) { diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala index a56dab4fba78a..3a6ee39dcc1e5 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala @@ -178,7 +178,7 @@ class CoordinatorPartitionWriterTest { "transactional-id", 10L, 5.toShort, - ApiKeys.TXN_OFFSET_COMMIT.latestVersion() + ApiKeys.TXN_OFFSET_COMMIT.latestVersion().toInt ) if (error == Errors.NONE) { From 870585520fdc45481bc50424db24abc661214fa8 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 16 Apr 2025 09:08:39 +0200 Subject: [PATCH 3/5] fix KafkaApis style --- .../main/scala/kafka/server/KafkaApis.scala | 125 ++++++++++++++---- 1 file changed, 98 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c754c67568939..70d2bd6627d4f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -349,7 +349,11 @@ class KafkaApis(val requestChannel: RequestChannel, .setGroupInstanceId(offsetCommitRequest.data.groupInstanceId) .setTopics(authorizedTopicsRequest.asJava) - groupCoordinator.commitOffsets(request.context, offsetCommitRequestData, requestLocal.bufferSupplier).handle[Unit] { (results, exception) => + groupCoordinator.commitOffsets( + request.context, + offsetCommitRequestData, + requestLocal.bufferSupplier + ).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(exception)) } else { @@ -1023,7 +1027,11 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest: OffsetFetchRequestData.OffsetFetchRequestGroup, requireStable: Boolean ): CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup] = { - groupCoordinator.fetchAllOffsets(requestContext, offsetFetchRequest, requireStable).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => + groupCoordinator.fetchAllOffsets( + requestContext, + offsetFetchRequest, + requireStable + ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => if (exception != null) { new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId(offsetFetchRequest.groupId) @@ -1056,11 +1064,15 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest.topics.asScala )(_.name) - groupCoordinator.fetchOffsets(requestContext, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId(offsetFetchRequest.groupId) - .setMemberId(offsetFetchRequest.memberId) - .setMemberEpoch(offsetFetchRequest.memberEpoch) - .setTopics(authorizedTopics.asJava), requireStable).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => + groupCoordinator.fetchOffsets( + requestContext, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(offsetFetchRequest.groupId) + .setMemberId(offsetFetchRequest.memberId) + .setMemberEpoch(offsetFetchRequest.memberEpoch) + .setTopics(authorizedTopics.asJava), + requireStable + ).handle[OffsetFetchResponseData.OffsetFetchResponseGroup] { (offsetFetchResponse, exception) => if (exception != null) { new OffsetFetchResponseData.OffsetFetchResponseGroup() .setGroupId(offsetFetchRequest.groupId) @@ -1222,7 +1234,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.describeGroups(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => + groupCoordinator.describeGroups( + request.context, + authorizedGroups.asJava + ).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, describeRequest.getErrorResponse(exception)) } else { @@ -1254,7 +1269,10 @@ class KafkaApis(val requestChannel: RequestChannel, val listGroupsRequest = request.body[ListGroupsRequest] val hasClusterDescribe = authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME, logIfDenied = false) - groupCoordinator.listGroups(request.context, listGroupsRequest.data).handle[Unit] { (response, exception) => + groupCoordinator.listGroups( + request.context, + listGroupsRequest.data + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, listGroupsRequest.getErrorResponse(exception)) } else { @@ -1283,7 +1301,11 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.joinGroup(request.context, joinGroupRequest.data, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => + groupCoordinator.joinGroup( + request.context, + joinGroupRequest.data, + requestLocal.bufferSupplier + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, joinGroupRequest.getErrorResponse(exception)) } else { @@ -1307,7 +1329,11 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.syncGroup(request.context, syncGroupRequest.data, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => + groupCoordinator.syncGroup( + request.context, + syncGroupRequest.data, + requestLocal.bufferSupplier + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, syncGroupRequest.getErrorResponse(exception)) } else { @@ -1327,7 +1353,11 @@ class KafkaApis(val requestChannel: RequestChannel, val (authorizedGroups, unauthorizedGroups) = authHelper.partitionSeqByAuthorized(request.context, DELETE, GROUP, groups)(identity) - groupCoordinator.deleteGroups(request.context, authorizedGroups.toList.asJava, requestLocal.bufferSupplier).handle[Unit] { (results, exception) => + groupCoordinator.deleteGroups( + request.context, + authorizedGroups.toList.asJava, + requestLocal.bufferSupplier + ).handle[Unit] { (results, exception) => val response = new DeleteGroupsResponseData() if (exception != null) { @@ -1358,7 +1388,10 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.heartbeat(request.context, heartbeatRequest.data).handle[Unit] { (response, exception) => + groupCoordinator.heartbeat( + request.context, + heartbeatRequest.data + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, heartbeatRequest.getErrorResponse(exception)) } else { @@ -1375,7 +1408,10 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, leaveGroupRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { - groupCoordinator.leaveGroup(request.context, leaveGroupRequest.normalizedData()).handle[Unit] { (response, exception) => + groupCoordinator.leaveGroup( + request.context, + leaveGroupRequest.normalizedData() + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, leaveGroupRequest.getErrorResponse(exception)) } else { @@ -1977,7 +2013,11 @@ class KafkaApis(val requestChannel: RequestChannel, .setTransactionalId(txnOffsetCommitRequest.data.transactionalId) .setTopics(authorizedTopicCommittedOffsets.asJava) - groupCoordinator.commitTransactionalOffsets(request.context, txnOffsetCommitRequestData, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => + groupCoordinator.commitTransactionalOffsets( + request.context, + txnOffsetCommitRequestData, + requestLocal.bufferSupplier + ).handle[Unit] { (response, exception) => if (exception != null) { sendResponse(txnOffsetCommitRequest.getErrorResponse(exception)) } else { @@ -2284,7 +2324,11 @@ class KafkaApis(val requestChannel: RequestChannel, .setGroupId(offsetDeleteRequest.data.groupId) .setTopics(authorizedTopicPartitions) - groupCoordinator.deleteOffsets(request.context, offsetDeleteRequestData, requestLocal.bufferSupplier).handle[Unit] { (response, exception) => + groupCoordinator.deleteOffsets( + request.context, + offsetDeleteRequestData, + requestLocal.bufferSupplier + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, offsetDeleteRequest.getErrorResponse(exception)) } else { @@ -2490,7 +2534,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.consumerGroupHeartbeat(request.context, consumerGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => + groupCoordinator.consumerGroupHeartbeat( + request.context, + consumerGroupHeartbeatRequest.data + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, consumerGroupHeartbeatRequest.getErrorResponse(exception)) } else { @@ -2524,7 +2571,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.consumerGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => + groupCoordinator.consumerGroupDescribe( + request.context, + authorizedGroups.asJava + ).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, consumerGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -2643,7 +2693,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.streamsGroupHeartbeat(request.context, streamsGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => + groupCoordinator.streamsGroupHeartbeat( + request.context, + streamsGroupHeartbeatRequest.data + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, streamsGroupHeartbeatRequest.getErrorResponse(exception)) } else { @@ -2699,7 +2752,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.streamsGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => + groupCoordinator.streamsGroupDescribe( + request.context, + authorizedGroups.asJava + ).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, streamsGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -2831,7 +2887,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.shareGroupHeartbeat(request.context, shareGroupHeartbeatRequest.data).handle[Unit] { (response, exception) => + groupCoordinator.shareGroupHeartbeat( + request.context, + shareGroupHeartbeatRequest.data + ).handle[Unit] { (response, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, shareGroupHeartbeatRequest.getErrorResponse(exception)) @@ -2864,7 +2923,10 @@ class KafkaApis(val requestChannel: RequestChannel, } } - groupCoordinator.shareGroupDescribe(request.context, authorizedGroups.asJava).handle[Unit] { (results, exception) => + groupCoordinator.shareGroupDescribe( + request.context, + authorizedGroups.asJava + ).handle[Unit] { (results, exception) => if (exception != null) { requestHelper.sendMaybeThrottle(request, shareGroupDescribeRequest.getErrorResponse(exception)) } else { @@ -3490,7 +3552,10 @@ class KafkaApis(val requestChannel: RequestChannel, private def describeShareGroupAllOffsetsForGroup(requestContext: RequestContext, groupDescribeOffsetsRequest: DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup ): CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] = { - groupCoordinator.describeShareGroupAllOffsets(requestContext, groupDescribeOffsetsRequest).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => + groupCoordinator.describeShareGroupAllOffsets( + requestContext, + groupDescribeOffsetsRequest + ).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => if (exception != null) { val error = Errors.forException(exception) new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup() @@ -3521,9 +3586,12 @@ class KafkaApis(val requestChannel: RequestChannel, groupDescribeOffsetsRequest.topics.asScala )(_.topicName) - groupCoordinator.describeShareGroupOffsets(requestContext, new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() - .setGroupId(groupDescribeOffsetsRequest.groupId) - .setTopics(authorizedTopics.asJava)).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => + groupCoordinator.describeShareGroupOffsets( + requestContext, + new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() + .setGroupId(groupDescribeOffsetsRequest.groupId) + .setTopics(authorizedTopics.asJava) + ).handle[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] { (groupDescribeOffsetsResponse, exception) => if (exception != null) { val error = Errors.forException(exception) new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup() @@ -3601,7 +3669,10 @@ class KafkaApis(val requestChannel: RequestChannel, return } - groupCoordinator.deleteShareGroupOffsets(request.context, new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics)).handle[Unit] { (responseData, exception) => { + groupCoordinator.deleteShareGroupOffsets( + request.context, + new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics) + ).handle[Unit] { (responseData, exception) => { if (exception != null) { requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception)) } else if (responseData.errorCode() != Errors.NONE.code) { From 35533d725256cd4cae43ae9d3bf44ffd0cc08dff Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 16 Apr 2025 09:13:24 +0200 Subject: [PATCH 4/5] fix style --- .../unit/kafka/server/KafkaApisTest.scala | 479 ++++++++++++++---- 1 file changed, 368 insertions(+), 111 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index b3d7254067a83..c644043168438 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -1010,7 +1010,11 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets(requestChannelRequest.context, offsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitOffsets( + requestChannelRequest.context, + offsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1050,7 +1054,11 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets(requestChannelRequest.context, offsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitOffsets( + requestChannelRequest.context, + offsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( @@ -1143,7 +1151,11 @@ class KafkaApisTest extends Logging { .setCommittedOffset(50)).asJava)).asJava) val future = new CompletableFuture[OffsetCommitResponseData]() - when(groupCoordinator.commitOffsets(requestChannelRequest.context, expectedOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitOffsets( + requestChannelRequest.context, + expectedOffsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1316,7 +1328,11 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(txnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, txnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets( + requestChannelRequest.context, + txnOffsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1357,7 +1373,11 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(txnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, txnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets( + requestChannelRequest.context, + txnOffsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1450,7 +1470,11 @@ class KafkaApisTest extends Logging { val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(expectedTxnOffsetCommitRequest.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets(requestChannelRequest.context, expectedTxnOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets( + requestChannelRequest.context, + expectedTxnOffsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -1549,7 +1573,11 @@ class KafkaApisTest extends Logging { val requestLocal = RequestLocal.withThreadConfinedCaching val future = new CompletableFuture[TxnOffsetCommitResponseData]() when(txnCoordinator.partitionFor(offsetCommitRequest.data.transactionalId)).thenReturn(0) - when(groupCoordinator.commitTransactionalOffsets(request.context, offsetCommitRequest.data, requestLocal.bufferSupplier)).thenReturn(future) + when(groupCoordinator.commitTransactionalOffsets( + request.context, + offsetCommitRequest.data, + requestLocal.bufferSupplier + )).thenReturn(future) future.complete(new TxnOffsetCommitResponseData() .setTopics(List( @@ -2890,7 +2918,11 @@ class KafkaApisTest extends Logging { 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, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteGroups( + requestChannelRequest.context, + List("group-1", "group-2", "group-3").asJava, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -2929,7 +2961,11 @@ class KafkaApisTest extends Logging { 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, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteGroups( + requestChannelRequest.context, + List("group-1", "group-2", "group-3").asJava, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -2984,7 +3020,11 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[DeleteGroupsResponseData.DeletableGroupResultCollection]() - when(groupCoordinator.deleteGroups(requestChannelRequest.context, List("group-2", "group-3").asJava, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteGroups( + requestChannelRequest.context, + List("group-2", "group-3").asJava, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDeleteGroupsRequest( requestChannelRequest, @@ -3028,7 +3068,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups(requestChannelRequest.context, describeGroupsRequest.groups)).thenReturn(future) + when(groupCoordinator.describeGroups( + requestChannelRequest.context, + describeGroupsRequest.groups + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3072,7 +3115,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build()) val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups(requestChannelRequest.context, describeGroupsRequest.groups)).thenReturn(future) + when(groupCoordinator.describeGroups( + requestChannelRequest.context, + describeGroupsRequest.groups + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3123,7 +3169,10 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]]() - when(groupCoordinator.describeGroups(requestChannelRequest.context, List("group-2").asJava)).thenReturn(future) + when(groupCoordinator.describeGroups( + requestChannelRequest.context, + List("group-2").asJava + )).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleDescribeGroupsRequest(requestChannelRequest) @@ -3177,7 +3226,11 @@ class KafkaApisTest extends Logging { val requestLocal = RequestLocal.withThreadConfinedCaching val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets(request.context, offsetDeleteRequest.data, requestLocal.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteOffsets( + request.context, + offsetDeleteRequest.data, + requestLocal.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, requestLocal) @@ -3266,7 +3319,11 @@ class KafkaApisTest extends Logging { ).asJava.iterator)) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets(requestChannelRequest.context, expectedOffsetDeleteRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteOffsets( + requestChannelRequest.context, + expectedOffsetDeleteRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handle( requestChannelRequest, @@ -3367,7 +3424,11 @@ class KafkaApisTest extends Logging { // The group coordinator is called even if there are no // topic-partitions left after the validation. - when(groupCoordinator.deleteOffsets(request.context, new OffsetDeleteRequestData().setGroupId(group), RequestLocal.noCaching.bufferSupplier)).thenReturn(CompletableFuture.completedFuture( + when(groupCoordinator.deleteOffsets( + request.context, + new OffsetDeleteRequestData().setGroupId(group), + RequestLocal.noCaching.bufferSupplier + )).thenReturn(CompletableFuture.completedFuture( new OffsetDeleteResponseData() )) val kafkaApis = createKafkaApis() @@ -3399,7 +3460,11 @@ class KafkaApisTest extends Logging { val request = buildRequest(offsetDeleteRequest) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets(request.context, offsetDeleteRequest.data, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteOffsets( + request.context, + offsetDeleteRequest.data, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, RequestLocal.noCaching) @@ -3429,7 +3494,11 @@ class KafkaApisTest extends Logging { val request = buildRequest(offsetDeleteRequest) val future = new CompletableFuture[OffsetDeleteResponseData]() - when(groupCoordinator.deleteOffsets(request.context, new OffsetDeleteRequestData().setGroupId(group), RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.deleteOffsets( + request.context, + new OffsetDeleteRequestData().setGroupId(group), // Nonexistent topics won't be passed to groupCoordinator. + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetDeleteRequest(request, RequestLocal.noCaching) @@ -7515,7 +7584,11 @@ class KafkaApisTest extends Logging { .setSessionTimeoutMs(joinGroupRequest.sessionTimeoutMs) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup(requestChannelRequest.context, expectedJoinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.joinGroup( + requestChannelRequest.context, + expectedJoinGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7554,7 +7627,11 @@ class KafkaApisTest extends Logging { .setSessionTimeoutMs(joinGroupRequest.sessionTimeoutMs) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup(requestChannelRequest.context, expectedJoinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.joinGroup( + requestChannelRequest.context, + expectedJoinGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7588,7 +7665,11 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new JoinGroupRequest.Builder(joinGroupRequest).build()) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup(requestChannelRequest.context, joinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.joinGroup( + requestChannelRequest.context, + joinGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleJoinGroupRequest( requestChannelRequest, @@ -7636,7 +7717,11 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new JoinGroupRequest.Builder(joinGroupRequest).build()) val future = new CompletableFuture[JoinGroupResponseData]() - when(groupCoordinator.joinGroup(requestChannelRequest.context, joinGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.joinGroup( + requestChannelRequest.context, + joinGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) var response: JoinGroupResponse = null when(requestChannel.sendResponse(any(), any(), any())).thenAnswer { _ => @@ -7675,7 +7760,11 @@ class KafkaApisTest extends Logging { .setProtocolName(if (version >= 5) "range" else null) val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.syncGroup( + requestChannelRequest.context, + expectedSyncGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7708,7 +7797,11 @@ class KafkaApisTest extends Logging { .setProtocolName("range") val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.syncGroup( + requestChannelRequest.context, + expectedSyncGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7757,7 +7850,11 @@ class KafkaApisTest extends Logging { .setMemberId("member") val future = new CompletableFuture[SyncGroupResponseData]() - when(groupCoordinator.syncGroup(requestChannelRequest.context, expectedSyncGroupRequest, RequestLocal.noCaching.bufferSupplier)).thenReturn(future) + when(groupCoordinator.syncGroup( + requestChannelRequest.context, + expectedSyncGroupRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleSyncGroupRequest( requestChannelRequest, @@ -7795,7 +7892,10 @@ class KafkaApisTest extends Logging { .setGenerationId(0) val future = new CompletableFuture[HeartbeatResponseData]() - when(groupCoordinator.heartbeat(requestChannelRequest.context, expectedHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.heartbeat( + requestChannelRequest.context, + expectedHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleHeartbeatRequest(requestChannelRequest) @@ -7820,7 +7920,10 @@ class KafkaApisTest extends Logging { .setGenerationId(0) val future = new CompletableFuture[HeartbeatResponseData]() - when(groupCoordinator.heartbeat(requestChannelRequest.context, expectedHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.heartbeat( + requestChannelRequest.context, + expectedHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleHeartbeatRequest(requestChannelRequest) @@ -7885,7 +7988,10 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) + when(groupCoordinator.leaveGroup( + requestChannelRequest.context, + expectedLeaveGroupRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -7927,7 +8033,10 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) + when(groupCoordinator.leaveGroup( + requestChannelRequest.context, + expectedLeaveGroupRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -7977,7 +8086,10 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) + when(groupCoordinator.leaveGroup( + requestChannelRequest.context, + expectedLeaveGroupRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleLeaveGroupRequest(requestChannelRequest) @@ -8006,7 +8118,10 @@ class KafkaApisTest extends Logging { ).asJava) val future = new CompletableFuture[LeaveGroupResponseData]() - when(groupCoordinator.leaveGroup(requestChannelRequest.context, expectedLeaveGroupRequest)).thenReturn(future) + when(groupCoordinator.leaveGroup( + requestChannelRequest.context, + expectedLeaveGroupRequest + )).thenReturn(future) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -8041,27 +8156,43 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List( - new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), false)).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("foo") + .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), + false + )).thenReturn(group1Future) val group2Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-2") - .setTopics(null), false)).thenReturn(group2Future) + when(groupCoordinator.fetchAllOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-2") + .setTopics(null), + false + )).thenReturn(group2Future) val group3Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-3") - .setTopics(null), false)).thenReturn(group3Future) + when(groupCoordinator.fetchAllOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-3") + .setTopics(null), + false + )).thenReturn(group3Future) val group4Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-4") - .setTopics(null), false)).thenReturn(group4Future) + when(groupCoordinator.fetchAllOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-4") + .setTopics(null), + false + )).thenReturn(group4Future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8141,11 +8272,15 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("foo") - .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), false)).thenReturn(future) + when(groupCoordinator.fetchOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("foo") + .setPartitionIndexes(List[Integer](0, 1).asJava)).asJava), + false + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8212,9 +8347,13 @@ class KafkaApisTest extends Logging { val requestChannelRequest = makeRequest(version) val future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(null), false)).thenReturn(future) + when(groupCoordinator.fetchAllOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(null), + false + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleOffsetFetchRequest(requestChannelRequest) @@ -8305,17 +8444,25 @@ class KafkaApisTest extends Logging { // group-1 is allowed and bar is allowed. val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), + false + )).thenReturn(group1Future) // group-3 is allowed and bar is allowed. val group3Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchAllOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-3") - .setTopics(null), false)).thenReturn(group3Future) + when(groupCoordinator.fetchAllOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-3") + .setTopics(null), + false + )).thenReturn(group3Future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -8446,18 +8593,26 @@ class KafkaApisTest extends Logging { // group-1 and group-2 are allowed and bar is allowed. val group1Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-1") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-1") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), + false + )).thenReturn(group1Future) val group2Future = new CompletableFuture[OffsetFetchResponseData.OffsetFetchResponseGroup]() - when(groupCoordinator.fetchOffsets(requestChannelRequest.context, new OffsetFetchRequestData.OffsetFetchRequestGroup() - .setGroupId("group-2") - .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() - .setName("bar") - .setPartitionIndexes(List[Integer](0).asJava)).asJava), false)).thenReturn(group1Future) + when(groupCoordinator.fetchOffsets( + requestChannelRequest.context, + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId("group-2") + .setTopics(List(new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName("bar") + .setPartitionIndexes(List[Integer](0).asJava)).asJava), + false + )).thenReturn(group1Future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -8558,7 +8713,10 @@ class KafkaApisTest extends Logging { .setTypesFilter(if (version >= 5) List("classic", "consumer").asJava else List.empty.asJava) val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) + when(groupCoordinator.listGroups( + requestChannelRequest.context, + expectedListGroupsRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -8599,7 +8757,10 @@ class KafkaApisTest extends Logging { .setTypesFilter(List("classic", "consumer").asJava) val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) + when(groupCoordinator.listGroups( + requestChannelRequest.context, + expectedListGroupsRequest + )).thenReturn(future) kafkaApis = createKafkaApis() kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -8692,7 +8853,10 @@ class KafkaApisTest extends Logging { val expectedListGroupsRequest = new ListGroupsRequestData() val future = new CompletableFuture[ListGroupsResponseData]() - when(groupCoordinator.listGroups(requestChannelRequest.context, expectedListGroupsRequest)).thenReturn(future) + when(groupCoordinator.listGroups( + requestChannelRequest.context, + expectedListGroupsRequest + )).thenReturn(future) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleListGroupsRequest(requestChannelRequest) @@ -9506,7 +9670,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) val future = new CompletableFuture[ConsumerGroupHeartbeatResponseData]() - when(groupCoordinator.consumerGroupHeartbeat(requestChannelRequest.context, consumerGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.consumerGroupHeartbeat( + requestChannelRequest.context, + consumerGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -9529,7 +9696,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupHeartbeatRequest.Builder(consumerGroupHeartbeatRequest).build()) val future = new CompletableFuture[ConsumerGroupHeartbeatResponseData]() - when(groupCoordinator.consumerGroupHeartbeat(requestChannelRequest.context, consumerGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.consumerGroupHeartbeat( + requestChannelRequest.context, + consumerGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -9634,7 +9804,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -9696,7 +9869,10 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -9720,7 +9896,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -9887,7 +10066,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -9913,7 +10095,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[StreamsGroupHeartbeatResult]() - when(groupCoordinator.streamsGroupHeartbeat(requestChannelRequest.context, streamsGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.streamsGroupHeartbeat( + requestChannelRequest.context, + streamsGroupHeartbeatRequest + )).thenReturn(future) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], isNotNull[util.List[Action]])).thenAnswer(invocation => { @@ -9966,7 +10151,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -10080,7 +10268,10 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), @@ -10101,7 +10292,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ConsumerGroupDescribeRequest.Builder(consumerGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( featureVersions = Seq(GroupVersion.GV_1) ) @@ -10126,7 +10320,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupDescribeRequest.Builder(streamsGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -10231,7 +10428,10 @@ class KafkaApisTest extends Logging { .thenReturn(Seq(AuthorizationResult.DENIED).asJava) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) future.complete(List().asJava) kafkaApis = createKafkaApis( authorizer = Some(authorizer), @@ -10252,7 +10452,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new StreamsGroupDescribeRequest.Builder(streamsGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") ) @@ -10297,7 +10500,10 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[StreamsGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.streamsGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.streamsGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), overrideProperties = Map(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG -> "classic,streams") @@ -10374,7 +10580,10 @@ class KafkaApisTest extends Logging { } val future = new CompletableFuture[util.List[ConsumerGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.consumerGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.consumerGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) kafkaApis = createKafkaApis( authorizer = Some(authorizer), featureVersions = Seq(GroupVersion.GV_1) @@ -10581,7 +10790,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() - when(groupCoordinator.shareGroupHeartbeat(requestChannelRequest.context, shareGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.shareGroupHeartbeat( + requestChannelRequest.context, + shareGroupHeartbeatRequest + )).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -10663,7 +10875,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() - when(groupCoordinator.shareGroupHeartbeat(requestChannelRequest.context, shareGroupHeartbeatRequest)).thenReturn(future) + when(groupCoordinator.shareGroupHeartbeat( + requestChannelRequest.context, + shareGroupHeartbeatRequest + )).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11137,9 +11352,15 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup1)).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupOffsets( + requestChannelRequest.context, + describeShareGroupOffsetsRequestGroup1 + )).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2)).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupOffsets( + requestChannelRequest.context, + describeShareGroupOffsetsRequestGroup2 + )).thenReturn(futureGroup2) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -11248,17 +11469,23 @@ class KafkaApisTest extends Logging { // The group coordinator will only be asked for information about topics which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics( - util.List.of( - new DescribeShareGroupOffsetsRequestTopic().setTopicName(topicName1).setPartitions(util.List.of(1, 2, 3)), + when(groupCoordinator.describeShareGroupOffsets( + requestChannelRequest.context, + new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics( + util.List.of( + new DescribeShareGroupOffsetsRequestTopic().setTopicName(topicName1).setPartitions(util.List.of(1, 2, 3)), + ) ) - ))).thenReturn(futureGroup1) + )).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics( - util.List.of( + when(groupCoordinator.describeShareGroupOffsets( + requestChannelRequest.context, + new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics( + util.List.of( + ) ) - ))).thenReturn(futureGroup2) + )).thenReturn(futureGroup2) val authorizer: Authorizer = mock(classOf[Authorizer]) val acls = Map( @@ -11411,10 +11638,16 @@ class KafkaApisTest extends Logging { // The group coordinator is being asked for information about all topics, not just those which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics(null))).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupAllOffsets( + requestChannelRequest.context, + new DescribeShareGroupOffsetsRequestGroup().setGroupId("group1").setTopics(null) + )).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics(null))).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupAllOffsets( + requestChannelRequest.context, + new DescribeShareGroupOffsetsRequestGroup().setGroupId("group2").setTopics(null) + )).thenReturn(futureGroup2) val authorizer: Authorizer = mock(classOf[Authorizer]) val acls = Map( @@ -11566,9 +11799,15 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup1)).thenReturn(futureGroup1) + when(groupCoordinator.describeShareGroupAllOffsets( + requestChannelRequest.context, + describeShareGroupOffsetsRequestGroup1 + )).thenReturn(futureGroup1) val futureGroup2 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupAllOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2)).thenReturn(futureGroup2) + when(groupCoordinator.describeShareGroupAllOffsets( + requestChannelRequest.context, + describeShareGroupOffsetsRequestGroup2 + )).thenReturn(futureGroup2) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -11678,7 +11917,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - when(groupCoordinator.describeShareGroupOffsets(requestChannelRequest.context, describeShareGroupOffsetsRequestGroup)).thenReturn(future) + when(groupCoordinator.describeShareGroupOffsets( + requestChannelRequest.context, + describeShareGroupOffsetsRequestGroup + )).thenReturn(future) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) @@ -11776,7 +12018,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsGroupCoordinatorRequestData)).thenReturn(resultFuture) + when(groupCoordinator.deleteShareGroupOffsets( + requestChannelRequest.context, + deleteShareGroupOffsetsGroupCoordinatorRequestData + )).thenReturn(resultFuture) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActionsGroup(groupId)))) @@ -11874,7 +12119,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(resultFuture) + when(groupCoordinator.deleteShareGroupOffsets( + requestChannelRequest.context, + deleteShareGroupOffsetsRequestData + )).thenReturn(resultFuture) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11955,7 +12203,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) - when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) + when(groupCoordinator.deleteShareGroupOffsets( + requestChannelRequest.context, + deleteShareGroupOffsetsRequestData + )).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -11998,7 +12249,10 @@ class KafkaApisTest extends Logging { .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) .setErrorMessage(Errors.UNKNOWN_SERVER_ERROR.message()) - when(groupCoordinator.deleteShareGroupOffsets(requestChannelRequest.context, deleteShareGroupOffsetsRequestData)).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) + when(groupCoordinator.deleteShareGroupOffsets( + requestChannelRequest.context, + deleteShareGroupOffsetsRequestData + )).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) kafkaApis = createKafkaApis( overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), @@ -12335,7 +12589,10 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData, true).build()) val future = new CompletableFuture[util.List[ShareGroupDescribeResponseData.DescribedGroup]]() - when(groupCoordinator.shareGroupDescribe(any[RequestContext], any[util.List[String]])).thenReturn(future) + when(groupCoordinator.shareGroupDescribe( + any[RequestContext], + any[util.List[String]] + )).thenReturn(future) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, From 921bc7d52f63f30c8fd32f0b455464f20ad65046 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 16 Apr 2025 13:50:40 +0200 Subject: [PATCH 5/5] revert unneeded change --- core/src/main/scala/kafka/server/KafkaApis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 70d2bd6627d4f..6a22963ac7d6a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3672,7 +3672,7 @@ class KafkaApis(val requestChannel: RequestChannel, groupCoordinator.deleteShareGroupOffsets( request.context, new DeleteShareGroupOffsetsRequestData().setGroupId(groupId).setTopics(authorizedTopics) - ).handle[Unit] { (responseData, exception) => { + ).handle[Unit] {(responseData, exception) => { if (exception != null) { requestHelper.sendMaybeThrottle(request, deleteShareGroupOffsetsRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, exception)) } else if (responseData.errorCode() != Errors.NONE.code) {