Copilot commented on code in PR #21272:
URL: https://github.com/apache/kafka/pull/21272#discussion_r2673815931


##########
core/src/main/scala/kafka/server/AutoTopicCreationManager.scala:
##########
@@ -208,99 +205,31 @@ class DefaultAutoTopicCreationManager(
     creatableTopics: Map[String, CreatableTopic],
     requestContext: Option[RequestContext]
   ): Seq[MetadataResponseTopic] = {
-    val topicsToCreate = new 
CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size)
-    topicsToCreate.addAll(creatableTopics.values.asJavaCollection)
-
-    val createTopicsRequest = new CreateTopicsRequest.Builder(
-      new CreateTopicsRequestData()
-        .setTimeoutMs(config.requestTimeoutMs)
-        .setTopics(topicsToCreate)
-    )
+    val createTopicsRequest: CreateTopicsRequest.Builder = 
makeCreateTopicsRequestBuilder(creatableTopics)
 
-    // Capture request header information for proper envelope response parsing
-    val requestHeaderForParsing = requestContext.map { context =>
-      val requestVersion =
-        channelManager.controllerApiVersions.toScala match {
-          case None =>
-            ApiKeys.CREATE_TOPICS.latestVersion()
-          case Some(nodeApiVersions) =>
-            nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS)
-        }
-
-      new RequestHeader(ApiKeys.CREATE_TOPICS,
-        requestVersion,
-        context.clientId,
-        context.correlationId)
+    val responseFuture = requestContext match {
+      case Some(context) => topicCreator.createTopicWithPrincipal(context, 
createTopicsRequest)
+      case None => 
topicCreator.createTopicWithoutPrincipal(createTopicsRequest)
     }
 
