You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/07/07 20:54:53 UTC

[GitHub] [kafka] hachikuji commented on a change in pull request #10794: KAFKA-12677: return not_controller error in envelope response itself

hachikuji commented on a change in pull request #10794:
URL: https://github.com/apache/kafka/pull/10794#discussion_r665698647



##########
File path: clients/src/main/java/org/apache/kafka/common/requests/ApiError.java
##########
@@ -36,10 +37,16 @@
     private final String message;
 
     public static ApiError fromThrowable(Throwable t) {
+        Throwable throwableToBeEncode = t;
+        // Future will wrap the original exception with completionException, which will return unexpected UNKNOWN_SERVER_ERROR.

Review comment:
       Hmm, a little strange to handle this case explicitly and not others. Would we want to handle `ExecutionException` as well for example? 
   
   Also, the comment seems to be missing some context. Which future does it refer to? Maybe it would be helpful to point to the specific usage that requires this.

##########
File path: core/src/main/scala/kafka/network/RequestChannel.scala
##########
@@ -124,8 +124,15 @@ object RequestChannel extends Logging {
     def buildResponseSend(abstractResponse: AbstractResponse): Send = {
       envelope match {
         case Some(request) =>
-          val responseBytes = context.buildResponseEnvelopePayload(abstractResponse)
-          val envelopeResponse = new EnvelopeResponse(responseBytes, Errors.NONE)
+          val envelopeResponse = if (abstractResponse.errorCounts().containsKey(Errors.NOT_CONTROLLER)) {
+            // Since it's Not Controller error response, we need to make envelope response with Not Controller error
+            // to notify the requester (i.e. BrokerToControllerRequestThread) to update active controller
+            new EnvelopeResponse(new EnvelopeResponseData()
+              .setErrorCode(Errors.NOT_CONTROLLER.code()))

Review comment:
       Ok, I think this makes sense. Basically we were missing the same logic we have in `KafkaApis.handleEnvelope` to verify the controller status. Instead, we just send the request into the controller thread. Do I have that right? I guess another way we could do it is to send the full envelope into the controller and let it set the NOT_CONTROLLER error at the right level, but this solution also seems reasonable.




-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org