Skip to content

Commit

Permalink
feat: support checking preconditions before starting core app (#9026)
Browse files Browse the repository at this point in the history
* feat: support checking preconditions before starting core app

This patch changes up the precondition checker a bit so that it
runs before the main KsqlRestApplication. This way, preconditions
can look out for critical config properties to arrive, before they
are handled by the actuall application.

Detailed changes:
- Run the precondition checker as a separate Executable implementation
  before running KsqlRestApplication. The precondition checker waits for
  any preconditions to pass, and while this is ongoing it runs a simple
  server that responds to all requests (other than the health probes)
  with a 503.
- Change the precondition interface to accept a properties loader so
  that precondition checkers can wait on the arrival of dynamic properties.
- Support multiple properties files as args (which are then overlaid)

* feedback

* tests pass
  • Loading branch information
rodesai authored Apr 20, 2022
1 parent 503e4cd commit 33a6a04
Show file tree
Hide file tree
Showing 22 changed files with 1,232 additions and 477 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

package io.confluent.ksql.properties;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableMap.Builder;
import com.google.common.collect.ImmutableSet;
Expand Down Expand Up @@ -76,9 +77,24 @@ public static Properties asProperties(final Map<String, ?> mapProps) {
* @return an immutable map of the loaded properties.
*/
public static Map<String, String> loadProperties(final File propertiesFile) {
final Map<String, String> properties = loadPropsFromFile(propertiesFile);
return loadProperties(ImmutableList.of(propertiesFile));
}

/**
* Load a list of property files. Properties are loaded from the first entry
* in the list to the last, meaning that any properties specified in later
* files take precedence.
*
* @param propertiesFiles the property files to load.
* @return an immutable map of the loaded properties.
*/
public static Map<String, String> loadProperties(final List<File> propertiesFiles) {
final Map<String, String> properties = new HashMap<>();
for (final File propertiesFile : propertiesFiles) {
properties.putAll(loadPropsFromFile(propertiesFile));
}
throwOnBlackListedProperties(properties);
return properties;
return ImmutableMap.copyOf(properties);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,12 @@
import static org.hamcrest.Matchers.not;
import static org.junit.Assert.assertThrows;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.confluent.ksql.test.util.KsqlTestFolder;
import io.confluent.ksql.util.KsqlException;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.util.Map;
Expand Down Expand Up @@ -69,6 +71,28 @@ public void shouldLoadPropsFromFile() {
assertThat(result.get("some.other.prop"), is("124"));
}

@Test
public void shouldLoadPropsFromMultipleFiles() throws IOException {
// Given:
givenPropsFileContains(
"# Comment" + System.lineSeparator()
+ "some.prop=some value" + System.lineSeparator()
+ "some.other.prop=124" + System.lineSeparator()
);
File other = TMP.newFile();
givenPropsFileContains(other, "some.prop=other value" + System.lineSeparator() + "a=b");

// When:
final Map<String, String> result = PropertiesUtil.loadProperties(
ImmutableList.of(propsFile, other)
);

// Then:
assertThat(result.get("some.prop"), is("other value"));
assertThat(result.get("some.other.prop"), is("124"));
assertThat(result.get("a"), is("b"));
}

@Test
public void shouldLoadImmutablePropsFromFile() {
// Given:
Expand Down Expand Up @@ -217,6 +241,10 @@ public void shouldNotThrowOnUnkownPropertyFromCoerceTypesWithIgnore() {
}

private void givenPropsFileContains(final String contents) {
givenPropsFileContains(propsFile, contents);
}

private void givenPropsFileContains(final File propsFile, final String contents) {
try {
Files.write(propsFile.toPath(), contents.getBytes(StandardCharsets.UTF_8));
} catch (final Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import java.util.function.Function;
import org.slf4j.Logger;

class LoggingRateLimiter {
public class LoggingRateLimiter {
// Print "You hit a rate limit" every 5 seconds
private static final double LIMIT_HIT_LOG_RATE = 0.2;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* Copyright 2022 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.api.server;


import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import io.confluent.ksql.api.util.ApiServerUtils;
import io.confluent.ksql.rest.server.state.ServerState;
import io.vertx.core.AbstractVerticle;
import io.vertx.core.Promise;
import io.vertx.core.http.HttpMethod;
import io.vertx.core.http.HttpServer;
import io.vertx.core.http.HttpServerOptions;
import io.vertx.ext.web.Router;
import java.util.Objects;

/**
* The server deploys multiple server verticles. This is where the HTTP2 requests are handled. The
* actual implementation of the endpoints is provided by an implementation of {@code Endpoints}.
*/
public class PreconditionVerticle extends AbstractVerticle {
private final HttpServerOptions httpServerOptions;
private final ServerState serverState;
private HttpServer httpServer;

@SuppressFBWarnings(value = "EI_EXPOSE_REP2")
public PreconditionVerticle(
final HttpServerOptions httpServerOptions,
final ServerState serverState
) {
this.httpServerOptions = Objects.requireNonNull(httpServerOptions);
this.serverState = Objects.requireNonNull(serverState, "serverState");
}

@Override
public void start(final Promise<Void> startPromise) {
httpServer = vertx.createHttpServer(httpServerOptions).requestHandler(setupRouter())
.exceptionHandler(ApiServerUtils::unhandledExceptionHandler);
httpServer.listen(ar -> {
if (ar.succeeded()) {
startPromise.complete();
} else {
startPromise.fail(ar.cause());
}
});
}

@Override
public void stop(final Promise<Void> stopPromise) {
if (httpServer == null) {
stopPromise.complete();
} else {
httpServer.close(stopPromise.future());
}
}

private Router setupRouter() {
final Router router = Router.router(vertx);
router.route(HttpMethod.GET, "/chc/ready").handler(ApiServerUtils::chcHandler);
router.route(HttpMethod.GET, "/chc/live").handler(ApiServerUtils::chcHandler);
router.route().handler(new ServerStateHandler(serverState));
router.route().failureHandler(new FailureHandler());
return router;
}
}
120 changes: 5 additions & 115 deletions ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/Server.java
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright 2020 Confluent Inc.
* Copyright 2022 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"); you may not use
* this file except in compliance with the License. You may obtain a copy of the
Expand All @@ -21,25 +21,22 @@
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import io.confluent.ksql.api.auth.AuthenticationPlugin;
import io.confluent.ksql.api.spi.Endpoints;
import io.confluent.ksql.api.util.ApiServerUtils;
import io.confluent.ksql.internal.PullQueryExecutorMetrics;
import io.confluent.ksql.properties.PropertiesUtil;
import io.confluent.ksql.rest.entity.PushQueryId;
import io.confluent.ksql.rest.server.KsqlRestConfig;
import io.confluent.ksql.rest.server.state.ServerState;
import io.confluent.ksql.security.KsqlSecurityExtension;
import io.confluent.ksql.util.FileWatcher;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.VertxCompletableFuture;
import io.confluent.ksql.util.VertxSslOptionsFactory;
import io.netty.handler.ssl.OpenSsl;
import io.vertx.core.Vertx;
import io.vertx.core.WorkerExecutor;
import io.vertx.core.http.ClientAuth;
import io.vertx.core.http.HttpConnection;
import io.vertx.core.http.HttpServerOptions;
import io.vertx.core.impl.ConcurrentHashSet;
import io.vertx.core.net.JksOptions;
import io.vertx.core.net.PfxOptions;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.Path;
Expand All @@ -54,7 +51,6 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.config.SslConfigs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -118,7 +114,7 @@ public synchronized void start() {
final LoggingRateLimiter loggingRateLimiter = new LoggingRateLimiter(config);
configureTlsCertReload(config);

final List<URI> listenUris = parseListeners(config);
final List<URI> listenUris = ApiServerUtils.parseListeners(config);
final Optional<URI> internalListenUri = parseInternalListener(config, listenUris);
final List<URI> allListenUris = new ArrayList<>(listenUris);
internalListenUri.ifPresent(allListenUris::add);
Expand Down Expand Up @@ -311,130 +307,24 @@ private static HttpServerOptions createHttpServerOptions(final KsqlRestConfig ks
: ksqlRestConfig.getClientAuth();

final String alias = ksqlRestConfig.getString(ksConfigName);
setTlsOptions(ksqlRestConfig, options, alias, clientAuth);
ApiServerUtils.setTlsOptions(ksqlRestConfig, options, alias, clientAuth);
}
return options;
}

private static void setTlsOptions(
final KsqlRestConfig ksqlRestConfig,
final HttpServerOptions options,
final String keyStoreAlias,
final ClientAuth clientAuth
) {
options.setUseAlpn(true).setSsl(true);
if (ksqlRestConfig.getBoolean(KsqlRestConfig.KSQL_SERVER_SNI_CHECK_ENABLE)) {
options.setSni(true);
}

configureTlsKeyStore(ksqlRestConfig, options, keyStoreAlias);
configureTlsTrustStore(ksqlRestConfig, options);

final List<String> enabledProtocols =
ksqlRestConfig.getList(KsqlRestConfig.SSL_ENABLED_PROTOCOLS_CONFIG);
if (!enabledProtocols.isEmpty()) {
options.setEnabledSecureTransportProtocols(new HashSet<>(enabledProtocols));
}

final List<String> cipherSuites =
ksqlRestConfig.getList(KsqlRestConfig.SSL_CIPHER_SUITES_CONFIG);
if (!cipherSuites.isEmpty()) {
// Vert.x does not yet support a method for setting cipher suites, so we use the following
// workaround instead. See https://github.com/eclipse-vertx/vert.x/issues/1507.
final Set<String> enabledCipherSuites = options.getEnabledCipherSuites();
enabledCipherSuites.clear();
enabledCipherSuites.addAll(cipherSuites);
}

options.setClientAuth(clientAuth);
}

private static void configureTlsKeyStore(
final KsqlRestConfig ksqlRestConfig,
final HttpServerOptions httpServerOptions,
final String keyStoreAlias
) {
final Map<String, String> props = PropertiesUtil.toMapStrings(ksqlRestConfig.originals());
final String keyStoreType = ksqlRestConfig.getString(KsqlRestConfig.SSL_KEYSTORE_TYPE_CONFIG);

if (keyStoreType.equals(KsqlRestConfig.SSL_STORE_TYPE_JKS)) {
final Optional<JksOptions> keyStoreOptions =
VertxSslOptionsFactory.buildJksKeyStoreOptions(props, Optional.ofNullable(keyStoreAlias));

keyStoreOptions.ifPresent(options -> httpServerOptions.setKeyStoreOptions(options));
} else if (keyStoreType.equals(KsqlRestConfig.SSL_STORE_TYPE_PKCS12)) {
final Optional<PfxOptions> keyStoreOptions =
VertxSslOptionsFactory.getPfxKeyStoreOptions(props);

keyStoreOptions.ifPresent(options -> httpServerOptions.setPfxKeyCertOptions(options));
}
}

private static void configureTlsTrustStore(
final KsqlRestConfig ksqlRestConfig,
final HttpServerOptions httpServerOptions
) {
final Map<String, String> props = PropertiesUtil.toMapStrings(ksqlRestConfig.originals());
final String trustStoreType =
ksqlRestConfig.getString(KsqlRestConfig.SSL_TRUSTSTORE_TYPE_CONFIG);

if (trustStoreType.equals(KsqlRestConfig.SSL_STORE_TYPE_JKS)) {
final Optional<JksOptions> trustStoreOptions =
VertxSslOptionsFactory.getJksTrustStoreOptions(props);

trustStoreOptions.ifPresent(options -> httpServerOptions.setTrustOptions(options));
} else if (trustStoreType.equals(KsqlRestConfig.SSL_STORE_TYPE_PKCS12)) {
final Optional<PfxOptions> trustStoreOptions =
VertxSslOptionsFactory.getPfxTrustStoreOptions(props);

trustStoreOptions.ifPresent(options -> httpServerOptions.setTrustOptions(options));
}
}

private static List<URI> parseListeners(final KsqlRestConfig config) {
final List<String> sListeners = config.getList(KsqlRestConfig.LISTENERS_CONFIG);
return parseListenerStrings(config, sListeners);
}

private static Optional<URI> parseInternalListener(
final KsqlRestConfig config,
final List<URI> listenUris
) {
if (config.getString(KsqlRestConfig.INTERNAL_LISTENER_CONFIG) == null) {
return Optional.empty();
}
final URI uri = parseListenerStrings(config,
final URI uri = ApiServerUtils.parseListenerStrings(config,
ImmutableList.of(config.getString(KsqlRestConfig.INTERNAL_LISTENER_CONFIG))).get(0);
if (listenUris.contains(uri)) {
return Optional.empty();
} else {
return Optional.of(uri);
}
}

private static List<URI> parseListenerStrings(
final KsqlRestConfig config,
final List<String> stringListeners) {
final List<URI> listeners = new ArrayList<>();
for (String listenerName : stringListeners) {
try {
final URI uri = new URI(listenerName);
final String scheme = uri.getScheme();
if (!"http".equalsIgnoreCase(scheme) && !"https".equalsIgnoreCase(scheme)) {
throw new ConfigException("Invalid URI scheme should be http or https: " + listenerName);
}
if ("https".equalsIgnoreCase(scheme)) {
final String keyStoreLocation = config
.getString(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG);
if (keyStoreLocation == null || keyStoreLocation.isEmpty()) {
throw new ConfigException("https listener specified but no keystore provided");
}
}
listeners.add(uri);
} catch (URISyntaxException e) {
throw new ConfigException("Invalid listener URI: " + listenerName);
}
}
return listeners;
}
}
Loading

0 comments on commit 33a6a04

Please sign in to comment.