Skip to content

KAFKA-19413: Extended AuthorizerIntegrationTest to cover StreamsGroupDescribe #19981

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 1 commit into from
Jun 18, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,12 @@ import java.time.Duration
import java.util
import java.util.concurrent.{ExecutionException, Semaphore}
import java.util.regex.Pattern
import java.util.{Comparator, Optional, Properties}
import java.util.{Comparator, Optional, Properties, UUID}
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.waitUntilTrue
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ListGroupsOptions, NewTopic}
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.consumer.internals.{StreamsRebalanceData, StreamsRebalanceListener}
import org.apache.kafka.clients.producer._
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
Expand All @@ -37,7 +38,7 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, AlterShareGroupOffsetsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, AlterShareGroupOffsetsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, StreamsGroupDescribeRequestData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
Expand Down Expand Up @@ -76,6 +77,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
val shareGroupDescribeConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW)))
val shareGroupAlterConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW)))
val streamsGroupReadAcl = Map(streamsGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)))
val streamsGroupDescribeAcl = Map(streamsGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)))
val clusterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW)))
val clusterCreateAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)))
val clusterAlterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER, ALLOW)))
Expand Down Expand Up @@ -225,7 +227,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
resp.data.errorCode)),
ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> ((resp: AlterShareGroupOffsetsResponse) => Errors.forCode(
resp.data.errorCode)),
ApiKeys.STREAMS_GROUP_HEARTBEAT -> ((resp: StreamsGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode))
ApiKeys.STREAMS_GROUP_HEARTBEAT -> ((resp: StreamsGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.STREAMS_GROUP_DESCRIBE -> ((resp: StreamsGroupDescribeResponse) =>
Errors.forCode(resp.data.groups.asScala.find(g => streamsGroup == g.groupId).head.errorCode))
)

def findErrorForTopicId(id: Uuid, response: AbstractResponse): Errors = {
Expand Down Expand Up @@ -294,7 +298,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> (shareGroupDescribeAcl ++ topicDescribeAcl),
ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> (shareGroupDeleteAcl ++ topicReadAcl),
ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> (shareGroupReadAcl ++ topicReadAcl),
ApiKeys.STREAMS_GROUP_HEARTBEAT -> (streamsGroupReadAcl ++ topicDescribeAcl)
ApiKeys.STREAMS_GROUP_HEARTBEAT -> (streamsGroupReadAcl ++ topicDescribeAcl),
ApiKeys.STREAMS_GROUP_DESCRIBE -> (streamsGroupDescribeAcl ++ topicDescribeAcl),
)

private def createMetadataRequest(allowAutoTopicCreation: Boolean) = {
Expand Down Expand Up @@ -870,6 +875,11 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
).asJava
))).build(ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion)

private def streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
new StreamsGroupDescribeRequestData()
.setGroupIds(List(streamsGroup).asJava)
.setIncludeAuthorizedOperations(false)).build(ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion)

private def sendRequests(requestKeyToRequest: mutable.Map[ApiKeys, AbstractRequest], topicExists: Boolean = true,
topicNames: Map[Uuid, String] = getTopicNames()) = {
for ((key, request) <- requestKeyToRequest) {
Expand Down Expand Up @@ -954,6 +964,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> deleteShareGroupOffsetsRequest,
ApiKeys.ALTER_SHARE_GROUP_OFFSETS -> alterShareGroupOffsetsRequest,
ApiKeys.STREAMS_GROUP_HEARTBEAT -> streamsGroupHeartbeatRequest,
ApiKeys.STREAMS_GROUP_DESCRIBE -> streamsGroupDescribeRequest,

// Delete the topic last
ApiKeys.DELETE_TOPICS -> deleteTopicsRequest
Expand Down Expand Up @@ -987,7 +998,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
ApiKeys.SHARE_FETCH -> createShareFetchRequest,
ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest,
ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest,
ApiKeys.STREAMS_GROUP_HEARTBEAT -> streamsGroupHeartbeatRequest
ApiKeys.STREAMS_GROUP_HEARTBEAT -> streamsGroupHeartbeatRequest,
ApiKeys.STREAMS_GROUP_DESCRIBE -> streamsGroupDescribeRequest
)

sendRequests(requestKeyToRequest, topicExists = false, topicNames)
Expand Down Expand Up @@ -3853,6 +3865,165 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
response.data().status())
}

