-
Notifications
You must be signed in to change notification settings - Fork 1k
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
fix: Reuse KsqlClient instance for inter node requests #5624
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 |
---|---|---|
|
@@ -48,9 +48,11 @@ | |
import io.confluent.ksql.name.SourceName; | ||
import io.confluent.ksql.parser.KsqlParser.ParsedStatement; | ||
import io.confluent.ksql.parser.KsqlParser.PreparedStatement; | ||
import io.confluent.ksql.properties.LocalProperties; | ||
import io.confluent.ksql.query.id.SpecificQueryIdGenerator; | ||
import io.confluent.ksql.rest.ErrorMessages; | ||
import io.confluent.ksql.rest.Errors; | ||
import io.confluent.ksql.rest.client.KsqlClient; | ||
import io.confluent.ksql.rest.client.RestResponse; | ||
import io.confluent.ksql.rest.entity.KsqlEntityList; | ||
import io.confluent.ksql.rest.entity.KsqlErrorMessage; | ||
|
@@ -101,6 +103,7 @@ | |
import io.confluent.ksql.version.metrics.collector.KsqlModuleType; | ||
import io.vertx.core.Vertx; | ||
import io.vertx.core.VertxOptions; | ||
import io.vertx.core.http.HttpClientOptions; | ||
import io.vertx.ext.dropwizard.DropwizardMetricsOptions; | ||
import io.vertx.ext.dropwizard.Match; | ||
import java.io.Console; | ||
|
@@ -112,6 +115,7 @@ | |
import java.nio.charset.StandardCharsets; | ||
import java.time.Duration; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.HashSet; | ||
import java.util.LinkedList; | ||
import java.util.List; | ||
|
@@ -210,7 +214,8 @@ public static SourceName getCommandsStreamName() { | |
final Consumer<KsqlConfig> rocksDBConfigSetterHandler, | ||
final PullQueryExecutor pullQueryExecutor, | ||
final Optional<HeartbeatAgent> heartbeatAgent, | ||
final Optional<LagReportingAgent> lagReportingAgent | ||
final Optional<LagReportingAgent> lagReportingAgent, | ||
final Vertx vertx | ||
) { | ||
log.debug("Creating instance of ksqlDB API server"); | ||
this.serviceContext = requireNonNull(serviceContext, "serviceContext"); | ||
|
@@ -236,12 +241,7 @@ public static SourceName getCommandsStreamName() { | |
this.pullQueryExecutor = requireNonNull(pullQueryExecutor, "pullQueryExecutor"); | ||
this.heartbeatAgent = requireNonNull(heartbeatAgent, "heartbeatAgent"); | ||
this.lagReportingAgent = requireNonNull(lagReportingAgent, "lagReportingAgent"); | ||
this.vertx = Vertx.vertx( | ||
new VertxOptions() | ||
.setMaxWorkerExecuteTimeUnit(TimeUnit.MILLISECONDS) | ||
.setMaxWorkerExecuteTime(Long.MAX_VALUE) | ||
.setMetricsOptions(setUpHttpMetrics(ksqlConfig))); | ||
this.vertx.exceptionHandler(t -> log.error("Unhandled exception in Vert.x", t)); | ||
this.vertx = requireNonNull(vertx, "vertx"); | ||
|
||
this.serverInfoResource = new ServerInfoResource(serviceContext, ksqlConfigNoPort); | ||
if (heartbeatAgent.isPresent()) { | ||
|
@@ -554,7 +554,7 @@ public static KsqlRestApplication buildApplication(final KsqlRestConfig restConf | |
new KsqlSchemaRegistryClientFactory(ksqlConfig, Collections.emptyMap())::get; | ||
final ServiceContext serviceContext = new LazyServiceContext(() -> | ||
RestServiceContextFactory.create(ksqlConfig, Optional.empty(), | ||
schemaRegistryClientFactory)); | ||
schemaRegistryClientFactory, Optional.empty())); | ||
|
||
return buildApplication( | ||
"", | ||
|
@@ -629,11 +629,27 @@ static KsqlRestApplication buildApplication( | |
|
||
final KsqlSecurityExtension securityExtension = loadSecurityExtension(ksqlConfig); | ||
|
||
final Vertx vertx = Vertx.vertx( | ||
new VertxOptions() | ||
.setMaxWorkerExecuteTimeUnit(TimeUnit.MILLISECONDS) | ||
.setMaxWorkerExecuteTime(Long.MAX_VALUE) | ||
.setMetricsOptions(setUpHttpMetrics(ksqlConfig))); | ||
vertx.exceptionHandler(t -> log.error("Unhandled exception in Vert.x", t)); | ||
|
||
final KsqlClient sharedClient = new KsqlClient( | ||
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. I have logic in I think the main solution is to move out that logic to a central place so that it can be used by this or |
||
toClientProps(ksqlConfig.originals()), | ||
Optional.empty(), | ||
new LocalProperties(ImmutableMap.of()), | ||
new HttpClientOptions().setMaxPoolSize(100), | ||
vertx | ||
); | ||
|
||
final KsqlSecurityContextProvider ksqlSecurityContextProvider = | ||
new DefaultKsqlSecurityContextProvider( | ||
securityExtension, | ||
RestServiceContextFactory::create, | ||
RestServiceContextFactory::create, ksqlConfig, schemaRegistryClientFactory); | ||
RestServiceContextFactory::create, ksqlConfig, schemaRegistryClientFactory, | ||
sharedClient); | ||
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. Does this mean that the shared client will be used for all internode communication? Also heartbeat and lag reporting? |
||
|
||
final Optional<AuthenticationPlugin> securityHandlerPlugin = loadAuthenticationPlugin( | ||
restConfig); | ||
|
@@ -730,8 +746,8 @@ static KsqlRestApplication buildApplication( | |
rocksDBConfigSetterHandler, | ||
pullQueryExecutor, | ||
heartbeatAgent, | ||
lagReportingAgent | ||
); | ||
lagReportingAgent, | ||
vertx); | ||
} | ||
|
||
private static Optional<HeartbeatAgent> initializeHeartbeatAgent( | ||
|
@@ -999,4 +1015,12 @@ private static KsqlRestConfig injectPathsWithoutAuthentication(final KsqlRestCon | |
return new KsqlRestConfig(restConfigs); | ||
} | ||
|
||
private static Map<String, String> toClientProps(final Map<String, Object> config) { | ||
final Map<String, String> clientProps = new HashMap<>(); | ||
for (Map.Entry<String, Object> entry : config.entrySet()) { | ||
clientProps.put(entry.getKey(), entry.getValue().toString()); | ||
} | ||
return clientProps; | ||
} | ||
|
||
} |
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.
This is already a singleton, so it's not a big savings, but you could use the same vertx object and
KsqlClient
here too, if you wanted. That way it wouldn't have to be optional and I think you might be able to remove some of theboolean sharedClient
s around.