chia7712 commented on code in PR #18465:
URL: https://github.com/apache/kafka/pull/18465#discussion_r1910495372


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -3483,19 +3456,7 @@ class KafkaApis(val requestChannel: RequestChannel,
   }
 
   def handleAllocateProducerIdsRequest(request: RequestChannel.Request): Unit 
= {
-    val zkSupport = 
metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request))
-    authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
-
-    val allocateProducerIdsRequest = request.body[AllocateProducerIdsRequest]
-
-    if (!zkSupport.controller.isActive)
-      requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs =>
-        allocateProducerIdsRequest.getErrorResponse(throttleTimeMs, 
Errors.NOT_CONTROLLER.exception))
-    else
-      
zkSupport.controller.allocateProducerIds(allocateProducerIdsRequest.data, 
producerIdsResponse =>
-        requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs =>
-          new 
AllocateProducerIdsResponse(producerIdsResponse.setThrottleTimeMs(throttleTimeMs)))
-      )
+    throw KafkaApis.shouldNeverReceive(request)

Review Comment:
   > So, we should have generic logic that handles that automatically before it 
gets here. We can probably just delete this whole method.
   
   totally agree that we should have a generic logic for those zk-related 
handler. The socket server can reject the zk-related requests automatically, 
and we can rewrite `maybeForwardToController` to throw 
`KafkaApis.shouldAlwaysForward` automatically. With those change, those 
zk-related handlers can be deleted directly.
   
   In short, the line `case ApiKeys.ALLOCATE_PRODUCER_IDS => 
handleAllocateProducerIdsRequest(request)` can be removed.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to