dengziming commented on code in PR #12185:
URL: https://github.com/apache/kafka/pull/12185#discussion_r879103834


##########
core/src/test/scala/unit/kafka/utils/TestUtils.scala:
##########
@@ -2195,22 +2193,18 @@ object TestUtils extends Logging {
 
     RequestHeader.parse(envelopeBuffer)
 
-    var requestContext = new RequestContext(envelopeHeader, "1", 
InetAddress.getLocalHost,
+    val envelopeContext = new RequestContext(envelopeHeader, "1", 
InetAddress.getLocalHost,
       KafkaPrincipal.ANONYMOUS, listenerName, SecurityProtocol.PLAINTEXT, 
ClientInformation.EMPTY,
       fromPrivilegedListener, Optional.of(principalSerde))
 
-    if (shouldSpyRequestContext) {
-      requestContext = Mockito.spy(requestContext)
-    }
-
     new RequestChannel.Request(
       processor = 1,
-      context = requestContext,
+      context = envelopeContext,
       startTimeNanos = startTimeNanos,
       memoryPool = MemoryPool.NONE,
       buffer = envelopeBuffer,
       metrics = requestChannelMetrics,
-      envelope = envelope

Review Comment:
   This is the source of the problem, envelope of the wrapped request should be 
None, right?



##########
core/src/test/scala/unit/kafka/network/RequestChannelTest.scala:
##########
@@ -191,84 +194,66 @@ class RequestChannelTest {
     
assertTrue(isValidJson(RequestConvertToJson.request(alterConfigs.loggableRequest).toString))
   }
 
-  @Test
-  def 
testEnvelopeBuildResponseSendShouldReturnNoErrorIfInnerResponseHasNoError(): 
Unit = {
-    val channelRequest = 
buildForwardRequestWithEnvelopeRequestAttached(buildMetadataRequest())
-
-    val envelopeResponseArgumentCaptor = 
ArgumentCaptor.forClass(classOf[EnvelopeResponse])
-
-    Mockito.doAnswer(_ => mockSend)
-      
.when(channelRequest.envelope.get.context).buildResponseSend(envelopeResponseArgumentCaptor.capture())
-
-    // create an inner response without error
-    val responseWithoutError = RequestTestUtils.metadataUpdateWith(2, 
Collections.singletonMap("a", 2))
-
-    // build an envelope response
-    channelRequest.buildResponseSend(responseWithoutError)
-
-    // expect the envelopeResponse result without error
-    val capturedValue: EnvelopeResponse = 
envelopeResponseArgumentCaptor.getValue
-    assertTrue(capturedValue.error().equals(Errors.NONE))
+  @ParameterizedTest
+  @EnumSource(value=classOf[Errors], names=Array("NONE", 
"CLUSTER_AUTHORIZATION_FAILED", "NOT_CONTROLLER"))
+  def testBuildEnvelopeResponse(error: Errors): Unit = {
+    val topic = "foo"
+    val createTopicRequest = buildCreateTopicRequest(topic)
+    val unwrapped = buildUnwrappedEnvelopeRequest(createTopicRequest)
+
+    val createTopicResponse = buildCreateTopicResponse(topic, error)
+    val envelopeResponse = buildEnvelopeResponse(unwrapped, 
createTopicResponse)
+
+    error match {
+      case Errors.NOT_CONTROLLER =>
+        assertEquals(Errors.NOT_CONTROLLER, envelopeResponse.error)
+        assertNull(envelopeResponse.responseData)
+      case _ =>
+        assertEquals(Errors.NONE, envelopeResponse.error)
+        val unwrappedResponse = 
AbstractResponse.parseResponse(envelopeResponse.responseData(), 
unwrapped.header)
+        assertEquals(createTopicResponse.data, unwrappedResponse.data)
+    }
   }
 
-  @Test
-  def 
testEnvelopeBuildResponseSendShouldReturnNoErrorIfInnerResponseHasNoNotControllerError():
 Unit = {
-    val channelRequest = 
buildForwardRequestWithEnvelopeRequestAttached(buildMetadataRequest())
-
-    val envelopeResponseArgumentCaptor = 
ArgumentCaptor.forClass(classOf[EnvelopeResponse])
-
-    Mockito.doAnswer(_ => mockSend)
-      
.when(channelRequest.envelope.get.context).buildResponseSend(envelopeResponseArgumentCaptor.capture())
-
-    // create an inner response with REQUEST_TIMED_OUT error
-    val responseWithTimeoutError = 
RequestTestUtils.metadataUpdateWith("cluster1", 2,
-      Collections.singletonMap("a", Errors.REQUEST_TIMED_OUT),
-      Collections.singletonMap("a", 2))
-
-    // build an envelope response
-    channelRequest.buildResponseSend(responseWithTimeoutError)
-
-    // expect the envelopeResponse result without error
-    val capturedValue: EnvelopeResponse = 
envelopeResponseArgumentCaptor.getValue
-    assertTrue(capturedValue.error().equals(Errors.NONE))
+  private def buildCreateTopicRequest(topic: String): CreateTopicsRequest = {
+    val requestData = new CreateTopicsRequestData()
+    requestData.topics.add(new CreatableTopic()
+      .setName(topic)
+      .setReplicationFactor(-1)
+      .setNumPartitions(-1)
+    )
+    new CreateTopicsRequest.Builder(requestData).build()
   }
 
-  @Test
-  def 
testEnvelopeBuildResponseSendShouldReturnNotControllerErrorIfInnerResponseHasOne():
 Unit = {
-    val channelRequest = 
buildForwardRequestWithEnvelopeRequestAttached(buildMetadataRequest())
-
-    val envelopeResponseArgumentCaptor = 
ArgumentCaptor.forClass(classOf[EnvelopeResponse])
-
-    Mockito.doAnswer(_ => mockSend)
-      
.when(channelRequest.envelope.get.context).buildResponseSend(envelopeResponseArgumentCaptor.capture())
-
-    // create an inner response with NOT_CONTROLLER error
-    val responseWithNotControllerError = 
RequestTestUtils.metadataUpdateWith("cluster1", 2,
-      Collections.singletonMap("a", Errors.NOT_CONTROLLER),
-      Collections.singletonMap("a", 2))
-
-    // build an envelope response
-    channelRequest.buildResponseSend(responseWithNotControllerError)
-
-    // expect the envelopeResponse result has NOT_CONTROLLER error
-    val capturedValue: EnvelopeResponse = 
envelopeResponseArgumentCaptor.getValue
-    assertTrue(capturedValue.error().equals(Errors.NOT_CONTROLLER))
+  private def buildCreateTopicResponse(
+    topic: String,
+    error: Errors,
+  ): CreateTopicsResponse = {
+    val responseData = new CreateTopicsResponseData()
+    responseData.topics.add(new CreateTopicsResponseData.CreatableTopicResult()
+      .setName(topic)
+      .setErrorCode(error.code)
+    )
+    new CreateTopicsResponse(responseData)
   }
 
-  private def buildMetadataRequest(): AbstractRequest = {
-    val resourceName = "topic-1"
-    val header = new RequestHeader(ApiKeys.METADATA, 
ApiKeys.METADATA.latestVersion,
-      clientId, 0)
+  private def buildUnwrappedEnvelopeRequest(request: AbstractRequest): 
RequestChannel.Request = {
+    val wrappedRequest = TestUtils.buildEnvelopeRequest(

Review Comment:
   If I understand correctly, the term "wrapped" means the request send from 
broker to controller, and "unwrapped" means the inner request which is parsed 
by the controller.



##########
core/src/test/scala/unit/kafka/network/RequestChannelTest.scala:
##########
@@ -191,84 +194,66 @@ class RequestChannelTest {
     
assertTrue(isValidJson(RequestConvertToJson.request(alterConfigs.loggableRequest).toString))
   }
 
-  @Test
-  def 
testEnvelopeBuildResponseSendShouldReturnNoErrorIfInnerResponseHasNoError(): 
Unit = {
-    val channelRequest = 
buildForwardRequestWithEnvelopeRequestAttached(buildMetadataRequest())
-
-    val envelopeResponseArgumentCaptor = 
ArgumentCaptor.forClass(classOf[EnvelopeResponse])
-
-    Mockito.doAnswer(_ => mockSend)
-      
.when(channelRequest.envelope.get.context).buildResponseSend(envelopeResponseArgumentCaptor.capture())
-
-    // create an inner response without error
-    val responseWithoutError = RequestTestUtils.metadataUpdateWith(2, 
Collections.singletonMap("a", 2))
-
-    // build an envelope response
-    channelRequest.buildResponseSend(responseWithoutError)
-
-    // expect the envelopeResponse result without error
-    val capturedValue: EnvelopeResponse = 
envelopeResponseArgumentCaptor.getValue
-    assertTrue(capturedValue.error().equals(Errors.NONE))
+  @ParameterizedTest
+  @EnumSource(value=classOf[Errors], names=Array("NONE", 
"CLUSTER_AUTHORIZATION_FAILED", "NOT_CONTROLLER"))
+  def testBuildEnvelopeResponse(error: Errors): Unit = {
+    val topic = "foo"
+    val createTopicRequest = buildCreateTopicRequest(topic)

Review Comment:
   We change`MetatadaRequest` to `CreateTopicRequest` because `MetatadaRequest` 
is not forwardable?



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