-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-3396 : Unauthorized topics are returned to the user #1428
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
Changes from all commits
6b68837
cce5726
c7aefe2
cf464fa
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -234,38 +234,37 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| val responseBody = new OffsetCommitResponse(results.asJava) | ||
| requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) | ||
| } else { | ||
| // filter non-existent topics | ||
| val invalidRequestsInfo = offsetCommitRequest.offsetData.asScala.filter { case (topicPartition, _) => | ||
| !metadataCache.contains(topicPartition.topic) | ||
|
|
||
| val (existingOrAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = offsetCommitRequest.offsetData.asScala.toMap.partition { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The first variable should be |
||
| case (topicPartition, _) => metadataCache.contains(topicPartition.topic) && authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: not much difference here, but it seems more natural to check authorization first.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok |
||
| } | ||
| val filteredRequestInfo = offsetCommitRequest.offsetData.asScala.toMap -- invalidRequestsInfo.keys | ||
|
|
||
| val (authorizedRequestInfo, unauthorizedRequestInfo) = filteredRequestInfo.partition { | ||
| case (topicPartition, offsetMetadata) => authorize(request.session, Read, new Resource(auth.Topic, topicPartition.topic)) | ||
| val (authorizedTopics, unauthorizedForReadTopics) = existingOrAuthorizedForDescribeTopics.partition { | ||
| case (topicPartition, _) => authorize(request.session, Read, new Resource(auth.Topic, topicPartition.topic)) | ||
| } | ||
|
|
||
| // the callback for sending an offset commit response | ||
| def sendResponseCallback(commitStatus: immutable.Map[TopicPartition, Short]) { | ||
| val mergedCommitStatus = commitStatus ++ unauthorizedRequestInfo.mapValues(_ => Errors.TOPIC_AUTHORIZATION_FAILED.code) | ||
| var combinedCommitStatus = commitStatus.mapValues(new JShort(_)) ++ | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should be a val. |
||
| unauthorizedForReadTopics.mapValues(_ => new JShort(Errors.TOPIC_AUTHORIZATION_FAILED.code)) ++ | ||
| nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new JShort(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)) | ||
|
|
||
| mergedCommitStatus.foreach { case (topicPartition, errorCode) => | ||
| combinedCommitStatus.foreach { case (topicPartition, errorCode) => | ||
| if (errorCode != Errors.NONE.code) { | ||
| debug(s"Offset commit request with correlation id ${header.correlationId} from client ${header.clientId} " + | ||
| s"on partition $topicPartition failed due to ${Errors.forCode(errorCode).exceptionName}") | ||
|
||
| } | ||
| } | ||
| val combinedCommitStatus = mergedCommitStatus.mapValues(new JShort(_)) ++ invalidRequestsInfo.map(_._1 -> new JShort(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)) | ||
|
|
||
| val responseHeader = new ResponseHeader(header.correlationId) | ||
| val responseBody = new OffsetCommitResponse(combinedCommitStatus.asJava) | ||
| requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) | ||
| } | ||
|
|
||
| if (authorizedRequestInfo.isEmpty) | ||
| if (authorizedTopics.isEmpty) | ||
| sendResponseCallback(Map.empty) | ||
| else if (header.apiVersion == 0) { | ||
| // for version 0 always store offsets to ZK | ||
| val responseInfo = authorizedRequestInfo.map { | ||
| val responseInfo = authorizedTopics.map { | ||
| case (topicPartition, partitionData) => | ||
| val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicPartition.topic) | ||
| try { | ||
|
|
@@ -302,7 +301,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| // - If v2 we use the default expiration timestamp | ||
| val currentTimestamp = SystemTime.milliseconds | ||
| val defaultExpireTimestamp = offsetRetention + currentTimestamp | ||
| val partitionData = authorizedRequestInfo.mapValues { partitionData => | ||
| val partitionData = authorizedTopics.mapValues { partitionData => | ||
| val metadata = if (partitionData.metadata == null) OffsetMetadata.NoMetadata else partitionData.metadata | ||
| new OffsetAndMetadata( | ||
| offsetMetadata = OffsetMetadata(partitionData.offset, metadata), | ||
|
|
@@ -555,7 +554,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| } | ||
|
|
||
| val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => | ||
| new PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED.code, List[JLong]().asJava) | ||
| new PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, List[JLong]().asJava) | ||
| ) | ||
|
|
||
| val responseMap = authorizedRequestInfo.map({case (topicPartition, partitionData) => | ||
|
|
@@ -606,7 +605,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| } | ||
|
|
||
| val unauthorizedResponseStatus = unauthorizedRequestInfo.mapValues(_ => { | ||
| new PartitionData(Errors.TOPIC_AUTHORIZATION_FAILED.code, | ||
| new PartitionData(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, | ||
| ListOffsetResponse.UNKNOWN_TIMESTAMP, | ||
| ListOffsetResponse.UNKNOWN_OFFSET) | ||
| }) | ||
|
|
@@ -782,7 +781,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| } else if (config.autoCreateTopicsEnable) { | ||
| createTopic(topic, config.numPartitions, config.defaultReplicationFactor) | ||
| } else { | ||
| new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, common.Topic.isInternal(topic), | ||
| new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, false, | ||
| java.util.Collections.emptyList()) | ||
| } | ||
| } | ||
|
|
@@ -811,25 +810,36 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| metadataRequest.topics.asScala.toSet | ||
| } | ||
|
|
||
| var (authorizedTopics, unauthorizedTopics) = | ||
| var (authorizedTopics, unauthorizedForDescribeTopics) = | ||
| topics.partition(topic => authorize(request.session, Describe, new Resource(auth.Topic, topic))) | ||
|
|
||
| var unauthorizedForCreateTopics = Set[String]() | ||
|
|
||
| if (authorizedTopics.nonEmpty) { | ||
| val nonExistingTopics = metadataCache.getNonExistingTopics(authorizedTopics) | ||
| if (config.autoCreateTopicsEnable && nonExistingTopics.nonEmpty) { | ||
| authorizer.foreach { az => | ||
|
||
| if (!az.authorize(request.session, Create, Resource.ClusterResource)) { | ||
| authorizedTopics --= nonExistingTopics | ||
| unauthorizedTopics ++= nonExistingTopics | ||
| unauthorizedForCreateTopics ++= nonExistingTopics | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| val unauthorizedTopicMetadata = unauthorizedTopics.map(topic => | ||
| new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, Topic.isInternal(topic), | ||
| val unauthorizedForCreateTopicMetadata = unauthorizedForCreateTopics.map(topic => | ||
| new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, common.Topic.isInternal(topic), | ||
| java.util.Collections.emptyList())) | ||
|
|
||
| // do not disclose the existence of unauthorized topics | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: should be obvious, but we may as well emphasize unauthorized for Describe.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thanks |
||
| val unauthorizedForDescribeTopicMetadata = | ||
| // In case of all topics, don't include unauthorized topics | ||
| if ((requestVersion == 0 && (metadataRequest.topics() == null || metadataRequest.topics().isEmpty)) || (metadataRequest.isAllTopics)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Doesn't seem like the parenthesis around |
||
| Set.empty[MetadataResponse.TopicMetadata] | ||
| else | ||
| unauthorizedForDescribeTopics.map(topic => | ||
| new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, false, java.util.Collections.emptyList())) | ||
|
|
||
| // In version 0, we returned an error when brokers with replicas were unavailable, | ||
| // while in higher versions we simply don't include the broker in the returned broker list | ||
| val errorUnavailableEndpoints = requestVersion == 0 | ||
|
|
@@ -839,7 +849,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| else | ||
| getTopicMetadata(authorizedTopics, request.securityProtocol, errorUnavailableEndpoints) | ||
|
|
||
| val completeTopicMetadata = topicMetadata ++ unauthorizedTopicMetadata | ||
| val completeTopicMetadata = topicMetadata ++ unauthorizedForCreateTopicMetadata ++ unauthorizedForDescribeTopicMetadata | ||
|
|
||
| val brokers = metadataCache.getAliveBrokers | ||
|
|
||
|
|
@@ -876,9 +886,8 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| val (authorizedTopicPartitions, unauthorizedTopicPartitions) = offsetFetchRequest.partitions.asScala.partition { topicPartition => | ||
| authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) | ||
| } | ||
| val unauthorizedTopicResponse = new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.TOPIC_AUTHORIZATION_FAILED.code) | ||
| val unauthorizedStatus = unauthorizedTopicPartitions.map(topicPartition => (topicPartition, unauthorizedTopicResponse)).toMap | ||
| val unknownTopicPartitionResponse = new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.UNKNOWN_TOPIC_OR_PARTITION.code) | ||
| val unauthorizedStatus = unauthorizedTopicPartitions.map(topicPartition => (topicPartition, unknownTopicPartitionResponse)).toMap | ||
|
|
||
| if (header.apiVersion == 0) { | ||
| // version 0 reads offsets from ZK | ||
|
|
@@ -1203,7 +1212,7 @@ class KafkaApis(val requestChannel: RequestChannel, | |
| }.toMap | ||
| sendResponseCallback(results) | ||
| } else { | ||
| // If no authorized topics return immediatly | ||
| // If no authorized topics return immediately | ||
| if (authorizedTopics.isEmpty) | ||
| sendResponseCallback(Map()) | ||
| else { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: unneeded newline?