-    val requestCompletionHandler = new ControllerRequestCompletionHandler {
-      override def onTimeout(): Unit = {
+    responseFuture.whenComplete {
+      (response, throwable) =>
         clearInflightRequests(creatableTopics)
-        debug(s"Auto topic creation timed out for ${creatableTopics.keys}.")
-      }
-
-      override def onComplete(response: ClientResponse): Unit = {
-        clearInflightRequests(creatableTopics)
-        if (response.authenticationException() != null) {
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
authentication exception")
-        } else if (response.versionMismatch() != null) {
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
invalid version exception")
-        } else {
-          if (response.hasResponse) {
-            response.responseBody() match {
-              case envelopeResponse: EnvelopeResponse =>
-                // Unwrap the envelope response to get the actual 
CreateTopicsResponse
-                val envelopeError = envelopeResponse.error()
-                if (envelopeError != Errors.NONE) {
-                  warn(s"Auto topic creation failed for 
${creatableTopics.keys} with envelope error: ${envelopeError}")
-                } else {
-                  requestHeaderForParsing match {
-                    case Some(requestHeader) =>
-                      try {
-                        // Use the captured request header for proper envelope 
response parsing
-                        val createTopicsResponse = 
AbstractResponse.parseResponse(
-                          envelopeResponse.responseData(), 
requestHeader).asInstanceOf[CreateTopicsResponse]
-
-                        
createTopicsResponse.data().topics().forEach(topicResult => {
-                          val error = Errors.forCode(topicResult.errorCode)
-                          if (error != Errors.NONE) {
-                            warn(s"Auto topic creation failed for 
${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}")
-                          }
-                        })
-                      } catch {
-                        case e: Exception =>
-                          warn(s"Failed to parse envelope response for auto 
topic creation of ${creatableTopics.keys}", e)
-                      }
-                    case None =>
-                      warn(s"Cannot parse envelope response without original 
request header information")
-                  }
-                }
-              case createTopicsResponse: CreateTopicsResponse =>
-                createTopicsResponse.data().topics().forEach(topicResult => {
-                  val error = Errors.forCode(topicResult.errorCode)
-                  if (error != Errors.NONE) {
-                    warn(s"Auto topic creation failed for ${topicResult.name} 
with error '${error.name}': ${topicResult.errorMessage}")
-                  }
-                })
-              case other =>
-                warn(s"Auto topic creation request received unexpected 
response type: ${other.getClass.getSimpleName}")
+        // Log any errors from the topic creation attempt
+        if (throwable != null) {
+          logError(creatableTopics, throwable)
+        } else if (response != null) {
+          response.data().topics().forEach(topicResult => {
+            val error = Errors.forCode(topicResult.errorCode)
+            if (error != Errors.NONE) {
+              warn(s"Auto topic creation failed for ${topicResult.name} with 
error '${error.name}': ${topicResult.errorMessage}")
             }
-          }
-          debug(s"Auto topic creation completed for ${creatableTopics.keys} 
with response ${response.responseBody}.")
+          })
+        } else {
+          throw new IllegalStateException("CreateTopicsResponse future 
completed with null response and no exception")

Review Comment:
   The exception thrown on line 229 will not be caught or handled properly 
because it's thrown in a CompletableFuture callback. When an exception is 
thrown in a whenComplete callback, it will cause the calling thread (the 
network thread that completes the future) to see an uncaught exception. This 
should use `responseFuture.completeExceptionally` or be logged as a warning 
instead of throwing.
   ```suggestion
             warn("CreateTopicsResponse future completed with null response and 
no exception")
   ```



##########
core/src/test/java/kafka/server/KRaftTopicCreatorTest.java:
##########
@@ -0,0 +1,577 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server;
+
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.NodeApiVersions;
+import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.ApiVersionsResponseData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ClientInformation;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.CreateTopicsRequest;
+import org.apache.kafka.common.requests.CreateTopicsResponse;
+import org.apache.kafka.common.requests.EnvelopeRequest;
+import org.apache.kafka.common.requests.EnvelopeResponse;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.requests.RequestContext;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.RequestUtils;
+import org.apache.kafka.common.requests.ResponseHeader;
+import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.security.auth.KafkaPrincipalSerde;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.SecurityUtils;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.ControllerRequestCompletionHandler;
+import org.apache.kafka.server.common.NodeToControllerChannelManager;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class KRaftTopicCreatorTest {
+
+    private static final int REQUEST_TIMEOUT = 100;
+
+    private NodeToControllerChannelManager brokerToController;
+    private KRaftTopicCreator kraftTopicCreator;
+
+    @BeforeEach
+    public void setup() {
+        brokerToController = mock(NodeToControllerChannelManager.class);
+
+        ApiVersionsResponseData.ApiVersion createTopicApiVersion = new 
ApiVersionsResponseData.ApiVersion()
+            .setApiKey(ApiKeys.CREATE_TOPICS.id)
+            .setMinVersion(ApiKeys.CREATE_TOPICS.oldestVersion())
+            .setMaxVersion(ApiKeys.CREATE_TOPICS.latestVersion());
+
+        when(brokerToController.controllerApiVersions())
+            
.thenReturn(Optional.of(NodeApiVersions.create(Collections.singleton(createTopicApiVersion))));
+
+        kraftTopicCreator = new KRaftTopicCreator(brokerToController);
+    }
+
+    @Test
+    public void testCreateTopicWithMetadataContextPassPrincipal() throws 
Exception {
+        String topicName = "topic";
+        KafkaPrincipal userPrincipal = new 
KafkaPrincipal(KafkaPrincipal.USER_TYPE, "user");
+        AtomicBoolean serializeIsCalled = new AtomicBoolean(false);
+
+        KafkaPrincipalSerde principalSerde = new KafkaPrincipalSerde() {
+            @Override
+            public byte[] serialize(KafkaPrincipal principal) {
+                assertEquals(principal, userPrincipal);
+                serializeIsCalled.set(true);
+                return Utils.utf8(principal.toString());
+            }
+
+            @Override
+            public KafkaPrincipal deserialize(byte[] bytes) {
+                return SecurityUtils.parseKafkaPrincipal(Utils.utf8(bytes));
+            }
+        };
+
+        RequestContext requestContext = 
initializeRequestContext(userPrincipal, Optional.of(principalSerde));
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        kraftTopicCreator.createTopicWithPrincipal(requestContext, 
createTopicsRequest);
+
+        assertTrue(serializeIsCalled.get());
+
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<AbstractRequest.Builder<? extends AbstractRequest>> 
argumentCaptor =
+            (ArgumentCaptor<AbstractRequest.Builder<? extends 
AbstractRequest>>) (ArgumentCaptor<?>) 
ArgumentCaptor.forClass(AbstractRequest.Builder.class);
+        verify(brokerToController).sendRequest(
+            argumentCaptor.capture(),
+            any(ControllerRequestCompletionHandler.class));
+
+        EnvelopeRequest capturedRequest = (EnvelopeRequest) 
argumentCaptor.getValue()
+            .build(ApiKeys.ENVELOPE.latestVersion());
+        assertEquals(userPrincipal, 
SecurityUtils.parseKafkaPrincipal(Utils.utf8(capturedRequest.requestPrincipal())));
+    }
+
+    @Test
+    public void 
testCreateTopicWithMetadataContextWhenPrincipalSerdeNotDefined() {
+        String topicName = "topic";
+        RequestContext requestContext = 
initializeRequestContext(KafkaPrincipal.ANONYMOUS, Optional.empty());
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        assertThrows(IllegalArgumentException.class,
+            () -> kraftTopicCreator.createTopicWithPrincipal(requestContext, 
createTopicsRequest));
+    }
+
+    @Test
+    public void testCreateTopicWithoutRequestContext() {
+        String topicName = "topic";
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        kraftTopicCreator.createTopicWithoutPrincipal(createTopicsRequest);
+
+        @SuppressWarnings("unchecked")
+        ArgumentCaptor<AbstractRequest.Builder<? extends AbstractRequest>> 
argumentCaptor =
+            (ArgumentCaptor<AbstractRequest.Builder<? extends 
AbstractRequest>>) (ArgumentCaptor<?>) 
ArgumentCaptor.forClass(AbstractRequest.Builder.class);
+        verify(brokerToController).sendRequest(
+            argumentCaptor.capture(),
+            any(ControllerRequestCompletionHandler.class));
+
+        AbstractRequest.Builder<?> capturedRequest = argumentCaptor.getValue();
+        assertTrue(capturedRequest instanceof CreateTopicsRequest.Builder,
+            "Should send CreateTopicsRequest.Builder when no request context 
provided");
+    }
+
+    @Test
+    public void testEnvelopeResponseSuccessfulParsing() throws Exception {
+        String topicName = "test-topic";
+        RequestContext requestContext = 
initializeRequestContextWithUserPrincipal();
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        CompletableFuture<CreateTopicsResponse> responseFuture =
+            kraftTopicCreator.createTopicWithPrincipal(requestContext, 
createTopicsRequest);
+
+        ArgumentCaptor<ControllerRequestCompletionHandler> argumentCaptor =
+            ArgumentCaptor.forClass(ControllerRequestCompletionHandler.class);
+        verify(brokerToController).sendRequest(
+            any(),
+            argumentCaptor.capture());
+
+        CreateTopicsResponseData createTopicsResponseData = new 
CreateTopicsResponseData();
+        CreateTopicsResponseData.CreatableTopicResult topicResult =
+            new CreateTopicsResponseData.CreatableTopicResult()
+                .setName(topicName)
+                .setErrorCode(Errors.NONE.code())
+                .setNumPartitions(1)
+                .setReplicationFactor((short) 1);
+        createTopicsResponseData.topics().add(topicResult);
+
+        CreateTopicsResponse createTopicsResponse = new 
CreateTopicsResponse(createTopicsResponseData);
+        short requestVersion = ApiKeys.CREATE_TOPICS.latestVersion();
+        int correlationId = requestContext.correlationId();
+        String clientId = requestContext.clientId();
+
+        ResponseHeader responseHeader = new ResponseHeader(
+            correlationId,
+            ApiKeys.CREATE_TOPICS.responseHeaderVersion(requestVersion)
+        );
+        ByteBuffer serializedResponse = RequestUtils.serialize(
+            responseHeader.data(),
+            responseHeader.headerVersion(),
+            createTopicsResponse.data(),
+            requestVersion
+        );
+
+        EnvelopeResponse envelopeResponse = new 
EnvelopeResponse(serializedResponse, Errors.NONE);
+        RequestHeader requestHeader = new RequestHeader(ApiKeys.ENVELOPE, 
(short) 0, clientId, correlationId);
+        ClientResponse clientResponse = new ClientResponse(
+            requestHeader, null, null, 0, 0, false, null, null, 
envelopeResponse
+        );
+
+        argumentCaptor.getValue().onComplete(clientResponse);
+
+        CreateTopicsResponse result = responseFuture.get();
+        assertEquals(1, result.data().topics().size());
+        assertEquals(topicName, 
result.data().topics().iterator().next().name());
+        assertEquals(Errors.NONE.code(), 
result.data().topics().iterator().next().errorCode());
+    }
+
+    @Test
+    public void testEnvelopeResponseWithEnvelopeError() {
+        String topicName = "test-topic";
+        RequestContext requestContext = 
initializeRequestContextWithUserPrincipal();
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        CompletableFuture<CreateTopicsResponse> responseFuture =
+            kraftTopicCreator.createTopicWithPrincipal(requestContext, 
createTopicsRequest);
+
+        ArgumentCaptor<ControllerRequestCompletionHandler> argumentCaptor =
+            ArgumentCaptor.forClass(ControllerRequestCompletionHandler.class);
+        verify(brokerToController).sendRequest(
+            any(),
+            argumentCaptor.capture());
+
+        EnvelopeResponse envelopeResponse = new 
EnvelopeResponse(ByteBuffer.allocate(0), Errors.UNSUPPORTED_VERSION);
+        RequestHeader requestHeader = new RequestHeader(
+            ApiKeys.ENVELOPE, (short) 0, requestContext.clientId(), 
requestContext.correlationId()
+        );
+        ClientResponse clientResponse = new ClientResponse(
+            requestHeader, null, null, 0, 0, false, null, null, 
envelopeResponse
+        );
+
+        argumentCaptor.getValue().onComplete(clientResponse);
+
+        assertThrows(ExecutionException.class, responseFuture::get);
+        assertTrue(responseFuture.isCompletedExceptionally());
+    }
+
+    @Test
+    public void testEnvelopeResponseParsingException() {
+        String topicName = "test-topic";
+        RequestContext requestContext = 
initializeRequestContextWithUserPrincipal();
+        CreateTopicsRequest.Builder createTopicsRequest = 
createCreateTopicsRequestBuilder(topicName);
+
+        CompletableFuture<CreateTopicsResponse> responseFuture =
+            kraftTopicCreator.createTopicWithPrincipal(requestContext, 
createTopicsRequest);
+
+        ArgumentCaptor<ControllerRequestCompletionHandler> argumentCaptor =
+            ArgumentCaptor.forClass(ControllerRequestCompletionHandler.class);
+        verify(brokerToController).sendRequest(
+            any(),
+            argumentCaptor.capture());
+
+        ByteBuffer malformedData = ByteBuffer.wrap("invalid response 
data".getBytes());
+        EnvelopeResponse envelopeResponse = new 
EnvelopeResponse(malformedData, Errors.NONE);
+        RequestHeader requestHeader = new RequestHeader(
+            ApiKeys.ENVELOPE, (short) 0, requestContext.clientId(), 
requestContext.correlationId()
+        );
+        ClientResponse clientResponse = new ClientResponse(
+            requestHeader, null, null, 0, 0, false, null, null, 
envelopeResponse
+        );
+
+        try {
+            argumentCaptor.getValue().onComplete(clientResponse);
+            ExecutionException exception = 
assertThrows(ExecutionException.class, responseFuture::get);
+            assertTrue(exception.getCause() instanceof RuntimeException);
+        } catch (RuntimeException e) {
+            assertTrue(responseFuture.isCompletedExceptionally());
+            ExecutionException exception = 
assertThrows(ExecutionException.class, responseFuture::get);
+            assertTrue(exception.getCause() instanceof RuntimeException);
+        }
+        assertTrue(responseFuture.isCompletedExceptionally());

Review Comment:
   The try-catch logic in this test is confusing and potentially incorrect. The 
test wraps the callback invocation in a try-catch expecting that it might throw 
a RuntimeException, but then has duplicate assertions in both the try and catch 
blocks. The structure suggests uncertainty about whether the exception will be 
thrown synchronously or captured in the future. This should be simplified - if 
the responseFuture is expected to be completed exceptionally, the test should 
just invoke the callback and then assert on the future state without the 
try-catch wrapper.
   ```suggestion
           argumentCaptor.getValue().onComplete(clientResponse);
           assertTrue(responseFuture.isCompletedExceptionally());
           ExecutionException exception = 
assertThrows(ExecutionException.class, responseFuture::get);
           assertTrue(exception.getCause() instanceof RuntimeException);
   ```



##########
core/src/main/scala/kafka/server/AutoTopicCreationManager.scala:
##########
@@ -401,109 +330,51 @@ class DefaultAutoTopicCreationManager(
 
   private def sendCreateTopicRequestWithErrorCaching(
     creatableTopics: Map[String, CreatableTopic],
-    requestContext: Option[RequestContext],
+    requestContext: RequestContext,
     timeoutMs: Long
-  ): Seq[MetadataResponseTopic] = {
-    val topicsToCreate = new 
CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size)
-    topicsToCreate.addAll(creatableTopics.values.asJavaCollection)
-
-    val createTopicsRequest = new CreateTopicsRequest.Builder(
-      new CreateTopicsRequestData()
-        .setTimeoutMs(config.requestTimeoutMs)
-        .setTopics(topicsToCreate)
-    )
-
-    // Capture request header information for proper envelope response parsing
-    val requestHeaderForParsing = requestContext.map { context =>
-      val requestVersion =
-        channelManager.controllerApiVersions.toScala match {
-          case None =>
-            ApiKeys.CREATE_TOPICS.latestVersion()
-          case Some(nodeApiVersions) =>
-            nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS)
-        }
-
-      new RequestHeader(ApiKeys.CREATE_TOPICS,
-        requestVersion,
-        context.clientId,
-        context.correlationId)
-    }
+  ): Unit = {
+    val createTopicsRequest: CreateTopicsRequest.Builder = 
makeCreateTopicsRequestBuilder(creatableTopics)
 
-    val requestCompletionHandler = new ControllerRequestCompletionHandler {
-      override def onTimeout(): Unit = {
-        clearInflightRequests(creatableTopics)
-        debug(s"Auto topic creation timed out for ${creatableTopics.keys}.")
-        cacheTopicCreationErrors(creatableTopics.keys.toSet, "Auto topic 
creation timed out.", timeoutMs)
-      }
+    val createTopicsResponseFuture = 
topicCreator.createTopicWithPrincipal(requestContext, createTopicsRequest)
 
-      override def onComplete(response: ClientResponse): Unit = {
+    createTopicsResponseFuture.whenComplete {
+      (response, throwable) =>
         clearInflightRequests(creatableTopics)
-        if (response.authenticationException() != null) {
-          val authException = response.authenticationException()
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
authentication exception: ${authException.getMessage}")
-          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
authException.getMessage, timeoutMs)
-        } else if (response.versionMismatch() != null) {
-          val versionException = response.versionMismatch()
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
version mismatch exception: ${versionException.getMessage}")
-          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
versionException.getMessage, timeoutMs)
+        // Log any errors from the topic creation attempt
+        if (throwable != null) {
+          logError(creatableTopics, throwable)
+          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
throwable.getMessage, timeoutMs)

Review Comment:
   There's a potential null pointer exception risk here. If 
`throwable.getMessage` returns null, the error cache will store a null error 
message. This should be handled with a null check and fallback message, similar 
to how it's done in `cacheTopicCreationErrorsFromResponse` on lines 389-391.
   ```suggestion
             val errorMessage = 
Option(throwable.getMessage).getOrElse(throwable.toString)
             cacheTopicCreationErrors(creatableTopics.keys.toSet, errorMessage, 
timeoutMs)
   ```



##########
core/src/main/scala/kafka/server/AutoTopicCreationManager.scala:
##########
@@ -401,109 +330,51 @@ class DefaultAutoTopicCreationManager(
 
   private def sendCreateTopicRequestWithErrorCaching(
     creatableTopics: Map[String, CreatableTopic],
-    requestContext: Option[RequestContext],
+    requestContext: RequestContext,
     timeoutMs: Long
-  ): Seq[MetadataResponseTopic] = {
-    val topicsToCreate = new 
CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size)
-    topicsToCreate.addAll(creatableTopics.values.asJavaCollection)
-
-    val createTopicsRequest = new CreateTopicsRequest.Builder(
-      new CreateTopicsRequestData()
-        .setTimeoutMs(config.requestTimeoutMs)
-        .setTopics(topicsToCreate)
-    )
-
-    // Capture request header information for proper envelope response parsing
-    val requestHeaderForParsing = requestContext.map { context =>
-      val requestVersion =
-        channelManager.controllerApiVersions.toScala match {
-          case None =>
-            ApiKeys.CREATE_TOPICS.latestVersion()
-          case Some(nodeApiVersions) =>
-            nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS)
-        }
-
-      new RequestHeader(ApiKeys.CREATE_TOPICS,
-        requestVersion,
-        context.clientId,
-        context.correlationId)
-    }
+  ): Unit = {
+    val createTopicsRequest: CreateTopicsRequest.Builder = 
makeCreateTopicsRequestBuilder(creatableTopics)
 
-    val requestCompletionHandler = new ControllerRequestCompletionHandler {
-      override def onTimeout(): Unit = {
-        clearInflightRequests(creatableTopics)
-        debug(s"Auto topic creation timed out for ${creatableTopics.keys}.")
-        cacheTopicCreationErrors(creatableTopics.keys.toSet, "Auto topic 
creation timed out.", timeoutMs)
-      }
+    val createTopicsResponseFuture = 
topicCreator.createTopicWithPrincipal(requestContext, createTopicsRequest)
 
-      override def onComplete(response: ClientResponse): Unit = {
+    createTopicsResponseFuture.whenComplete {
+      (response, throwable) =>
         clearInflightRequests(creatableTopics)
-        if (response.authenticationException() != null) {
-          val authException = response.authenticationException()
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
authentication exception: ${authException.getMessage}")
-          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
authException.getMessage, timeoutMs)
-        } else if (response.versionMismatch() != null) {
-          val versionException = response.versionMismatch()
-          warn(s"Auto topic creation failed for ${creatableTopics.keys} with 
version mismatch exception: ${versionException.getMessage}")
-          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
versionException.getMessage, timeoutMs)
+        // Log any errors from the topic creation attempt
+        if (throwable != null) {
+          logError(creatableTopics, throwable)
+          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
throwable.getMessage, timeoutMs)
+        } else if (response != null) {
+          debug(s"Auto topic creation completed for ${creatableTopics.keys} 
with response $response.")
+          cacheTopicCreationErrorsFromResponse(response, timeoutMs)
         } else {
-          if (response.hasResponse) {
-            response.responseBody() match {
-              case envelopeResponse: EnvelopeResponse =>
-                // Unwrap the envelope response to get the actual 
CreateTopicsResponse
-                val envelopeError = envelopeResponse.error()
-                if (envelopeError != Errors.NONE) {
-                  warn(s"Auto topic creation failed for 
${creatableTopics.keys} with envelope error: ${envelopeError}")
-                  cacheTopicCreationErrors(creatableTopics.keys.toSet, 
s"Envelope error: ${envelopeError}", timeoutMs)
-                } else {
-                  requestHeaderForParsing match {
-                    case Some(requestHeader) =>
-                      try {
-                        // Use the captured request header for proper envelope 
response parsing
-                        val createTopicsResponse = 
AbstractResponse.parseResponse(
-                          envelopeResponse.responseData(), 
requestHeader).asInstanceOf[CreateTopicsResponse]
-
-                        
cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs)
-                      } catch {
-                        case e: Exception =>
-                          warn(s"Failed to parse envelope response for auto 
topic creation of ${creatableTopics.keys}", e)
-                          cacheTopicCreationErrors(creatableTopics.keys.toSet, 
s"Response parsing error: ${e.getMessage}", timeoutMs)
-                      }
-                    case None =>
-                      warn(s"Cannot parse envelope response without original 
request header information")
-                      cacheTopicCreationErrors(creatableTopics.keys.toSet, 
"Missing request header for envelope parsing", timeoutMs)
-                  }
-                }
-              case createTopicsResponse: CreateTopicsResponse =>
-                cacheTopicCreationErrorsFromResponse(createTopicsResponse, 
timeoutMs)
-              case unexpectedResponse =>
-                warn(s"Auto topic creation request received unexpected 
response type: ${unexpectedResponse.getClass.getSimpleName}")
-                cacheTopicCreationErrors(creatableTopics.keys.toSet, 
s"Unexpected response type: ${unexpectedResponse.getClass.getSimpleName}", 
timeoutMs)
-            }
-            debug(s"Auto topic creation completed for ${creatableTopics.keys} 
with response ${response.responseBody}.")
-          }
+          throw new IllegalStateException("CreateTopicsResponse future 
completed with null response and no exception")

Review Comment:
   The exception thrown on line 351 will not be caught or handled properly 
because it's thrown in a CompletableFuture callback. When an exception is 
thrown in a whenComplete callback, it will cause the calling thread (the 
network thread that completes the future) to see an uncaught exception. This 
should be logged as an error instead of throwing.
   ```suggestion
             val ex = new IllegalStateException("CreateTopicsResponse future 
completed with null response and no exception")
             error(s"Auto topic creation failed for ${creatableTopics.keys} due 
to unexpected future completion state", ex)
             cacheTopicCreationErrors(creatableTopics.keys.toSet, 
ex.getMessage, timeoutMs)
   ```



##########
core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala:
##########
@@ -119,109 +168,87 @@ class AutoTopicCreationManagerTest {
                               replicationFactor: Short = 1): Unit = {
     autoTopicCreationManager = new DefaultAutoTopicCreationManager(
       config,
-      brokerToController,
       groupCoordinator,
       transactionCoordinator,
       shareCoordinator,
       mockTime,
+      topicCreator,
       topicErrorCacheCapacity = testCacheCapacity)
 
-    val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection
-    topicsCollection.add(getNewTopic(topicName, numPartitions, 
replicationFactor))
-    val requestBody = new CreateTopicsRequest.Builder(
-      new CreateTopicsRequestData()
-        .setTopics(topicsCollection)
-        .setTimeoutMs(requestTimeout))
+    // Set up the topicCreator to return a successful response
+    val createTopicsResponseData = new CreateTopicsResponseData()
+    val topicResult = new CreatableTopicResult()
+      .setName(topicName)
+      .setErrorCode(Errors.NONE.code())
+    createTopicsResponseData.topics().add(topicResult)
+    val response = new CreateTopicsResponse(createTopicsResponseData)
+    topicCreator.setResponseForWithoutPrincipal(response)
 
-    // Calling twice with the same topic will only trigger one forwarding.
-    createTopicAndVerifyResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, topicName, 
isInternal)
+    // Calling twice with the same topic will only trigger one call to 
topicCreator
+    // because the second call will see the topic as already in-flight
     createTopicAndVerifyResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, topicName, 
isInternal)

Review Comment:
   The comment on line 188 states "because the second call will see the topic 
as already in-flight", but this is misleading. Looking at the test setup, line 
185 sets the response to return immediately with a completed future. This means 
the topic won't be in-flight when the second call happens because the callback 
at lines 215-231 in AutoTopicCreationManager.scala will have already cleared 
the inflight state via clearInflightRequests. The comment should be updated to 
reflect that the second call tests the behavior after the first request 
completes, not that it's blocked by inflight state.



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