jolshan commented on a change in pull request #10184:
URL: https://github.com/apache/kafka/pull/10184#discussion_r585895209
##########
File path: core/src/main/scala/kafka/server/ControllerApis.scala
##########
@@ -154,6 +161,147 @@ class ControllerApis(val requestChannel: RequestChannel,
requestThrottleMs => createResponseCallback(requestThrottleMs))
}
+ def handleDeleteTopics(request: RequestChannel.Request): Unit = {
+ val responses = deleteTopics(request.body[DeleteTopicsRequest].data(),
+ request.context.apiVersion(),
+ authHelper.authorize(request.context, DELETE, CLUSTER, CLUSTER_NAME),
+ names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC,
names)(n => n),
+ names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC,
names)(n => n))
+ requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => {
+ val responseData = new DeleteTopicsResponseData().
+ setResponses(new DeletableTopicResultCollection(responses.iterator())).
+ setThrottleTimeMs(throttleTimeMs)
+ new DeleteTopicsResponse(responseData)
+ })
+ }
+
+ def deleteTopics(request: DeleteTopicsRequestData,
+ apiVersion: Int,
+ hasClusterAuth: Boolean,
+ getDescribableTopics: Iterable[String] => Set[String],
+ getDeletableTopics: Iterable[String] => Set[String]):
util.List[DeletableTopicResult] = {
+ if (!config.deleteTopicEnable) {
+ if (apiVersion < 3) {
+ throw new InvalidRequestException("Topic deletion is disabled.")
+ } else {
+ throw new TopicDeletionDisabledException()
+ }
+ }
+ val responses = new util.ArrayList[DeletableTopicResult]
+ val duplicatedTopicNames = new util.HashSet[String]
+ val topicNamesToResolve = new util.HashSet[String]
+ val topicIdsToResolve = new util.HashSet[Uuid]
+ val duplicatedTopicIds = new util.HashSet[Uuid]
+
+ def appendResponse(name: String, id: Uuid, error: ApiError): Unit = {
+ responses.add(new DeletableTopicResult().
+ setName(name).
+ setTopicId(id).
+ setErrorCode(error.error().code()).
+ setErrorMessage(error.message()))
+ }
+
+ def maybeAppendToTopicNamesToResolve(name: String): Unit = {
+ if (duplicatedTopicNames.contains(name) ||
!topicNamesToResolve.add(name)) {
+ appendResponse(name, ZERO_UUID, new ApiError(INVALID_REQUEST,
"Duplicate topic name."))
+ topicNamesToResolve.remove(name)
+ duplicatedTopicNames.add(name)
+ }
+ }
+
+ def maybeAppendToIdsToResolve(id: Uuid): Unit = {
+ if (duplicatedTopicIds.contains(id) || !topicIdsToResolve.add(id)) {
+ appendResponse(null, id, new ApiError(INVALID_REQUEST, "Duplicate
topic ID."))
+ topicIdsToResolve.remove(id)
+ duplicatedTopicIds.add(id)
+ }
+ }
+
+
request.topicNames().iterator().asScala.foreach(maybeAppendToTopicNamesToResolve)
+
+ request.topics().iterator().asScala.foreach {
+ topic => if (topic.name() == null) {
+ if (topic.topicId.equals(ZERO_UUID)) {
+ appendResponse(null, ZERO_UUID, new ApiError(INVALID_REQUEST,
+ "Neither topic name nor id were specified."))
+ } else {
+ maybeAppendToIdsToResolve(topic.topicId())
+ }
+ } else {
+ if (topic.topicId().equals(ZERO_UUID)) {
+ maybeAppendToTopicNamesToResolve(topic.name())
+ } else {
+ appendResponse(topic.name(), topic.topicId(), new
ApiError(INVALID_REQUEST,
+ "You may not specify both topic name and topic id."))
+ }
+ }
+ }
+
+ val idToName = new util.HashMap[Uuid, String]
+
+ def maybeAppendToIdToName(id: Uuid, name: String): Unit = {
+ if (duplicatedTopicIds.contains(id) || idToName.put(id, name) != null) {
+ appendResponse(name, id, new ApiError(INVALID_REQUEST,
+ "The same topic was specified by name and by id."))
+ idToName.remove(id)
+ duplicatedTopicIds.add(id)
+ }
+ }
+ controller.findTopicIds(topicNamesToResolve).get().asScala.foreach {
+ case (name, idOrError) => if (idOrError.isError) {
+ appendResponse(name, ZERO_UUID, idOrError.error())
+ } else {
+ maybeAppendToIdToName(idOrError.result(), name)
+ }
+ }
+ controller.findTopicNames(topicIdsToResolve).get().asScala.foreach {
+ case (id, nameOrError) => if (nameOrError.isError) {
+ appendResponse(null, id, nameOrError.error())
+ } else {
+ maybeAppendToIdToName(id, nameOrError.result())
+ }
+ }
+
+ if (!hasClusterAuth) {
+ val authorizedDescribeTopics =
getDescribableTopics(idToName.values().asScala)
+ val authorizedDeleteTopics =
getDeletableTopics(idToName.values().asScala)
+ val iterator = idToName.entrySet().iterator()
+ while (iterator.hasNext) {
+ val entry = iterator.next()
+ val topicName = entry.getValue
+ if (!authorizedDeleteTopics.contains(topicName)) {
+ val topicId = entry.getKey
+ if (!authorizedDescribeTopics.contains(topicName)) {
+ // If the user is not authorized to describe the topic, we pretend
that it
Review comment:
Oh sorry this just got changed due to
https://issues.apache.org/jira/browse/KAFKA-12394
So the case of no delete permission, no describe permission, topic ID
provided is now TOPIC_AUTHORIZATION_FAILED. This may have been what you had
initially.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]