C0urante commented on code in PR #15149: URL: https://github.com/apache/kafka/pull/15149#discussion_r1447427024
########## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/HerderRequestHandler.java: ########## @@ -41,18 +41,11 @@ public class HerderRequestHandler { private final RestClient restClient; - private volatile long requestTimeoutMs; + private final RestRequestTimeout requestTimeout; Review Comment: This field is final, it wouldn't make sense to mark it volatile. The underlying mutable value that it returns from `timeoutMs()` is still marked volatile in the only non-test implementation [here](https://github.com/C0urante/kafka/blob/0713736b9884084c360126558e4b6d92d4435bfa/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java#L533). ########## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java: ########## @@ -45,25 +45,40 @@ public void initializeResources(Herder herder) { } @Override - protected Collection<ConnectResource> regularResources() { + protected Collection<Class<?>> regularResources() { return Arrays.asList( - new RootResource(herder), - new ConnectorsResource(herder, config, restClient), - new InternalConnectResource(herder, restClient), - new ConnectorPluginsResource(herder) + RootResource.class, + ConnectorsResource.class, + InternalConnectResource.class, + ConnectorPluginsResource.class ); } @Override - protected Collection<ConnectResource> adminResources() { + protected Collection<Class<?>> adminResources() { return Arrays.asList( - new LoggingResource(herder) + LoggingResource.class ); } @Override protected void configureRegularResources(ResourceConfig resourceConfig) { registerRestExtensions(herder, resourceConfig); + resourceConfig.register(new Binder()); + } + + private class Binder extends AbstractBinder { + @Override + protected void configure() { + bind(herder).to(Herder.class); + bind(restClient).to(RestClient.class); + bind(config).to(RestServerConfig.class); + } + } + Review Comment: This is true, but not a blocker. ########## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/HerderRequestHandler.java: ########## @@ -41,18 +41,11 @@ public class HerderRequestHandler { private final RestClient restClient; - private volatile long requestTimeoutMs; + private final RestRequestTimeout requestTimeout; - public HerderRequestHandler(RestClient restClient, long requestTimeoutMs) { + public HerderRequestHandler(RestClient restClient, RestRequestTimeout requestTimeout) { this.restClient = restClient; - this.requestTimeoutMs = requestTimeoutMs; - } - - public void requestTimeoutMs(long requestTimeoutMs) { - if (requestTimeoutMs < 1) { - throw new IllegalArgumentException("REST request timeout must be positive"); - } Review Comment: I removed it for two reasons: 1. This code path is only ever used in testing code 2. It's unclear that the validation would be safer than allowing negative values. At least on my machine, it doesn't cause any issues with, e.g., `ConvertingFutureCallback::get`. -- 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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org