-
Notifications
You must be signed in to change notification settings - Fork 14.9k
MINOR: Refactor auto topic creation to separate envelope logic #21272
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,167 @@ | ||
| /* | ||
| * 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.common.errors.TimeoutException; | ||
| 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.AbstractResponse; | ||
| import org.apache.kafka.common.requests.CreateTopicsRequest; | ||
| import org.apache.kafka.common.requests.CreateTopicsResponse; | ||
| import org.apache.kafka.common.requests.EnvelopeResponse; | ||
| import org.apache.kafka.common.requests.RequestContext; | ||
| import org.apache.kafka.common.requests.RequestHeader; | ||
| import org.apache.kafka.server.common.ControllerRequestCompletionHandler; | ||
| import org.apache.kafka.server.common.NodeToControllerChannelManager; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.concurrent.CompletableFuture; | ||
|
|
||
| /** | ||
| * KRaft implementation of TopicCreator that forwards CreateTopics requests to the controller. | ||
| * When creating topics with a principal, requests are wrapped in an envelope to preserve the | ||
| * original request context for authorization. | ||
| */ | ||
| public class KRaftTopicCreator implements TopicCreator { | ||
|
|
||
| private final NodeToControllerChannelManager channelManager; | ||
|
|
||
| public KRaftTopicCreator(NodeToControllerChannelManager channelManager) { | ||
| this.channelManager = channelManager; | ||
| } | ||
|
|
||
| @Override | ||
| public CompletableFuture<CreateTopicsResponse> createTopicWithPrincipal( | ||
| RequestContext requestContext, | ||
| CreateTopicsRequest.Builder createTopicsRequest | ||
| ) { | ||
| CompletableFuture<CreateTopicsResponse> responseFuture = new CompletableFuture<>(); | ||
|
|
||
| short requestVersion = channelManager.controllerApiVersions() | ||
| .map(v -> v.latestUsableVersion(ApiKeys.CREATE_TOPICS)) | ||
| .orElse(ApiKeys.CREATE_TOPICS.latestVersion()); | ||
|
|
||
| RequestHeader requestHeader = new RequestHeader( | ||
| ApiKeys.CREATE_TOPICS, | ||
| requestVersion, | ||
| requestContext.clientId(), | ||
| requestContext.correlationId() | ||
| ); | ||
|
|
||
| AbstractRequest.Builder<? extends AbstractRequest> envelopeRequest = ForwardingManager$.MODULE$.buildEnvelopeRequest( | ||
| requestContext, | ||
| createTopicsRequest.build(requestHeader.apiVersion()) | ||
| .serializeWithHeader(requestHeader) | ||
| ); | ||
|
|
||
| ControllerRequestCompletionHandler handler = new ControllerRequestCompletionHandler() { | ||
| @Override | ||
| public void onTimeout() { | ||
| responseFuture.completeExceptionally( | ||
| new TimeoutException("CreateTopicsRequest to controller timed out") | ||
| ); | ||
| } | ||
|
|
||
| @Override | ||
| public void onComplete(ClientResponse response) { | ||
| if (response.authenticationException() != null) { | ||
| responseFuture.completeExceptionally(response.authenticationException()); | ||
| } else if (response.versionMismatch() != null) { | ||
| responseFuture.completeExceptionally(response.versionMismatch()); | ||
| } else if (response.wasDisconnected()) { | ||
| responseFuture.completeExceptionally(new IOException("Disconnected before receiving CreateTopicsResponse")); | ||
| } else if (response.hasResponse()) { | ||
| if (response.responseBody() instanceof EnvelopeResponse envelopeResponse) { | ||
| Errors envelopeError = envelopeResponse.error(); | ||
| if (envelopeError != Errors.NONE) { | ||
| responseFuture.completeExceptionally(envelopeError.exception()); | ||
| } else { | ||
| try { | ||
| CreateTopicsResponse createTopicsResponse = (CreateTopicsResponse) AbstractResponse.parseResponse( | ||
| envelopeResponse.responseData(), | ||
| requestHeader | ||
| ); | ||
| responseFuture.complete(createTopicsResponse); | ||
| } catch (Exception e) { | ||
| responseFuture.completeExceptionally(e); | ||
| } | ||
| } | ||
| } else { | ||
| responseFuture.completeExceptionally( | ||
| new IllegalStateException("Expected EnvelopeResponse but got: " + | ||
| response.responseBody().getClass().getSimpleName()) | ||
| ); | ||
| } | ||
| } else { | ||
| responseFuture.completeExceptionally( | ||
| new IllegalStateException("Got no response body for EnvelopeResponse") | ||
| ); | ||
| } | ||
| } | ||
| }; | ||
|
|
||
| channelManager.sendRequest(envelopeRequest, handler); | ||
| return responseFuture; | ||
| } | ||
|
|
||
| @Override | ||
| public CompletableFuture<CreateTopicsResponse> createTopicWithoutPrincipal( | ||
| CreateTopicsRequest.Builder createTopicsRequest | ||
| ) { | ||
| CompletableFuture<CreateTopicsResponse> responseFuture = new CompletableFuture<>(); | ||
|
|
||
| ControllerRequestCompletionHandler handler = new ControllerRequestCompletionHandler() { | ||
| @Override | ||
| public void onTimeout() { | ||
| responseFuture.completeExceptionally( | ||
| new TimeoutException("CreateTopicsRequest to controller timed out") | ||
| ); | ||
| } | ||
|
|
||
| @Override | ||
| public void onComplete(ClientResponse response) { | ||
| if (response.authenticationException() != null) { | ||
| responseFuture.completeExceptionally(response.authenticationException()); | ||
| } else if (response.versionMismatch() != null) { | ||
| responseFuture.completeExceptionally(response.versionMismatch()); | ||
| } else if (response.wasDisconnected()) { | ||
| responseFuture.completeExceptionally(new IOException("Disconnected before receiving CreateTopicsResponse")); | ||
| } else if (response.hasResponse()) { | ||
| if (response.responseBody() instanceof CreateTopicsResponse createTopicsResponse) { | ||
| responseFuture.complete(createTopicsResponse); | ||
| } else { | ||
| responseFuture.completeExceptionally( | ||
| new IllegalStateException("Expected CreateTopicsResponse but got: " + | ||
| response.responseBody().getClass().getSimpleName()) | ||
| ); | ||
| } | ||
| } else { | ||
| responseFuture.completeExceptionally( | ||
| new IllegalStateException("Got no response body for CreateTopicsRequest") | ||
| ); | ||
| } | ||
| } | ||
| }; | ||
|
|
||
| channelManager.sendRequest(createTopicsRequest, handler); | ||
| return responseFuture; | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,56 @@ | ||
| /* | ||
| * 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.common.requests.CreateTopicsRequest; | ||
| import org.apache.kafka.common.requests.CreateTopicsResponse; | ||
| import org.apache.kafka.common.requests.RequestContext; | ||
|
|
||
| import java.util.concurrent.CompletableFuture; | ||
|
|
||
| /** | ||
| * Abstraction for creating topics via the controller. | ||
| * Allows different implementations to be used interchangeably | ||
| * by the AutoTopicCreationManager, enabling better separation of concerns and testability. | ||
| */ | ||
| public interface TopicCreator { | ||
|
|
||
| /** | ||
| * Send a create topics request with principal for user-initiated topic creation. | ||
| * The request context is used to preserve the original client principal for auditing. | ||
| * | ||
| * @param requestContext The request context containing the client principal. | ||
| * @param createTopicsRequest The topics to be created. | ||
| * @return A future of the create topics response. This future will be completed on the network thread. | ||
| */ | ||
| CompletableFuture<CreateTopicsResponse> createTopicWithPrincipal( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Have you considered using method overloading? topicCreator.createTopics(context, request);
topicCreator.createTopics(request);
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a fairly significant difference (topic creation without principal vs. topic creation with principal) that may be relevant in terms of security, so I'd rather make this difference very explicit. |
||
| RequestContext requestContext, | ||
| CreateTopicsRequest.Builder createTopicsRequest | ||
| ); | ||
|
|
||
| /** | ||
| * Send a create topics request without principal for internal topic creation (e.g., consumer offsets, transaction state). | ||
| * No request context is required since these are system-initiated requests. | ||
| * | ||
| * @param createTopicsRequest The topics to be created. | ||
| * @return A future of the create topics response. This future will be completed on the network thread. | ||
| */ | ||
| CompletableFuture<CreateTopicsResponse> createTopicWithoutPrincipal( | ||
| CreateTopicsRequest.Builder createTopicsRequest | ||
| ); | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As a follow-up, maybe we should move the
ForwardingManagerhelper to theservermodule along withTopicCreatorThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, let's do that in a separate PR.