private def createStreamsGroupToDescribe(
topicAsSourceTopic: Boolean,
topicAsRepartitionSinkTopic: Boolean,
topicAsRepartitionSourceTopic: Boolean,
topicAsStateChangelogTopics: Boolean
): Unit = {
createTopicWithBrokerPrincipal(sourceTopic)
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), streamsGroupResource)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), sourceTopicResource)
streamsConsumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, streamsGroup)
streamsConsumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
val consumer = createStreamsConsumer(streamsRebalanceData = new StreamsRebalanceData(
UUID.randomUUID(),
Optional.empty(),
util.Map.of(
"subtopology-0", new StreamsRebalanceData.Subtopology(
if (topicAsSourceTopic) util.Set.of(sourceTopic, topic) else util.Set.of(sourceTopic),
if (topicAsRepartitionSinkTopic) util.Set.of(topic) else util.Set.of(),
if (topicAsRepartitionSourceTopic)
util.Map.of(topic, new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.empty(), util.Map.of()))
else util.Map.of(),
if (topicAsStateChangelogTopics)
util.Map.of(topic, new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.empty(), util.Map.of()))
else util.Map.of(),
util.Set.of()
)),
Map.empty[String, String].asJava
))
consumer.subscribe(
if (topicAsSourceTopic || topicAsRepartitionSourceTopic) util.Set.of(sourceTopic, topic) else util.Set.of(sourceTopic),
new StreamsRebalanceListener {
override def onTasksRevoked(tasks: util.Set[StreamsRebalanceData.TaskId]): Optional[Exception] =
Optional.empty()

override def onTasksAssigned(assignment: StreamsRebalanceData.Assignment): Optional[Exception] =
Optional.empty()

override def onAllTasksLost(): Optional[Exception] =
Optional.empty()
}
)
consumer.poll(Duration.ofMillis(500L))
removeAllClientAcls()
}

@ParameterizedTest
@CsvSource(Array(
"true, false, false, false",
"false, true, false, false",
"false, false, true, false",
"false, false, false, true"
))
def testStreamsGroupDescribeWithGroupDescribeAndTopicDescribeAcl(
topicAsSourceTopic: Boolean,
topicAsRepartitionSinkTopic: Boolean,
topicAsRepartitionSourceTopic: Boolean,
topicAsStateChangelogTopics: Boolean
): Unit = {
createStreamsGroupToDescribe(
topicAsSourceTopic,
topicAsRepartitionSinkTopic,
topicAsRepartitionSourceTopic,
topicAsStateChangelogTopics
)
addAndVerifyAcls(streamsGroupDescribeAcl(streamsGroupResource), streamsGroupResource)
addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)

val request = streamsGroupDescribeRequest
val resource = Set[ResourceType](GROUP, TOPIC)
sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
}

@ParameterizedTest
@CsvSource(Array(
"true, false, false, false",
"false, true, false, false",
"false, false, true, false",
"false, false, false, true"
))
def testStreamsGroupDescribeWithOperationAll(
topicAsSourceTopic: Boolean,
topicAsRepartitionSinkTopic: Boolean,
topicAsRepartitionSourceTopic: Boolean,
topicAsStateChangelogTopics: Boolean
): Unit = {
createStreamsGroupToDescribe(
topicAsSourceTopic,
topicAsRepartitionSinkTopic,
topicAsRepartitionSourceTopic,
topicAsStateChangelogTopics
)

val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW)
addAndVerifyAcls(Set(allowAllOpsAcl), streamsGroupResource)
addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
addAndVerifyAcls(Set(allowAllOpsAcl), topicResource)

val request = streamsGroupDescribeRequest
val resource = Set[ResourceType](GROUP, TOPIC)
sendRequestAndVerifyResponseError(request, resource, isAuthorized = true)
}

@ParameterizedTest
@CsvSource(Array(
"true, false, false, false",
"false, true, false, false",
"false, false, true, false",
"false, false, false, true"
))
def testStreamsGroupDescribeWithoutGroupDescribeAcl(
topicAsSourceTopic: Boolean,
topicAsRepartitionSinkTopic: Boolean,
topicAsRepartitionSourceTopic: Boolean,
topicAsStateChangelogTopics: Boolean
): Unit = {
createStreamsGroupToDescribe(
topicAsSourceTopic,
topicAsRepartitionSinkTopic,
topicAsRepartitionSourceTopic,
topicAsStateChangelogTopics
)
addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic
addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource)

val request = streamsGroupDescribeRequest
val resource = Set[ResourceType](GROUP, TOPIC)
sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
}

@ParameterizedTest
@CsvSource(Array(
"true, false, false, false",
"false, true, false, false",
"false, false, true, false",
"false, false, false, true"
))
def testStreamsGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(
topicAsSourceTopic: Boolean,
topicAsRepartitionSinkTopic: Boolean,
topicAsRepartitionSourceTopic: Boolean,
topicAsStateChangelogTopics: Boolean
): Unit = {
createStreamsGroupToDescribe(
topicAsSourceTopic,
topicAsRepartitionSinkTopic,
topicAsRepartitionSourceTopic,
topicAsStateChangelogTopics
)

val request = streamsGroupDescribeRequest
val resource = Set[ResourceType](GROUP, TOPIC)
addAndVerifyAcls(sourceTopicDescribeAcl(sourceTopicResource), sourceTopicResource) // Always added, since we need a source topic

sendRequestAndVerifyResponseError(request, resource, isAuthorized = false)
}

private def sendAndReceiveFirstRegexHeartbeat(memberId: String,
listenerName: ListenerName): ConsumerGroupHeartbeatResponseData = {
val request = new ConsumerGroupHeartbeatRequest.Builder(
Expand Down
Loading