cmccabe commented on a change in pull request #10184:
URL: https://github.com/apache/kafka/pull/10184#discussion_r583962844



##########
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())

Review comment:
       It's certainly awkward that for topic IDs, we need to check existence 
first (since otherwise we have nothing to give to the authorizer) but for topic 
names, we check authorization first. But you're right, this is a leak. I'll fix 
it




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to