diff --git a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberBlackboxVerificationTest.java b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberBlackboxVerificationTest.java index a9e698193b67..75ef296218ef 100644 --- a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberBlackboxVerificationTest.java +++ b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberBlackboxVerificationTest.java @@ -15,7 +15,7 @@ package io.confluent.ksql.api.tck; -import io.confluent.ksql.api.server.BaseSubscriber; +import io.confluent.ksql.reactive.BaseSubscriber; import io.vertx.core.Context; import io.vertx.core.Vertx; import io.vertx.core.json.JsonObject; diff --git a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberWhiteboxVerificationTest.java b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberWhiteboxVerificationTest.java index bc4cb79fab3b..29d9c1e1b9f8 100644 --- a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberWhiteboxVerificationTest.java +++ b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BaseSubscriberWhiteboxVerificationTest.java @@ -15,7 +15,7 @@ package io.confluent.ksql.api.tck; -import io.confluent.ksql.api.server.BaseSubscriber; +import io.confluent.ksql.reactive.BaseSubscriber; import io.vertx.core.Context; import io.vertx.core.Vertx; import io.vertx.core.json.JsonObject; diff --git a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BufferedPublisherVerificationTest.java b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BufferedPublisherVerificationTest.java index 676cfae1bdd2..9418e5594066 100644 --- a/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BufferedPublisherVerificationTest.java +++ b/ksql-api-reactive-streams-tck/src/test/java/io/confluent/ksql/api/tck/BufferedPublisherVerificationTest.java @@ -15,7 +15,7 @@ package io.confluent.ksql.api.tck; -import io.confluent.ksql.api.server.BufferedPublisher; +import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.core.Context; import io.vertx.core.Vertx; import io.vertx.core.json.JsonObject; diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java index bb066d945cba..7c546d411b48 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java @@ -15,8 +15,6 @@ package io.confluent.ksql.cli; -import static java.nio.charset.StandardCharsets.UTF_8; - import io.confluent.ksql.cli.console.Console; import io.confluent.ksql.cli.console.KsqlTerminal.StatusClosable; import io.confluent.ksql.cli.console.OutputFormat; @@ -31,10 +29,11 @@ import io.confluent.ksql.parser.SqlBaseParser.SetPropertyContext; import io.confluent.ksql.parser.SqlBaseParser.StatementContext; import io.confluent.ksql.parser.SqlBaseParser.UnsetPropertyContext; +import io.confluent.ksql.reactive.BaseSubscriber; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.client.KsqlRestClient; -import io.confluent.ksql.rest.client.QueryStream; import io.confluent.ksql.rest.client.RestResponse; +import io.confluent.ksql.rest.client.StreamPublisher; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatusEntity; import io.confluent.ksql.rest.entity.KsqlEntity; @@ -47,26 +46,20 @@ import io.confluent.ksql.util.ParserUtil; import io.confluent.ksql.util.Version; import io.confluent.ksql.util.WelcomeMsgUtils; +import io.vertx.core.Context; import java.io.Closeable; -import java.io.InputStream; import java.io.PrintWriter; import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.Scanner; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.BiFunction; import java.util.function.Supplier; -import org.apache.commons.compress.utils.IOUtils; import org.jline.reader.EndOfFileException; import org.jline.reader.UserInterruptException; import org.jline.terminal.Terminal; +import org.reactivestreams.Subscription; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,8 +77,6 @@ public class Cli implements KsqlRequestExecutor, Closeable { .put(UnsetPropertyContext.class, Cli::unsetPropertyFromCtxt) .build(); - private final ExecutorService queryStreamExecutorService; - private final Long streamedQueryRowLimit; private final Long streamedQueryTimeoutMs; @@ -116,7 +107,6 @@ public static Cli build( this.streamedQueryTimeoutMs = streamedQueryTimeoutMs; this.restClient = restClient; this.terminal = terminal; - this.queryStreamExecutorService = Executors.newSingleThreadExecutor(); this.remoteServerState = new RemoteServerState(); final Supplier versionSuppler = @@ -220,7 +210,6 @@ private void displayWelcomeMessage() { @Override public void close() { - queryStreamExecutorService.shutdownNow(); terminal.close(); } @@ -310,68 +299,44 @@ private void printKsqlResponse(final RestResponse response) { } } - private void streamResults(final QueryStream queryStream) { - final Future queryStreamFuture = queryStreamExecutorService.submit(() -> { - for (long rowsRead = 0; limitNotReached(rowsRead) && queryStream.hasNext(); ) { - final StreamedRow row = queryStream.next(); - - terminal.printStreamedRow(row); - if (row.isTerminal()) { - break; - } - - if (row.getRow().isPresent()) { - rowsRead++; - } - } - }); - - terminal.handle(Terminal.Signal.INT, signal -> { - terminal.handle(Terminal.Signal.INT, Terminal.SignalHandler.SIG_IGN); - queryStream.close(); - }); - - try { - if (streamedQueryTimeoutMs != null) { - try { - queryStreamFuture.get(streamedQueryTimeoutMs, TimeUnit.MILLISECONDS); - } catch (final TimeoutException exception) { - queryStream.close(); - } - } - - queryStreamFuture.get(); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } catch (final ExecutionException e) { - if (e.getCause() instanceof RuntimeException) { - throw (RuntimeException)e.getCause(); - } - throw new RuntimeException(e.getCause()); - } finally { - terminal.writer().println("Query terminated"); - terminal.flush(); - } - } - - private boolean limitNotReached(final long rowsRead) { - return streamedQueryRowLimit == null || rowsRead < streamedQueryRowLimit; - } - @SuppressWarnings({"try", "unused"}) // ignored param is required to compile. private void handleQuery( final String statement, final SqlBaseParser.QueryStatementContext query ) { - final RestResponse queryResponse = - makeKsqlRequest(statement, restClient::makeQueryRequest); + final RestResponse> queryResponse = + makeKsqlRequest(statement, restClient::makeQueryRequestStreamed); if (!queryResponse.isSuccessful()) { terminal.printErrorMessage(queryResponse.getErrorMessage()); + terminal.flush(); } else { - try (QueryStream queryStream = queryResponse.getResponse(); - StatusClosable toClose = terminal.setStatusMessage("Press CTRL-C to interrupt")) { - streamResults(queryStream); + try (StatusClosable toClose = terminal.setStatusMessage("Press CTRL-C to interrupt")) { + final StreamPublisher publisher = queryResponse.getResponse(); + final CompletableFuture future = new CompletableFuture<>(); + final QueryStreamSubscriber subscriber = new QueryStreamSubscriber(publisher.getContext(), + future); + publisher.subscribe(subscriber); + + terminal.handle(Terminal.Signal.INT, signal -> { + terminal.handle(Terminal.Signal.INT, Terminal.SignalHandler.SIG_IGN); + subscriber.close(); + future.complete(null); + }); + + try { + if (streamedQueryTimeoutMs != null) { + future.get(streamedQueryTimeoutMs, TimeUnit.MILLISECONDS); + } else { + future.get(); + } + } catch (Exception e) { + LOGGER.error("Unexpected exception in waiting for query", e); + } finally { + terminal.writer().println("Query terminated"); + terminal.flush(); + publisher.close(); + } } } } @@ -381,42 +346,37 @@ private void handlePrintedTopic( final String printTopic, final SqlBaseParser.PrintTopicContext ignored ) { - final RestResponse topicResponse = + final RestResponse> topicResponse = makeKsqlRequest(printTopic, restClient::makePrintTopicRequest); if (topicResponse.isSuccessful()) { - try (Scanner topicStreamScanner = new Scanner(topicResponse.getResponse(), UTF_8.name()); - StatusClosable toClose = terminal.setStatusMessage("Press CTRL-C to interrupt") - ) { - final Future topicPrintFuture = queryStreamExecutorService.submit(() -> { - while (!Thread.currentThread().isInterrupted() && topicStreamScanner.hasNextLine()) { - final String line = topicStreamScanner.nextLine(); - if (!line.isEmpty()) { - terminal.writer().println(line); - terminal.flush(); - } - } - }); + + try (StatusClosable toClose = terminal.setStatusMessage("Press CTRL-C to interrupt")) { + final CompletableFuture future = new CompletableFuture<>(); + final StreamPublisher publisher = topicResponse.getResponse(); + final PrintTopicSubscriber subscriber = new PrintTopicSubscriber(publisher.getContext(), + future); + publisher.subscribe(subscriber); terminal.handle(Terminal.Signal.INT, signal -> { terminal.handle(Terminal.Signal.INT, Terminal.SignalHandler.SIG_IGN); - topicPrintFuture.cancel(true); + subscriber.close(); + future.complete(null); }); try { - topicPrintFuture.get(); - } catch (final CancellationException exception) { - IOUtils.closeQuietly(topicResponse.getResponse()); - terminal.writer().println("Topic printing ceased"); - terminal.flush(); - } catch (final Exception e) { - throw new RuntimeException(e); + future.get(); + } catch (Exception e) { + LOGGER.error("Unexpected exception in waiting for print topic completion", e); + } finally { + publisher.close(); } + terminal.writer().println("Topic printing ceased"); } } else { terminal.writer().println(topicResponse.getErrorMessage().getMessage()); - terminal.flush(); } + terminal.flush(); } @SuppressWarnings("unused") @@ -511,4 +471,105 @@ private void setRequestPipelining(final boolean newSetting) { requestPipelining = newSetting; } } + + private class QueryStreamSubscriber extends BaseSubscriber { + + private final CompletableFuture future; + private boolean closed; + private long rowsRead; + + QueryStreamSubscriber(final Context context, final CompletableFuture future) { + super(context); + this.future = Objects.requireNonNull(future); + } + + @Override + protected void afterSubscribe(final Subscription subscription) { + makeRequest(1); + } + + @Override + protected synchronized void handleValue(final StreamedRow row) { + if (closed) { + return; + } + terminal.printStreamedRow(row); + terminal.flush(); + if (row.isTerminal()) { + future.complete(null); + close(); + return; + } + if (row.getRow().isPresent()) { + rowsRead++; + if (streamedQueryRowLimit != null && streamedQueryRowLimit == rowsRead) { + future.complete(null); + close(); + return; + } + } + makeRequest(1); + } + + @Override + protected void handleComplete() { + future.complete(null); + } + + @Override + protected void handleError(final Throwable t) { + future.completeExceptionally(t); + } + + synchronized void close() { + closed = true; + context.runOnContext(v -> cancel()); + } + } + + private class PrintTopicSubscriber extends BaseSubscriber { + + private final CompletableFuture future; + private boolean closed; + + PrintTopicSubscriber(final Context context, final CompletableFuture future) { + super(context); + this.future = Objects.requireNonNull(future); + } + + @Override + protected void afterSubscribe(final Subscription subscription) { + makeRequest(1); + } + + @Override + protected synchronized void handleValue(final String line) { + if (closed) { + return; + } + if (line.isEmpty()) { + // Ignore it - the server can insert these + } else { + terminal.writer().println(line); + terminal.flush(); + makeRequest(1); + } + } + + @Override + protected void handleComplete() { + future.complete(null); + } + + @Override + protected void handleError(final Throwable t) { + future.completeExceptionally(t); + } + + synchronized void close() { + closed = true; + context.runOnContext(v -> cancel()); + } + } + } diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java index d7d369651d39..ffaadd612303 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java @@ -163,7 +163,6 @@ public class CliTest { private static KsqlRestClient restClient; - private Console console; private TestTerminal terminal; private TestRowCaptor rowCaptor; @@ -402,8 +401,9 @@ public void shouldPrintTopicWithJsonValue() { // Then: assertThatEventually(() -> terminal.getOutputString(), containsString("Value format: JSON")); - assertThat(terminal.getOutputString(), containsString("Key format: KAFKA_BIGINT or KAFKA_DOUBLE")); - assertThat(terminal.getOutputString(), containsString("," + assertThatEventually(() -> terminal.getOutputString(), + containsString("Key format: KAFKA_BIGINT or KAFKA_DOUBLE")); + assertThatEventually(() -> terminal.getOutputString(), containsString("," + " key: 1, " + "value: {" + "\"ORDERTIME\":1," @@ -422,10 +422,12 @@ public void shouldPrintTopicWithDelimitedValue() { run("print " + DELIMITED_TOPIC + " FROM BEGINNING INTERVAL 1 LIMIT 2;", localCli); // Then: - assertThatEventually(() -> terminal.getOutputString(), containsString("Value format: KAFKA_STRING")); + assertThatEventually(() -> terminal.getOutputString(), + containsString("Value format: KAFKA_STRING")); assertThat(terminal.getOutputString(), containsString("Key format: KAFKA_STRING")); assertThat(terminal.getOutputString(), containsString(", key: , value: ")); - assertThat(terminal.getOutputString(), containsString(", key: ITEM_1, value: ITEM_1,home cinema")); + assertThat(terminal.getOutputString(), + containsString(", key: ITEM_1, value: ITEM_1,home cinema")); } @Test diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/SslFunctionalTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/SslFunctionalTest.java index 12333f394ab9..00f31dc8a741 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/SslFunctionalTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/SslFunctionalTest.java @@ -30,6 +30,7 @@ import io.confluent.common.utils.IntegrationTest; import io.confluent.ksql.integration.IntegrationTestHarness; import io.confluent.ksql.integration.Retry; +import io.confluent.ksql.rest.client.KsqlClient; import io.confluent.ksql.rest.client.KsqlRestClient; import io.confluent.ksql.rest.client.KsqlRestClientException; import io.confluent.ksql.rest.client.RestResponse; @@ -40,14 +41,14 @@ import io.confluent.rest.RestConfig; import java.io.EOFException; import java.net.URI; -import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLHandshakeException; -import javax.ws.rs.ProcessingException; import kafka.zookeeper.ZooKeeperClientException; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.http.HttpStatus; @@ -109,15 +110,20 @@ public static void classSetUp() { @Before public void setUp() { - clientProps = Collections.emptyMap(); + clientProps = new HashMap<>(); + clientProps.put(KsqlClient.DISABLE_HOSTNAME_VERIFICATION_PROP_NAME, "true"); sslContextFactory = new Server(); } @Test public void shouldNotBeAbleToUseCliIfClientDoesNotTrustServerCert() { + + // Given: + givenClientConfguredWithoutTruststore(); + // Then: expectedException.expect(KsqlRestClientException.class); - expectedException.expectCause(is(instanceOf(ProcessingException.class))); + expectedException.expectCause(is(instanceOf(ExecutionException.class))); expectedException.expectCause(hasCause(is(instanceOf(SSLHandshakeException.class)))); // When: @@ -138,6 +144,9 @@ public void shouldBeAbleToUseCliOverHttps() { @Test public void shouldNotBeAbleToUseWssIfClientDoesNotTrustServerCert() throws Exception { + // Given: + givenClientConfguredWithoutTruststore(); + // Then: expectedException.expect(either( both(hasCause(hasCause(hasMessage( @@ -164,7 +173,10 @@ public void shouldBeAbleToUseWss() throws Exception { private void givenTrustStoreConfigured() { // HTTP: - clientProps = ClientTrustStore.trustStoreProps(); + clientProps = new HashMap<>(); + clientProps.putAll(ClientTrustStore.trustStoreProps()); + clientProps.put(KsqlClient.DISABLE_HOSTNAME_VERIFICATION_PROP_NAME, "true"); + clientProps.put(KsqlClient.TLS_ENABLED_PROP_NAME, "true"); // WS: sslContextFactory.setTrustStorePath(ClientTrustStore.trustStorePath()); @@ -172,6 +184,12 @@ private void givenTrustStoreConfigured() { sslContextFactory.setEndpointIdentificationAlgorithm(""); } + private void givenClientConfguredWithoutTruststore() { + clientProps = new HashMap<>(); + clientProps.put(KsqlClient.DISABLE_HOSTNAME_VERIFICATION_PROP_NAME, "true"); + clientProps.put(KsqlClient.TLS_ENABLED_PROP_NAME, "true"); + } + private Code canMakeCliRequest() { final String serverAddress = REST_APP.getHttpsListener().toString(); diff --git a/ksql-cli/src/test/resources/log4j.properties b/ksql-cli/src/test/resources/log4j.properties index 0d7ed64eb30e..964c2476a54b 100644 --- a/ksql-cli/src/test/resources/log4j.properties +++ b/ksql-cli/src/test/resources/log4j.properties @@ -21,5 +21,4 @@ log4j.rootLogger=WARN, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n - log4j.logger.io.confluent.ksql.cli.Cli=DEBUG \ No newline at end of file diff --git a/ksql-common/pom.xml b/ksql-common/pom.xml index bcc9cb5ce8b9..8d13dfa545ad 100644 --- a/ksql-common/pom.xml +++ b/ksql-common/pom.xml @@ -15,8 +15,8 @@ --> + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -39,11 +39,11 @@ guava - - io.confluent - kafka-connect-avro-converter - ${confluent.version} - + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + io.confluent @@ -93,6 +93,24 @@ test + + io.vertx + vertx-core + ${vertx.version} + + + + io.vertx + vertx-codegen + ${vertx.version} + + + + org.reactivestreams + reactive-streams + ${reactive-streams.version} + + diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BasePublisher.java b/ksql-common/src/main/java/io/confluent/ksql/reactive/BasePublisher.java similarity index 95% rename from ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BasePublisher.java rename to ksql-common/src/main/java/io/confluent/ksql/reactive/BasePublisher.java index 88545de099bf..2e4513dde699 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BasePublisher.java +++ b/ksql-common/src/main/java/io/confluent/ksql/reactive/BasePublisher.java @@ -13,9 +13,9 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.server; +package io.confluent.ksql.reactive; -import io.confluent.ksql.api.impl.Utils; +import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import java.util.Objects; import org.reactivestreams.Publisher; @@ -50,7 +50,7 @@ public BasePublisher(final Context ctx) { @Override public void subscribe(final Subscriber subscriber) { Objects.requireNonNull(subscriber); - if (Utils.isEventLoopAndSameContext(ctx)) { + if (VertxUtils.isEventLoopAndSameContext(ctx)) { doSubscribe(subscriber); } else { ctx.runOnContext(v -> doSubscribe(subscriber)); @@ -61,8 +61,12 @@ public void close() { ctx.runOnContext(v -> doClose()); } + public Context getContext() { + return ctx; + } + protected void checkContext() { - Utils.checkContext(ctx); + VertxUtils.checkContext(ctx); } protected final void sendError(final Exception e) { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BaseSubscriber.java b/ksql-common/src/main/java/io/confluent/ksql/reactive/BaseSubscriber.java similarity index 97% rename from ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BaseSubscriber.java rename to ksql-common/src/main/java/io/confluent/ksql/reactive/BaseSubscriber.java index 74eb898e87eb..a1b4997d31c1 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BaseSubscriber.java +++ b/ksql-common/src/main/java/io/confluent/ksql/reactive/BaseSubscriber.java @@ -13,9 +13,9 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.server; +package io.confluent.ksql.reactive; -import io.confluent.ksql.api.impl.Utils; +import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import java.util.Objects; import org.reactivestreams.Subscriber; @@ -144,11 +144,11 @@ protected final void complete() { } protected final void checkContext() { - Utils.checkContext(context); + VertxUtils.checkContext(context); } private void runOnRightContext(final Runnable runnable) { - if (Utils.isEventLoopAndSameContext(context)) { + if (VertxUtils.isEventLoopAndSameContext(context)) { // Execute directly runnable.run(); } else { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BufferedPublisher.java b/ksql-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java similarity index 99% rename from ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BufferedPublisher.java rename to ksql-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java index f3062e9f63b9..da283b93a444 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/BufferedPublisher.java +++ b/ksql-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.server; +package io.confluent.ksql.reactive; import io.vertx.core.Context; import java.util.ArrayDeque; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/VertxCompletableFuture.java b/ksql-common/src/main/java/io/confluent/ksql/util/VertxCompletableFuture.java similarity index 97% rename from ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/VertxCompletableFuture.java rename to ksql-common/src/main/java/io/confluent/ksql/util/VertxCompletableFuture.java index 98814f0b73b7..0aa575052e7c 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/VertxCompletableFuture.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/VertxCompletableFuture.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.impl; +package io.confluent.ksql.util; import io.vertx.core.AsyncResult; import io.vertx.core.Handler; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/Utils.java b/ksql-common/src/main/java/io/confluent/ksql/util/VertxUtils.java similarity index 95% rename from ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/Utils.java rename to ksql-common/src/main/java/io/confluent/ksql/util/VertxUtils.java index 14bdb83d93d8..84271e86ef6a 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/impl/Utils.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/VertxUtils.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.impl; +package io.confluent.ksql.util; import io.vertx.core.Context; import io.vertx.core.Future; @@ -24,9 +24,9 @@ * General purpose utils (not limited to the server, could be used by client too) for the API * module. */ -public final class Utils { +public final class VertxUtils { - private Utils() { + private VertxUtils() { } /** diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/BufferedPublisherTest.java b/ksql-common/src/test/java/io/confluent/ksql/reactive/BufferedPublisherTest.java similarity index 97% rename from ksql-rest-app/src/test/java/io/confluent/ksql/api/BufferedPublisherTest.java rename to ksql-common/src/test/java/io/confluent/ksql/reactive/BufferedPublisherTest.java index a1776e083575..852abb6d0bd5 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/BufferedPublisherTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/reactive/BufferedPublisherTest.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api; +package io.confluent.ksql.reactive; import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; import static org.hamcrest.MatcherAssert.assertThat; @@ -21,9 +21,7 @@ import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.fail; -import io.confluent.ksql.api.server.BufferedPublisher; -import io.confluent.ksql.api.utils.AsyncAssert; -import io.confluent.ksql.api.utils.TestUtils; +import io.confluent.ksql.test.util.AsyncAssert; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import org.junit.Test; @@ -64,7 +62,7 @@ public void shouldNotAllowSettingDrainHandlerMoreThanOnce() throws Exception { latch.countDown(); } }); - TestUtils.awaitLatch(latch); + awaitLatch(latch); } @Test diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/PublisherTestBase.java b/ksql-common/src/test/java/io/confluent/ksql/reactive/PublisherTestBase.java similarity index 96% rename from ksql-rest-app/src/test/java/io/confluent/ksql/api/PublisherTestBase.java rename to ksql-common/src/test/java/io/confluent/ksql/reactive/PublisherTestBase.java index 0ab2b788391f..bd773c10d61e 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/PublisherTestBase.java +++ b/ksql-common/src/test/java/io/confluent/ksql/reactive/PublisherTestBase.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api; +package io.confluent.ksql.reactive; import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; import static org.hamcrest.MatcherAssert.assertThat; @@ -24,12 +24,12 @@ import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.core.IsInstanceOf.instanceOf; -import io.confluent.ksql.api.utils.TestUtils; import io.vertx.core.Context; import io.vertx.core.Vertx; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -191,7 +191,7 @@ protected void execOnContextAndWait(Runnable action) throws Exception { action.run(); latch.countDown(); }); - TestUtils.awaitLatch(latch); + awaitLatch(latch); } protected void shouldDeliver(int numRequested, int numDelivered) throws Exception { @@ -211,6 +211,9 @@ public synchronized void onSubscribe(final Subscription sub) { assertThat(subscriber.getError(), is(nullValue())); } + protected static void awaitLatch(CountDownLatch latch) throws Exception { + assertThat(latch.await(2000, TimeUnit.MILLISECONDS), is(true)); + } protected static class TestSubscriber implements Subscriber { diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/ReactiveSubscriberTest.java b/ksql-common/src/test/java/io/confluent/ksql/reactive/ReactiveSubscriberTest.java similarity index 96% rename from ksql-rest-app/src/test/java/io/confluent/ksql/api/ReactiveSubscriberTest.java rename to ksql-common/src/test/java/io/confluent/ksql/reactive/ReactiveSubscriberTest.java index 9b272b1429b3..adf90d87b20f 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/ReactiveSubscriberTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/reactive/ReactiveSubscriberTest.java @@ -13,17 +13,16 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api; +package io.confluent.ksql.reactive; -import static io.confluent.ksql.api.utils.TestUtils.awaitLatch; import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; -import io.confluent.ksql.api.server.BaseSubscriber; import io.vertx.core.Context; import io.vertx.core.Vertx; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.junit.After; import org.junit.Before; @@ -148,6 +147,10 @@ private void checkCorrectContext(AtomicBoolean wrongContext, Context context) { } } + protected static void awaitLatch(CountDownLatch latch) throws Exception { + assertThat(latch.await(2000, TimeUnit.MILLISECONDS), is(true)); + } + private static class TestSubscription implements Subscription { private boolean cancelled; diff --git a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java index 679599df308a..94e08051cf82 100644 --- a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java +++ b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java @@ -322,11 +322,11 @@ private PlannedStatementIterator( ) { this.statements = requireNonNull(statements, "statements"); this.executionContext = requireNonNull(executionContext, "executionContext"); - this.sessionProperties = + this.sessionProperties = new SessionProperties( requireNonNull(overrides, "overrides"), new KsqlHostInfo("host", 50), - buildURL()); + buildUrl()); this.ksqlConfig = requireNonNull(ksqlConfig, "ksqlConfig"); this.stubKafkaService = requireNonNull(stubKafkaService, "stubKafkaService"); this.schemaInjector = requireNonNull(schemaInjector, "schemaInjector"); @@ -472,7 +472,7 @@ List getSources() { } } - private static URL buildURL() { + private static URL buildUrl() { try { return new URL("https://someHost:9876"); } catch (final MalformedURLException e) { diff --git a/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java b/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java index f35a963d97ae..671cb99ebe58 100644 --- a/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java +++ b/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java @@ -27,12 +27,10 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.json.JsonMapper; import io.confluent.ksql.rest.client.KsqlRestClient; -import io.confluent.ksql.rest.client.QueryStream; import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.KsqlEntity; import io.confluent.ksql.rest.entity.KsqlEntityList; @@ -272,11 +270,11 @@ private List sendQueryStatements( .collect(Collectors.toList()); } - private Optional sendQueryStatement( + private Optional> sendQueryStatement( final RestTestCase testCase, final String sql ) { - final RestResponse resp = restClient.makeQueryRequest(sql, null); + final RestResponse> resp = restClient.makeQueryRequest(sql, null); if (resp.isErroneous()) { handleErrorResponse(testCase, resp); @@ -474,7 +472,7 @@ private void waitForWarmStateStore( final long threshold = System.currentTimeMillis() + MAX_STATIC_WARM_UP.toMillis(); while (System.currentTimeMillis() < threshold) { - final RestResponse resp = restClient.makeQueryRequest(querySql, null); + final RestResponse> resp = restClient.makeQueryRequest(querySql, null); if (resp.isErroneous()) { final KsqlErrorMessage errorMessage = resp.getErrorMessage(); LOG.info("Server responded with an error code to a pull query. " @@ -551,17 +549,8 @@ static List admin(final KsqlEntityList adminResponses) { .collect(Collectors.toList()); } - static RqttResponse query(final QueryStream queryStream) { - final Builder responses = ImmutableList.builder(); - - while (queryStream.hasNext()) { - final StreamedRow row = queryStream.next(); - responses.add(row); - } - - queryStream.close(); - - return new RqttQueryResponse(responses.build()); + static RqttResponse query(final List rows) { + return new RqttQueryResponse(rows); } void verify( diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java index 673214a5c975..e8c7277743c7 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/json/KsqlTypesDeserializationModule.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Confluent Inc. + * Copyright 2020 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 diff --git a/ksql-rest-app/pom.xml b/ksql-rest-app/pom.xml index f738da5be8fa..6591838caeb3 100644 --- a/ksql-rest-app/pom.xml +++ b/ksql-rest-app/pom.xml @@ -35,8 +35,6 @@ ${main-class} true true - 3.8.5 - 1.0.3 2.0.29.Final diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/BlockingQueryPublisher.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/BlockingQueryPublisher.java index 804fee040f91..1044cd8137c2 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/BlockingQueryPublisher.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/BlockingQueryPublisher.java @@ -17,9 +17,9 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.server.BasePublisher; import io.confluent.ksql.api.server.PushQueryHandle; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.reactive.BasePublisher; import io.vertx.core.Context; import io.vertx.core.WorkerExecutor; import java.util.List; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/InsertsSubscriber.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/InsertsSubscriber.java index c6d77155cc8b..4ade921d12b3 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/InsertsSubscriber.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/InsertsSubscriber.java @@ -16,13 +16,12 @@ package io.confluent.ksql.api.plugin; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.impl.Utils; -import io.confluent.ksql.api.server.BaseSubscriber; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; import io.confluent.ksql.logging.processing.NoopProcessingLogContext; import io.confluent.ksql.metastore.model.DataSource; +import io.confluent.ksql.reactive.BaseSubscriber; +import io.confluent.ksql.reactive.BufferedPublisher; import io.confluent.ksql.schema.ksql.DefaultSqlValueCoercer; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.ksql.SqlValueCoercer; @@ -32,6 +31,7 @@ import io.confluent.ksql.serde.ValueSerdeFactory; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import io.vertx.core.WorkerExecutor; import io.vertx.core.json.JsonObject; @@ -160,7 +160,7 @@ protected void handleValue(final JsonObject jsonObject) { } private void handleResult(final InsertResult result) { - Utils.checkContext(context); + VertxUtils.checkContext(context); final boolean full = acksPublisher.accept(result); if (full) { if (!drainHandlerSet) { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/KsqlServerEndpoints.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/KsqlServerEndpoints.java index 8825a0817b8b..ef43dbe7ca3d 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/KsqlServerEndpoints.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/KsqlServerEndpoints.java @@ -16,7 +16,6 @@ package io.confluent.ksql.api.plugin; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.ksql.api.impl.Utils; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; import io.confluent.ksql.api.server.PushQueryHandle; @@ -43,6 +42,7 @@ import io.confluent.ksql.util.KsqlStatementException; import io.confluent.ksql.util.QueryMetadata; import io.confluent.ksql.util.ReservedInternalTopics; +import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import io.vertx.core.WorkerExecutor; import io.vertx.core.json.JsonObject; @@ -89,7 +89,7 @@ public QueryPublisher createQueryPublisher( final WorkerExecutor workerExecutor) { // Must be run on worker as all this stuff is slow - Utils.checkIsWorker(); + VertxUtils.checkIsWorker(); properties.put("auto.offset.reset", "earliest"); @@ -184,7 +184,7 @@ public InsertsStreamSubscriber createInsertsSubscriber(final String target, final JsonObject properties, final Subscriber acksSubscriber, final Context context, final WorkerExecutor workerExecutor) { - Utils.checkIsWorker(); + VertxUtils.checkIsWorker(); final ServiceContext serviceContext = createServiceContext(new DummyPrincipal()); final DataSource dataSource = getDataSource(ksqlEngine.getMetaStore(), SourceName.of(target)); diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/PullQueryPublisher.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/PullQueryPublisher.java index 37640df863e1..77e43542df67 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/PullQueryPublisher.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/plugin/PullQueryPublisher.java @@ -16,8 +16,8 @@ package io.confluent.ksql.api.plugin; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.core.Context; import java.util.ArrayList; import java.util.List; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/AcksSubscriber.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/AcksSubscriber.java index 8baa007dd434..60b6260258f1 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/AcksSubscriber.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/AcksSubscriber.java @@ -19,6 +19,7 @@ import io.confluent.ksql.api.server.protocol.InsertAck; import io.confluent.ksql.api.server.protocol.InsertError; +import io.confluent.ksql.reactive.BaseSubscriber; import io.vertx.core.Context; import io.vertx.core.http.HttpServerResponse; import java.util.Objects; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/ConnectionQueryManager.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/ConnectionQueryManager.java index 17b0f4db1192..9996c4a560ab 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/ConnectionQueryManager.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/ConnectionQueryManager.java @@ -15,8 +15,8 @@ package io.confluent.ksql.api.server; -import io.confluent.ksql.api.impl.Utils; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import io.vertx.core.Handler; import io.vertx.core.http.HttpConnection; @@ -56,7 +56,7 @@ private ConnectionQueries getConnectionQueries(final HttpServerRequest request) } private void checkContext() { - Utils.checkContext(context); + VertxUtils.checkContext(context); } private class ConnectionQueries implements Handler { diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/InsertsStreamHandler.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/InsertsStreamHandler.java index 4b5c8d0de6eb..fabe47043bdc 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/InsertsStreamHandler.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/InsertsStreamHandler.java @@ -18,10 +18,11 @@ import static io.confluent.ksql.api.server.QueryStreamHandler.DELIMITED_CONTENT_TYPE; import static io.confluent.ksql.api.server.ServerUtils.deserialiseObject; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.server.protocol.InsertError; import io.confluent.ksql.api.server.protocol.InsertsStreamArgs; import io.confluent.ksql.api.spi.Endpoints; +import io.confluent.ksql.reactive.BufferedPublisher; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.Context; import io.vertx.core.Handler; import io.vertx.core.WorkerExecutor; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QueryStreamHandler.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QueryStreamHandler.java index f2a4a9701512..52f41f88a3cb 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QueryStreamHandler.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QueryStreamHandler.java @@ -15,12 +15,12 @@ package io.confluent.ksql.api.server; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; import io.confluent.ksql.api.server.protocol.QueryStreamArgs; import io.confluent.ksql.api.spi.Endpoints; import io.confluent.ksql.api.spi.QueryPublisher; import io.confluent.ksql.util.KsqlStatementException; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.Context; import io.vertx.core.Handler; import io.vertx.core.json.JsonObject; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QuerySubscriber.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QuerySubscriber.java index e3f9fae623ef..74a1e9b38192 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QuerySubscriber.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/QuerySubscriber.java @@ -19,6 +19,7 @@ import io.confluent.ksql.GenericRow; import io.confluent.ksql.api.server.protocol.ErrorResponse; +import io.confluent.ksql.reactive.BaseSubscriber; import io.vertx.core.Context; import io.vertx.core.http.HttpServerResponse; import java.util.Objects; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/Server.java b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/Server.java index 936c66b71957..1ba02d103b47 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/Server.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/api/server/Server.java @@ -15,9 +15,9 @@ package io.confluent.ksql.api.server; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.spi.Endpoints; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.DeploymentOptions; import io.vertx.core.Vertx; import io.vertx.core.WorkerExecutor; diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java index bbc7d4af6ac0..d1df27264224 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/DefaultKsqlClient.java @@ -18,13 +18,10 @@ import static java.util.Objects.requireNonNull; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.properties.LocalProperties; import io.confluent.ksql.rest.client.KsqlClient; import io.confluent.ksql.rest.client.KsqlTarget; -import io.confluent.ksql.rest.client.QueryStream; import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.KsqlEntityList; @@ -33,6 +30,7 @@ import io.confluent.ksql.rest.entity.StreamedRow; import io.confluent.ksql.services.SimpleKsqlClient; import io.confluent.ksql.util.KsqlHostInfo; +import io.vertx.core.http.HttpClientOptions; import java.net.URI; import java.util.List; import java.util.Map; @@ -49,7 +47,8 @@ final class DefaultKsqlClient implements SimpleKsqlClient { new KsqlClient( ImmutableMap.of(), Optional.empty(), - new LocalProperties(ImmutableMap.of()) + new LocalProperties(ImmutableMap.of()), + createClientOptions() ) ); } @@ -84,21 +83,14 @@ public RestResponse> makeQueryRequest( .target(serverEndPoint) .properties(properties); - final RestResponse resp = getTarget(target, authHeader) + final RestResponse> resp = getTarget(target, authHeader) .postQueryRequest(sql, Optional.empty()); if (resp.isErroneous()) { return RestResponse.erroneous(resp.getStatusCode(), resp.getErrorMessage()); } - final QueryStream stream = resp.getResponse(); - - final Builder rows = ImmutableList.builder(); - while (stream.hasNext()) { - rows.add(stream.next()); - } - - return RestResponse.successful(resp.getStatusCode(), rows.build()); + return RestResponse.successful(resp.getStatusCode(), resp.getResponse()); } @Override @@ -137,4 +129,9 @@ private KsqlTarget getTarget(final KsqlTarget target, final Optional aut .map(target::authorizationHeader) .orElse(target); } + + private static HttpClientOptions createClientOptions() { + return new HttpClientOptions().setMaxPoolSize(100); + } + } diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java index 877d90893417..4bfa4ef9907a 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/services/ServerInternalKsqlClient.java @@ -17,7 +17,6 @@ import static java.util.Objects.requireNonNull; -import io.confluent.ksql.rest.client.KsqlClientUtil; import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.KsqlEntityList; @@ -33,6 +32,8 @@ import java.util.List; import java.util.Map; import javax.ws.rs.core.Response; +import org.eclipse.jetty.http.HttpStatus; +import org.eclipse.jetty.http.HttpStatus.Code; /** * A KSQL client implementation that sends requests to KsqlResource directly, rather than going @@ -61,11 +62,14 @@ public RestResponse makeKsqlRequest( ) { final KsqlRequest request = new KsqlRequest(sql, Collections.emptyMap(), null); final Response response = ksqlResource.handleKsqlStatements(securityContext, request); - return KsqlClientUtil.toRestResponse( - response, - KSQL_PATH, - r -> (KsqlEntityList) r.getEntity() - ); + + final Code statusCode = HttpStatus.getCode(response.getStatus()); + if (statusCode != Code.OK) { + // It always returns ok + throw new IllegalStateException("Unexpected failure"); + } + + return RestResponse.successful(statusCode, (KsqlEntityList) response.getEntity()); } @Override diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/ApiTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/ApiTest.java index 1f77a45f0032..7ee6d863cc32 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/ApiTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/ApiTest.java @@ -31,7 +31,6 @@ import static org.hamcrest.Matchers.nullValue; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.server.PushQueryId; import io.confluent.ksql.api.utils.InsertsResponse; import io.confluent.ksql.api.utils.ListRowGenerator; @@ -39,6 +38,7 @@ import io.confluent.ksql.api.utils.ReceiveStream; import io.confluent.ksql.api.utils.SendStream; import io.confluent.ksql.parser.exception.ParseFailedException; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.buffer.Buffer; import io.vertx.core.json.JsonArray; import io.vertx.core.json.JsonObject; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java index bacb9beb2b35..fe059aadd964 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java @@ -20,12 +20,12 @@ import static org.hamcrest.Matchers.is; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.server.ApiServerConfig; import io.confluent.ksql.api.server.Server; import io.confluent.ksql.api.utils.ListRowGenerator; import io.confluent.ksql.api.utils.QueryResponse; import io.confluent.ksql.api.utils.ReceiveStream; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.Vertx; import io.vertx.core.buffer.Buffer; import io.vertx.core.http.HttpVersion; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java index c529cec9a08d..3fe53eb82452 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java @@ -15,12 +15,12 @@ package io.confluent.ksql.api; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; import io.confluent.ksql.api.spi.Endpoints; import io.confluent.ksql.api.spi.QueryPublisher; import io.confluent.ksql.api.utils.RowGenerator; +import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.core.Context; import io.vertx.core.Vertx; import io.vertx.core.WorkerExecutor; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestInsertsSubscriber.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestInsertsSubscriber.java index 31463b26a35c..13b29f31b3a7 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestInsertsSubscriber.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestInsertsSubscriber.java @@ -15,10 +15,10 @@ package io.confluent.ksql.api; -import io.confluent.ksql.api.server.BaseSubscriber; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; +import io.confluent.ksql.reactive.BaseSubscriber; +import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.core.Context; import io.vertx.core.json.JsonObject; import java.util.ArrayList; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java index db50618f1fd8..ad224d7454d0 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java @@ -16,9 +16,9 @@ package io.confluent.ksql.api; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.server.BasePublisher; import io.confluent.ksql.api.spi.QueryPublisher; import io.confluent.ksql.api.utils.RowGenerator; +import io.confluent.ksql.reactive.BasePublisher; import io.vertx.core.Context; import java.util.List; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java index fb8afde6e66e..86e3c3a7bb5e 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/integration/ApiIntegrationTest.java @@ -25,7 +25,6 @@ import static org.hamcrest.Matchers.startsWith; import io.confluent.common.utils.IntegrationTest; -import io.confluent.ksql.api.impl.VertxCompletableFuture; import io.confluent.ksql.api.server.ErrorCodes; import io.confluent.ksql.api.utils.InsertsResponse; import io.confluent.ksql.api.utils.QueryResponse; @@ -40,6 +39,7 @@ import io.confluent.ksql.test.util.secure.Credentials; import io.confluent.ksql.test.util.secure.SecureKafkaHelper; import io.confluent.ksql.util.PageViewDataProvider; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.Vertx; import io.vertx.core.buffer.Buffer; import io.vertx.core.http.HttpVersion; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java index e3e97ac29567..5067c34e0361 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java @@ -15,12 +15,12 @@ package io.confluent.ksql.api.perf; -import io.confluent.ksql.api.server.BaseSubscriber; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; import io.confluent.ksql.api.spi.Endpoints; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.reactive.BaseSubscriber; +import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.codegen.annotations.Nullable; import io.vertx.core.Context; import io.vertx.core.Handler; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java index fe103e082e4e..d4f04c9ce2b9 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java @@ -20,12 +20,12 @@ import static io.confluent.ksql.api.perf.RunnerUtils.DEFAULT_ROW; import io.confluent.ksql.GenericRow; -import io.confluent.ksql.api.impl.VertxCompletableFuture; -import io.confluent.ksql.api.server.BufferedPublisher; import io.confluent.ksql.api.server.InsertResult; import io.confluent.ksql.api.server.InsertsStreamSubscriber; import io.confluent.ksql.api.spi.Endpoints; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.reactive.BufferedPublisher; +import io.confluent.ksql.util.VertxCompletableFuture; import io.vertx.core.Context; import io.vertx.core.WorkerExecutor; import io.vertx.core.buffer.Buffer; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/TestUtils.java b/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/TestUtils.java index 8cf1272b0517..0c4674964489 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/TestUtils.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/TestUtils.java @@ -15,19 +15,10 @@ package io.confluent.ksql.api.utils; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; - import java.net.URL; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; public class TestUtils { - public static void awaitLatch(CountDownLatch latch) throws Exception { - assertThat(latch.await(2000, TimeUnit.MILLISECONDS), is(true)); - } - public static String findFilePath(String fileName) { ClassLoader cl = Thread.currentThread().getContextClassLoader(); URL url = cl.getResource(fileName); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientFunctionalTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientFunctionalTest.java deleted file mode 100644 index e91ba2d67d79..000000000000 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientFunctionalTest.java +++ /dev/null @@ -1,533 +0,0 @@ -/* - * Copyright 2019 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.rest.client; - -import static io.confluent.ksql.GenericRow.genericRow; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.notNullValue; -import static org.junit.Assert.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import io.confluent.ksql.properties.LocalProperties; -import io.confluent.ksql.rest.Errors; -import io.confluent.ksql.rest.entity.CommandId; -import io.confluent.ksql.rest.entity.CommandStatus; -import io.confluent.ksql.rest.entity.CommandStatuses; -import io.confluent.ksql.rest.entity.ExecutionPlan; -import io.confluent.ksql.rest.entity.KsqlEntityList; -import io.confluent.ksql.rest.entity.KsqlErrorMessage; -import io.confluent.ksql.rest.entity.ServerInfo; -import io.confluent.ksql.rest.entity.StreamedRow; -import io.confluent.ksql.rest.server.ExecutableServer; -import io.confluent.ksql.rest.server.KsqlRestConfig; -import io.confluent.ksql.rest.server.mock.MockApplication; -import io.confluent.ksql.rest.server.mock.MockStreamedQueryResource; -import io.confluent.ksql.rest.server.mock.MockStreamedQueryResource.TestStreamWriter; -import io.confluent.rest.ApplicationServer; -import io.confluent.rest.RestConfig; -import java.net.URI; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.Invocation; -import javax.ws.rs.client.WebTarget; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; -import org.apache.http.HttpStatus; -import org.apache.kafka.streams.StreamsConfig; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class KsqlRestClientFunctionalTest { - - private MockApplication mockApplication; - private ExecutableServer mockServer; - private KsqlRestClient ksqlRestClient; - - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - - @Before - public void init() throws Exception { - final Map props = ImmutableMap.builder() - .put(KsqlRestConfig.LISTENERS_CONFIG, "http://localhost:0") - .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") - .put(RestConfig.SHUTDOWN_GRACEFUL_MS_CONFIG, (int) TimeUnit.SECONDS.toMillis(30)) - .build(); - - final KsqlRestConfig ksqlRestConfig = new KsqlRestConfig(props); - mockApplication = new MockApplication(ksqlRestConfig); - mockServer = new ExecutableServer<>( - new ApplicationServer<>(ksqlRestConfig), - ImmutableList.of(mockApplication) - ); - - mockServer.startAsync(); - - ksqlRestClient = buildClient(mockApplication.getServerAddress()); - } - - @After - public void cleanUp() { - try { - mockServer.triggerShutdown(); - } catch (final Exception e) { - System.err.println("Failed to stop app"); - e.printStackTrace(System.err); - } - } - - @Test - public void testKsqlResource() { - final RestResponse results = - ksqlRestClient.makeKsqlRequest("Test request", null); - - assertThat(results, is(notNullValue())); - assertThat(results.isSuccessful(), is(true)); - - final KsqlEntityList ksqlEntityList = results.getResponse(); - assertThat(ksqlEntityList, hasSize(1)); - assertThat(ksqlEntityList.get(0), is(instanceOf(ExecutionPlan.class))); - } - - @Test - public void testStreamRowFromServer() throws InterruptedException { - // Given: - final RestResponse queryResponse = - ksqlRestClient.makeQueryRequest("Select *", null); - - final ReceiverThread receiver = new ReceiverThread(queryResponse); - - final MockStreamedQueryResource.TestStreamWriter writer = getResponseWriter(); - - // When: - writer.enq("hello"); - writer.enq("world"); - writer.enq("{\"row\":null,\"errorMessage\":null,\"finalMessage\":\"Limit Reached\"}"); - writer.finished(); - - // Then: - assertThat(receiver.getRows(), contains( - StreamedRow.row(genericRow("hello")), - StreamedRow.row(genericRow("world")), - StreamedRow.finalMessage("Limit Reached"))); - } - - @Test - public void shouldHandleSlowResponsesFromServer() throws InterruptedException { - // Given: - givenResponsesDelayedBy(Duration.ofSeconds(3)); - - final RestResponse queryResponse = - ksqlRestClient.makeQueryRequest("Select *", null); - - final ReceiverThread receiver = new ReceiverThread(queryResponse); - - final MockStreamedQueryResource.TestStreamWriter writer = getResponseWriter(); - - // When: - writer.enq("hello"); - writer.enq("world"); - writer.enq("{\"row\":null,\"errorMessage\":null,\"finalMessage\":\"Limit Reached\"}"); - writer.finished(); - - // Then: - assertThat(receiver.getRows(), contains( - StreamedRow.row(genericRow("hello")), - StreamedRow.row(genericRow("world")), - StreamedRow.finalMessage("Limit Reached"))); - } - - @Test - public void shouldReturnFalseFromHasNextIfClosedAsynchronously() throws Exception { - // Given: - final RestResponse queryResponse = - ksqlRestClient.makeQueryRequest("Select *", null); - - final QueryStream stream = queryResponse.getResponse(); - - final Thread closeThread = givenStreamWillCloseIn(Duration.ofMillis(500), stream); - - // When: - final boolean result = stream.hasNext(); - - // Then: - assertThat(result, is(false)); - closeThread.join(1_000); - assertThat("invalid test", closeThread.isAlive(), is(false)); - } - - @Test - public void testStatus() { - // When: - final RestResponse response = ksqlRestClient.makeStatusRequest(); - - // Then: - assertThat(response, is(notNullValue())); - assertThat(response.isSuccessful(), is(true)); - assertThat(response.getResponse(), is(new CommandStatuses(ImmutableMap.of( - new CommandId(CommandId.Type.TOPIC, "c1", CommandId.Action.CREATE), - CommandStatus.Status.SUCCESS, - new CommandId(CommandId.Type.TOPIC, "c2", CommandId.Action.CREATE), - CommandStatus.Status.ERROR - )))); - } - - @Test - public void shouldReturnStatusForSpecificCommand() { - // When: - final RestResponse response = ksqlRestClient.makeStatusRequest("TOPIC/c1/CREATE"); - - // Then: - assertThat(response, is(notNullValue())); - assertThat(response.isSuccessful(), is(true)); - assertThat(response.getResponse().getStatus(), is(CommandStatus.Status.SUCCESS)); - } - - @Test(expected = KsqlRestClientException.class) - public void shouldThrowOnInvalidServerAddress() { - buildClient("not-valid-address"); - } - - @Test - public void shouldParseSingleServerAddress() throws Exception { - final String singleServerAddress = "http://singleServer:8088"; - final URI singleServerURI = new URI (singleServerAddress); - try (KsqlRestClient client = buildClient(singleServerAddress)) { - assertThat(client.getServerAddress(), is(singleServerURI)); - } - } - - @Test - public void shouldParseMultipleServerAddresses() throws Exception { - final String firstServerAddress = "http://firstServer:8088"; - final String multipleServerAddresses = firstServerAddress + ",http://secondServer:8088"; - final URI firstServerURI = new URI (firstServerAddress); - try (KsqlRestClient client = buildClient(multipleServerAddresses)) { - assertThat(client.getServerAddress(), is(firstServerURI)); - } - } - - @Test - public void shouldThrowIfAnyServerAddressIsInvalid() { - expectedException.expect(KsqlRestClientException.class); - expectedException.expectMessage("The supplied serverAddress is invalid: secondBuggyServer.8088"); - buildClient("http://firstServer:8088,secondBuggyServer.8088"); - } - - @Test - public void shouldHandleNotFoundOnGetRequests() { - // Given: - givenServerWillReturn(Status.NOT_FOUND); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.getStatusCode().getCode(), is(HttpStatus.SC_NOT_FOUND)); - assertThat(response.getErrorMessage().getErrorCode(), is(40400)); - assertThat(response.getErrorMessage().getMessage(), - containsString("Path not found. Path='/info'. " - + "Check your ksql http url to make sure you are connecting to a ksql server.")); - } - - @Test - public void shouldHandleNotFoundOnPostRequests() { - // Given: - givenServerWillReturn(Status.NOT_FOUND); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("whateva", null); - - // Then: - assertThat(response.getStatusCode().getCode(), is(HttpStatus.SC_NOT_FOUND)); - assertThat(response.getErrorMessage().getErrorCode(), is(40400)); - assertThat(response.getErrorMessage().getMessage(), - containsString("Path not found. Path='/ksql'. " - + "Check your ksql http url to make sure you are connecting to a ksql server.")); - } - - @Test - public void shouldHandleUnauthorizedOnGetRequests() { - // Given: - givenServerWillReturn(Status.UNAUTHORIZED); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), is(Errors.ERROR_CODE_UNAUTHORIZED)); - assertThat(response.getErrorMessage().getMessage(), - is("Could not authenticate successfully with the supplied credentials.")); - } - - @Test - public void shouldHandleUnauthorizedOnPostRequests() { - // Given: - givenServerWillReturn(Status.UNAUTHORIZED); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("whateva", null); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), is(Errors.ERROR_CODE_UNAUTHORIZED)); - assertThat(response.getErrorMessage().getMessage(), - is("Could not authenticate successfully with the supplied credentials.")); - } - - @Test - public void shouldHandleForbiddenOnGetRequests() { - // Given: - givenServerWillReturn(Status.FORBIDDEN); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), is(Errors.ERROR_CODE_FORBIDDEN)); - assertThat(response.getErrorMessage().getMessage(), - is("You are forbidden from using this cluster.")); - } - - @Test - public void shouldHandleForbiddenOnPostRequests() { - // Given: - givenServerWillReturn(Status.FORBIDDEN); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("whateva", null); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), is(Errors.ERROR_CODE_FORBIDDEN)); - assertThat(response.getErrorMessage().getMessage(), - is("You are forbidden from using this cluster.")); - } - - @Test - public void shouldHandleErrorMessageOnGetRequests() { - // Given: - givenServerWillReturn(new KsqlErrorMessage(40000, "ouch", ImmutableList.of("s1", "s2"))); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.getStatusCode().getCode(), is(HttpStatus.SC_BAD_REQUEST)); - assertThat(response.getErrorMessage().getErrorCode(), is(40000)); - assertThat(response.getErrorMessage().getMessage(), is("ouch")); - assertThat(response.getErrorMessage().getStackTrace(), is(ImmutableList.of("s1", "s2"))); - } - - @Test - public void shouldHandleErrorMessageOnPostRequests() { - // Given: - givenServerWillReturn(new KsqlErrorMessage(40100, "ouch", ImmutableList.of("s1", "s2"))); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("whateva", null); - - // Then: - assertThat(response.getStatusCode().getCode(), is(HttpStatus.SC_UNAUTHORIZED)); - assertThat(response.getErrorMessage().getErrorCode(), is(40100)); - assertThat(response.getErrorMessage().getMessage(), is("ouch")); - assertThat(response.getErrorMessage().getStackTrace(), is(ImmutableList.of("s1", "s2"))); - } - - @Test - public void shouldHandleArbitraryErrorsOnGetRequests() { - // Given: - givenServerWillReturn(Status.EXPECTATION_FAILED); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), - is(Errors.toErrorCode(Status.EXPECTATION_FAILED.getStatusCode()))); - assertThat(response.getErrorMessage().getMessage(), - is("The server returned an unexpected error: Expectation Failed")); - } - - @Test - public void shouldHandleArbitraryErrorsOnPostRequests() { - // Given: - givenServerWillReturn(Status.EXPECTATION_FAILED); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("whateva", null); - - // Then: - assertThat(response.getErrorMessage().getErrorCode(), - is(Errors.toErrorCode(Status.EXPECTATION_FAILED.getStatusCode()))); - assertThat(response.getErrorMessage().getMessage(), - is("The server returned an unexpected error: Expectation Failed")); - } - - @Test - public void shouldHandleSuccessOnGetRequests() { - // Given: - final ServerInfo expectedEntity = new ServerInfo("1", "cid", "sid"); - givenServerWillReturn(expectedEntity); - - // When: - final RestResponse response = ksqlRestClient.getServerInfo(); - - // Then: - assertThat(response.get(), is(expectedEntity)); - } - - @Test - public void shouldHandleSuccessOnPostRequests() { - // Given: - final KsqlEntityList expectedEntity = new KsqlEntityList(); - givenServerWillReturn(expectedEntity); - - // When: - final RestResponse response = ksqlRestClient.makeKsqlRequest("foo", null); - - // Then: - assertThat(response.get(), is(expectedEntity)); - } - - private void givenServerWillReturn(final KsqlErrorMessage errorMessage) { - final int statusCode = Errors.toStatusCode(errorMessage.getErrorCode()); - givenServerWillReturn(statusCode, Optional.of(errorMessage)); - } - - private void givenServerWillReturn(final Status statusCode) { - givenServerWillReturn(statusCode.getStatusCode(), Optional.empty()); - } - - private void givenServerWillReturn(final Object entity) { - givenServerWillReturn(Status.OK.getStatusCode(), Optional.of(entity)); - } - - @SuppressWarnings("unchecked") - private void givenServerWillReturn(final int statusCode, final Optional entity) { - final Response response = mock(Response.class); - when(response.getStatus()).thenReturn(statusCode); - when(response.getStatusInfo()).thenReturn(Status.fromStatusCode(statusCode)); - - entity.ifPresent(e -> when(response.readEntity((Class) e.getClass())).thenReturn(e)); - - final Invocation.Builder builder = mock(Invocation.Builder.class); - when(builder.headers(any())).thenReturn(builder); - when(builder.property(any(), any())).thenReturn(builder); - when(builder.get()).thenReturn(response); - when(builder.post(any())).thenReturn(response); - - final WebTarget target = mock(WebTarget.class); - when(target.path(any())).thenReturn(target); - when(target.request(MediaType.APPLICATION_JSON_TYPE)).thenReturn(builder); - - final Client client = mock(Client.class); - when(client.target(any(URI.class))).thenReturn(target); - - final LocalProperties localProps = new LocalProperties(ImmutableMap.of()); - final KsqlClient ksqlClient = new KsqlClient(client, Optional.empty(), localProps); - ksqlRestClient = new KsqlRestClient(ksqlClient, "http://0.0.0.0", localProps); - } - - private void givenResponsesDelayedBy(final Duration delay) { - mockApplication.getStreamedQueryResource().setResponseDelay(delay.toMillis()); - } - - private TestStreamWriter getResponseWriter() { - final MockStreamedQueryResource sqr = mockApplication.getStreamedQueryResource(); - // There can be multiple writers, due to some requests timing out and retrying. - // The last is the one we want: - return Iterables.getLast(sqr.getWriters()); - } - - private static Thread givenStreamWillCloseIn(final Duration duration, final QueryStream stream) { - final Thread thread = new Thread(() -> { - try { - Thread.sleep(duration.toMillis()); - stream.close(); - } catch (final Exception e) { - // Meh - } - }); - thread.setDaemon(true); - thread.start(); - return thread; - } - - private static KsqlRestClient buildClient(final String serverAddress) { - return KsqlRestClient.create( - serverAddress, - ImmutableMap.of(), - ImmutableMap.of(), - Optional.empty() - ); - } - - private static final class ReceiverThread { - - private final QueryStream queryStream; - private final List rows = new CopyOnWriteArrayList<>(); - private final AtomicReference exception = new AtomicReference<>(); - private final Thread thread; - - private ReceiverThread(final RestResponse queryResponse) { - assertThat("not successful", queryResponse.isSuccessful(), is(true)); - this.queryStream = queryResponse.getResponse(); - this.thread = new Thread(() -> { - try { - while (queryStream.hasNext()) { - final StreamedRow row = queryStream.next(); - rows.add(row); - } - - } catch (final Exception e) { - exception.set(e); - } - }, "receiver-thread"); - thread.setDaemon(true); - thread.start(); - } - - private List getRows() throws InterruptedException { - thread.join(20_000); - assertThat("Receive thread still running", thread.isAlive(), is(false)); - if (exception.get() != null) { - throw new RuntimeException(exception.get()); - } - return rows; - } - } -} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/MockKsqlSecurityExtension.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/MockKsqlSecurityExtension.java index 7c2c1a0d3e16..b14ffcdfb7cd 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/MockKsqlSecurityExtension.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/MockKsqlSecurityExtension.java @@ -22,8 +22,9 @@ public void initialize(final KsqlConfig ksqlConfig) { @Override public Optional getAuthorizationProvider() { - return Optional.of((user, method, path) -> - MockKsqlSecurityExtension.provider.checkEndpointAccess(user, method, path)); + return Optional.of( + (user, method, path) -> MockKsqlSecurityExtension.provider + .checkEndpointAccess(user, method, path)); } @Override diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java index 1dbf0c1a2f45..ddf8771cfaa8 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java @@ -15,14 +15,11 @@ package io.confluent.ksql.rest.integration; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableMap; import com.google.common.net.UrlEscapers; import io.confluent.ksql.json.JsonMapper; import io.confluent.ksql.rest.client.BasicCredentials; import io.confluent.ksql.rest.client.KsqlRestClient; -import io.confluent.ksql.rest.client.QueryStream; import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatus.Status; @@ -92,17 +89,12 @@ static List makeQueryRequest( ) { try (final KsqlRestClient restClient = restApp.buildKsqlClient(userCreds)) { - final RestResponse res = restClient.makeQueryRequest(sql, null, properties); + final RestResponse> res = + restClient.makeQueryRequest(sql, null, properties); throwOnError(res); - final QueryStream s = res.getResponse(); - - final Builder builder = ImmutableList.builder(); - while (s.hasNext()) { - builder.add(s.next()); - } - return builder.build(); + return res.getResponse(); } } @@ -114,7 +106,8 @@ static KsqlErrorMessage makeQueryRequestWithError( ) { try (final KsqlRestClient restClient = restApp.buildKsqlClient(userCreds)) { - final RestResponse res = restClient.makeQueryRequest(sql, null, properties); + final RestResponse> res = + restClient.makeQueryRequest(sql, null, properties); throwOnNoError(res); diff --git a/ksql-rest-client/pom.xml b/ksql-rest-client/pom.xml index 34b7c83030ae..68f221874529 100644 --- a/ksql-rest-client/pom.xml +++ b/ksql-rest-client/pom.xml @@ -44,22 +44,18 @@ ksql-parser - - io.confluent - rest-utils - - - - org.apache.httpcomponents - httpclient - - com.fasterxml.jackson.datatype jackson-datatype-jdk8 ${jackson.version} + + org.reactivestreams + reactive-streams + ${reactive-streams.version} + + @@ -69,6 +65,14 @@ test + + io.vertx + vertx-web + ${vertx.version} + test + + + diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/HttpClientBuilder.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/HttpClientBuilder.java deleted file mode 100644 index 959008c4a9f1..000000000000 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/HttpClientBuilder.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright 2019 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.rest.client; - -import com.google.common.annotations.VisibleForTesting; -import io.confluent.ksql.json.JsonMapper; -import io.confluent.ksql.rest.client.ssl.DefaultSslClientConfigurer; -import io.confluent.ksql.rest.client.ssl.SslClientConfigurer; -import io.confluent.rest.validation.JacksonMessageBodyProvider; -import java.util.Map; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; - -final class HttpClientBuilder { - - private HttpClientBuilder() { - } - - static Client buildClient(final Map clientProps) { - return buildClient(ClientBuilder.newBuilder(), new DefaultSslClientConfigurer(), clientProps); - } - - @VisibleForTesting - static Client buildClient( - final javax.ws.rs.client.ClientBuilder clientBuilder, - final SslClientConfigurer sslClientConfigurer, - final Map clientProps - ) { - try { - clientBuilder.register(new JacksonMessageBodyProvider(JsonMapper.INSTANCE.mapper)); - - sslClientConfigurer.configureSsl(clientBuilder, clientProps); - - return clientBuilder.build(); - } catch (final Exception e) { - throw new KsqlRestClientException("Failed to configure rest client", e); - } - } -} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java index 2f26d8542d38..7e131f440c87 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClient.java @@ -15,19 +15,24 @@ package io.confluent.ksql.rest.client; -import static java.util.Objects.requireNonNull; - import com.fasterxml.jackson.databind.DeserializationFeature; import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.json.JsonMapper; import io.confluent.ksql.parser.json.KsqlTypesDeserializationModule; import io.confluent.ksql.properties.LocalProperties; +import io.vertx.core.Vertx; +import io.vertx.core.VertxException; +import io.vertx.core.http.HttpClient; +import io.vertx.core.http.HttpClientOptions; +import io.vertx.core.net.JksOptions; +import io.vertx.core.net.SocketAddress; import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.Base64; import java.util.Map; +import java.util.Objects; import java.util.Optional; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.WebTarget; -import org.glassfish.jersey.client.authentication.HttpAuthenticationFeature; +import org.apache.kafka.common.config.SslConfigs; @SuppressWarnings("WeakerAccess") // Public API public final class KsqlClient implements AutoCloseable { @@ -38,37 +43,119 @@ public final class KsqlClient implements AutoCloseable { JsonMapper.INSTANCE.mapper.registerModule(new KsqlTypesDeserializationModule(false)); } - private final Client httpClient; + public static final String DISABLE_HOSTNAME_VERIFICATION_PROP_NAME + = "ksql.client.disable.hostname.verification"; + public static final String TLS_ENABLED_PROP_NAME = "ksql.client.enable.tls"; + + private final Vertx vertx; + private final HttpClient httpClient; private final LocalProperties localProperties; + private final Optional basicAuthHeader; + private final boolean isTls; public KsqlClient( final Map clientProps, final Optional credentials, - final LocalProperties localProperties + final LocalProperties localProperties, + final HttpClientOptions httpClientOptions ) { - this(HttpClientBuilder.buildClient(clientProps), credentials, localProperties); + this(Vertx.vertx(), clientProps, credentials, localProperties, httpClientOptions); } @VisibleForTesting KsqlClient( - final Client httpClient, + final HttpClient httpClient, + final boolean isTls, final Optional credentials, final LocalProperties localProperties ) { - this.httpClient = requireNonNull(httpClient, "httpClient"); - this.localProperties = requireNonNull(localProperties, "localProperties"); + this(null, httpClient, isTls, credentials, localProperties); + } + + private KsqlClient( + final Vertx vertx, + final HttpClient httpClient, + final boolean isTls, + final Optional credentials, + final LocalProperties localProperties + ) { + this.vertx = vertx; + this.httpClient = Objects.requireNonNull(httpClient, "httpClient"); + this.isTls = isTls; + this.basicAuthHeader = createBasicAuthHeader( + Objects.requireNonNull(credentials, "credentials")); + this.localProperties = Objects.requireNonNull(localProperties, "localProperties"); + } - credentials.ifPresent(creds -> { - httpClient.register(HttpAuthenticationFeature.basic(creds.username(), creds.password())); - }); + private KsqlClient( + final Vertx vertx, + final Map clientProps, + final Optional credentials, + final LocalProperties localProperties, + final HttpClientOptions httpClientOptions + ) { + this(vertx, createHttpClient(vertx, clientProps, httpClientOptions), httpClientOptions.isSsl(), + credentials, localProperties); } public KsqlTarget target(final URI server) { - final WebTarget target = httpClient.target(server); - return new KsqlTarget(target, localProperties, Optional.empty()); + final boolean isUriTls = server.getScheme().equalsIgnoreCase("https"); + if (isTls != isUriTls) { + throw new KsqlRestClientException("Cannot make request with scheme " + server.getScheme() + + " as client is configured " + (isTls ? "with" : "without") + " tls"); + } + return new KsqlTarget(httpClient, + SocketAddress.inetSocketAddress(server.getPort(), server.getHost()), localProperties, + basicAuthHeader); } public void close() { - httpClient.close(); + try { + httpClient.close(); + } catch (Exception ignore) { + // Ignore + } + if (vertx != null) { + vertx.close(); + } + } + + private static Optional createBasicAuthHeader( + final Optional credentials) { + return credentials.map(basicCredentials -> "Basic " + Base64.getEncoder() + .encodeToString((basicCredentials.username() + + ":" + basicCredentials.password()).getBytes(StandardCharsets.UTF_8)) + ); } + + private static HttpClient createHttpClient(final Vertx vertx, + final Map clientProps, + final HttpClientOptions httpClientOptions) { + if ("true".equals(clientProps.get(DISABLE_HOSTNAME_VERIFICATION_PROP_NAME))) { + httpClientOptions.setVerifyHost(false); + } + if ("true".equals(clientProps.get(TLS_ENABLED_PROP_NAME))) { + httpClientOptions.setSsl(true); + } + final String trustStoreLocation = clientProps.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); + if (trustStoreLocation != null) { + final String suppliedTruststorePassword = clientProps + .get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG); + httpClientOptions.setTrustStoreOptions(new JksOptions().setPath(trustStoreLocation) + .setPassword(suppliedTruststorePassword == null ? "" : suppliedTruststorePassword)); + final String keyStoreLocation = clientProps.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + if (keyStoreLocation != null) { + final String suppliedKeyStorePassord = clientProps + .get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + httpClientOptions.setKeyStoreOptions(new JksOptions().setPath(keyStoreLocation) + .setPassword(suppliedTruststorePassword == null ? "" : suppliedKeyStorePassord)); + } + } + try { + return vertx.createHttpClient(httpClientOptions); + } catch (VertxException e) { + throw new KsqlRestClientException(e.getMessage(), e); + } + } + } diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClientUtil.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClientUtil.java index ab574a957413..8224acb2958f 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClientUtil.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlClientUtil.java @@ -15,12 +15,14 @@ package io.confluent.ksql.rest.client; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.confluent.ksql.json.JsonMapper; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.vertx.core.buffer.Buffer; import java.util.Optional; import java.util.function.Function; import javax.naming.AuthenticationException; -import javax.ws.rs.core.Response; import org.eclipse.jetty.http.HttpStatus; import org.eclipse.jetty.http.HttpStatus.Code; @@ -30,22 +32,41 @@ private KsqlClientUtil() { } public static RestResponse toRestResponse( - final Response response, + final ResponseWithBody resp, final String path, - final Function mapper + final Function mapper ) { - final Code statusCode = HttpStatus.getCode(response.getStatus()); + final Code statusCode = HttpStatus.getCode(resp.getResponse().statusCode()); return statusCode == Code.OK - ? RestResponse.successful(statusCode, mapper.apply(response)) - : createErrorResponse(path, response); + ? RestResponse.successful(statusCode, mapper.apply(resp)) + : createErrorResponse(path, resp); + } + + static T deserialize(final Buffer buffer, final Class clazz) { + final ObjectMapper objectMapper = JsonMapper.INSTANCE.mapper; + try { + return objectMapper.readValue(buffer.getBytes(), clazz); + } catch (Exception e) { + throw new KsqlRestClientException("Failed to deserialise object", e); + } + } + + static Buffer serialize(final Object object) { + final ObjectMapper objectMapper = JsonMapper.INSTANCE.mapper; + try { + final byte[] bytes = objectMapper.writeValueAsBytes(object); + return Buffer.buffer(bytes); + } catch (Exception e) { + throw new KsqlRestClientException("Failed to serialise object", e); + } } private static RestResponse createErrorResponse( final String path, - final Response response + final ResponseWithBody resp ) { - final Code statusCode = HttpStatus.getCode(response.getStatus()); - final Optional errorMessage = tryReadErrorMessage(response); + final Code statusCode = HttpStatus.getCode(resp.getResponse().statusCode()); + final Optional errorMessage = tryReadErrorMessage(resp); if (errorMessage.isPresent()) { return RestResponse.erroneous(statusCode, errorMessage.get()); } @@ -68,12 +89,13 @@ private static RestResponse createErrorResponse( return RestResponse.erroneous( statusCode, "The server returned an unexpected error: " - + response.getStatusInfo().getReasonPhrase()); + + resp.getResponse().statusMessage()); } - private static Optional tryReadErrorMessage(final Response response) { + private static Optional tryReadErrorMessage( + final ResponseWithBody resp) { try { - return Optional.ofNullable(response.readEntity(KsqlErrorMessage.class)); + return Optional.ofNullable(deserialize(resp.getBody(), KsqlErrorMessage.class)); } catch (final Exception e) { return Optional.empty(); } @@ -93,4 +115,5 @@ private static KsqlErrorMessage forbiddenErrorMsg() { new AuthenticationException("You are forbidden from using this cluster.") ); } + } diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java index b46e84d508fe..52d2f55288a9 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java @@ -28,17 +28,16 @@ import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.ServerInfo; +import io.confluent.ksql.rest.entity.StreamedRow; +import io.vertx.core.http.HttpClientOptions; import java.io.Closeable; -import java.io.InputStream; import java.net.URI; import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.Future; import java.util.stream.Collectors; -import javax.ws.rs.core.Response; public class KsqlRestClient implements Closeable { @@ -60,7 +59,8 @@ public static KsqlRestClient create( final Optional creds ) { final LocalProperties localProperties = new LocalProperties(localProps); - final KsqlClient client = new KsqlClient(clientProps, creds, localProperties); + final KsqlClient client = new KsqlClient(clientProps, creds, localProperties, + new HttpClientOptions()); return new KsqlRestClient(client, serverAddress, localProperties); } @@ -91,21 +91,21 @@ public RestResponse getServerHealth() { return target().getServerHealth(); } - public Future makeAsyncHeartbeatRequest( + public void makeAsyncHeartbeatRequest( final KsqlHostInfoEntity host, final long timestamp ) { - return target().postAsyncHeartbeatRequest(host, timestamp); + target().postAsyncHeartbeatRequest(host, timestamp); } public RestResponse makeClusterStatusRequest() { return target().getClusterStatus(); } - public Future makeAsyncLagReportingRequest( + public void makeAsyncLagReportingRequest( final LagReportingMessage lagReportingMessage ) { - return target().postAsyncLagReportingRequest(lagReportingMessage); + target().postAsyncLagReportingRequest(lagReportingMessage); } public RestResponse makeKsqlRequest(final String ksql) { @@ -124,11 +124,17 @@ public RestResponse makeStatusRequest(final String commandId) { return target().getStatus(commandId); } - public RestResponse makeQueryRequest(final String ksql, final Long commandSeqNum) { - return target().postQueryRequest(ksql, Optional.ofNullable(commandSeqNum)); + public RestResponse> makeQueryRequestStreamed(final String ksql, + final Long commandSeqNum) { + return target().postQueryRequestStreamed(ksql, Optional.ofNullable(commandSeqNum)); } - public RestResponse makeQueryRequest(final String ksql, + public RestResponse> makeQueryRequest(final String ksql, + final Long commandSeqNum) { + return makeQueryRequest(ksql, commandSeqNum, null); + } + + public RestResponse> makeQueryRequest(final String ksql, final Long commandSeqNum, final Map properties) { KsqlTarget target = target(); if (properties != null) { @@ -137,7 +143,7 @@ public RestResponse makeQueryRequest(final String ksql, return target.postQueryRequest(ksql, Optional.ofNullable(commandSeqNum)); } - public RestResponse makePrintTopicRequest( + public RestResponse> makePrintTopicRequest( final String ksql, final Long commandSeqNum ) { diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java index 41f5f5a487bf..6214c2707a1a 100644 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java @@ -15,6 +15,8 @@ package io.confluent.ksql.rest.client; +import static io.confluent.ksql.rest.client.KsqlClientUtil.deserialize; +import static io.confluent.ksql.rest.client.KsqlClientUtil.serialize; import static java.util.Objects.requireNonNull; import io.confluent.ksql.properties.LocalProperties; @@ -28,27 +30,30 @@ import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.LagReportingMessage; import io.confluent.ksql.rest.entity.ServerInfo; -import java.io.InputStream; -import java.net.SocketTimeoutException; +import io.confluent.ksql.rest.entity.StreamedRow; +import io.confluent.ksql.util.VertxCompletableFuture; +import io.vertx.core.Vertx; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpClient; +import io.vertx.core.http.HttpClientRequest; +import io.vertx.core.http.HttpClientResponse; +import io.vertx.core.http.HttpMethod; +import io.vertx.core.net.SocketAddress; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; import java.util.function.Function; -import javax.ws.rs.ProcessingException; -import javax.ws.rs.client.Entity; -import javax.ws.rs.client.WebTarget; -import javax.ws.rs.core.HttpHeaders; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.MultivaluedHashMap; -import javax.ws.rs.core.MultivaluedMap; -import javax.ws.rs.core.Response; -import org.glassfish.jersey.client.ClientProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @SuppressWarnings("WeakerAccess") // Public API public final class KsqlTarget { - private static final int MAX_TIMEOUT = (int) TimeUnit.SECONDS.toMillis(32); + private static final Logger log = LoggerFactory.getLogger(KsqlTarget.class); private static final String STATUS_PATH = "/status"; private static final String KSQL_PATH = "/ksql"; @@ -57,26 +62,32 @@ public final class KsqlTarget { private static final String CLUSTERSTATUS_PATH = "/clusterStatus"; private static final String LAG_REPORT_PATH = "/lag"; - private final WebTarget target; + private final HttpClient httpClient; + private final SocketAddress socketAddress; private final LocalProperties localProperties; private final Optional authHeader; KsqlTarget( - final WebTarget target, + final HttpClient httpClient, + final SocketAddress socketAddress, final LocalProperties localProperties, final Optional authHeader ) { - this.target = requireNonNull(target, "target"); + this.httpClient = requireNonNull(httpClient, "httpClient"); + this.socketAddress = requireNonNull(socketAddress, "socketAddress"); this.localProperties = requireNonNull(localProperties, "localProperties"); this.authHeader = requireNonNull(authHeader, "authHeader"); } public KsqlTarget authorizationHeader(final String authHeader) { - return new KsqlTarget(target, localProperties, Optional.of(authHeader)); + return new KsqlTarget(httpClient, socketAddress, localProperties, + Optional.of(authHeader)); } public KsqlTarget properties(final Map properties) { - return new KsqlTarget(target, new LocalProperties(properties), authHeader); + return new KsqlTarget(httpClient, socketAddress, + new LocalProperties(properties), + authHeader); } public RestResponse getServerInfo() { @@ -87,14 +98,13 @@ public RestResponse getServerHealth() { return get("/healthcheck", HealthCheckResponse.class); } - public Future postAsyncHeartbeatRequest( + public void postAsyncHeartbeatRequest( final KsqlHostInfoEntity host, final long timestamp ) { - return postAsync( + executeRequestAsync( HEARTBEAT_PATH, - new HeartbeatMessage(host, timestamp), - Optional.empty() + new HeartbeatMessage(host, timestamp) ); } @@ -102,13 +112,12 @@ public RestResponse getClusterStatus() { return get(CLUSTERSTATUS_PATH, ClusterStatusResponse.class); } - public Future postAsyncLagReportingRequest( + public void postAsyncLagReportingRequest( final LagReportingMessage lagReportingMessage ) { - return postAsync( + executeRequestAsync( LAG_REPORT_PATH, - lagReportingMessage, - Optional.empty() + lagReportingMessage ); } @@ -126,40 +135,38 @@ public RestResponse postKsqlRequest( ) { return post( KSQL_PATH, - ksqlRequest(ksql, previousCommandSeqNum), - Optional.empty(), - true, - r -> r.readEntity(KsqlEntityList.class) + createKsqlRequest(ksql, previousCommandSeqNum), + r -> deserialize(r.getBody(), KsqlEntityList.class) ); } - public RestResponse postQueryRequest( + public RestResponse> postQueryRequest( final String ksql, final Optional previousCommandSeqNum ) { return post( QUERY_PATH, - ksqlRequest(ksql, previousCommandSeqNum), - Optional.of(QueryStream.READ_TIMEOUT_MS), - false, - QueryStream::new + createKsqlRequest(ksql, previousCommandSeqNum), + KsqlTarget::toRows ); } - public RestResponse postPrintTopicRequest( + public RestResponse> postQueryRequestStreamed( + final String sql, + final Optional previousCommandSeqNum + ) { + return executeQueryRequestWithStreamResponse(sql, previousCommandSeqNum, + buff -> deserialize(buff, StreamedRow.class)); + } + + public RestResponse> postPrintTopicRequest( final String ksql, final Optional previousCommandSeqNum ) { - return post( - QUERY_PATH, - ksqlRequest(ksql, previousCommandSeqNum), - Optional.empty(), - false, - r -> (InputStream) r.getEntity() - ); + return executeQueryRequestWithStreamResponse(ksql, previousCommandSeqNum, Object::toString); } - private KsqlRequest ksqlRequest( + private KsqlRequest createKsqlRequest( final String ksql, final Optional previousCommandSeqNum ) { @@ -171,89 +178,124 @@ private KsqlRequest ksqlRequest( } private RestResponse get(final String path, final Class type) { - try (Response response = target - .path(path) - .request(MediaType.APPLICATION_JSON_TYPE) - .headers(headers()) - .get() - ) { - return KsqlClientUtil.toRestResponse(response, path, r -> r.readEntity(type)); - } catch (final Exception e) { - throw new KsqlRestClientException("Error issuing GET to KSQL server. path:" + path, e); - } + return executeRequestSync(HttpMethod.GET, path, null, r -> deserialize(r.getBody(), type)); } private RestResponse post( final String path, final Object jsonEntity, - final Optional readTimeoutMs, - final boolean closeResponse, - final Function mapper + final Function mapper ) { - Response response = null; + return executeRequestSync(HttpMethod.POST, path, jsonEntity, mapper); + } - try { - response = target - .path(path) - .request(MediaType.APPLICATION_JSON_TYPE) - .property(ClientProperties.READ_TIMEOUT, readTimeoutMs.orElse(0)) - .headers(headers()) - .post(Entity.json(jsonEntity)); - - return KsqlClientUtil.toRestResponse(response, path, mapper); - } catch (final ProcessingException e) { - if (shouldRetry(readTimeoutMs, e)) { - return post(path, jsonEntity, calcReadTimeout(readTimeoutMs), closeResponse, mapper); - } - throw new KsqlRestClientException("Error issuing POST to KSQL server. path:" + path, e); - } catch (final Exception e) { - throw new KsqlRestClientException("Error issuing POST to KSQL server. path:" + path, e); - } finally { - if (response != null && closeResponse) { - response.close(); - } - } + private void executeRequestAsync( + final String path, + final Object jsonEntity + ) { + execute(HttpMethod.POST, path, jsonEntity, (resp, vcf) -> { + }).exceptionally(t -> { + log.error("Unexpected exception in async request", t); + return null; + }); } - private Future postAsync( + private RestResponse executeRequestSync( + final HttpMethod httpMethod, final String path, - final Object jsonEntity, - final Optional readTimeoutMs + final Object requestBody, + final Function mapper ) { - try { - // Performs an asynchronous request - return target - .path(path) - .request(MediaType.APPLICATION_JSON_TYPE) - .property(ClientProperties.READ_TIMEOUT, readTimeoutMs.orElse(0)) - .headers(headers()) - .async() - .post(Entity.json(jsonEntity)); - } catch (final ProcessingException e) { - if (shouldRetry(readTimeoutMs, e)) { - return postAsync(path, jsonEntity, calcReadTimeout(readTimeoutMs)); - } - throw new KsqlRestClientException("Error issuing POST to KSQL server. path:" + path, e); - } catch (final Exception e) { - throw new KsqlRestClientException("Error issuing POST to KSQL server. path:" + path, e); - } + return executeSync(httpMethod, path, requestBody, mapper, (resp, vcf) -> { + resp.bodyHandler(buff -> vcf.complete(new ResponseWithBody(resp, buff))); + }); } - private MultivaluedMap headers() { - final MultivaluedMap headers = new MultivaluedHashMap<>(); - authHeader.ifPresent(v -> headers.add(HttpHeaders.AUTHORIZATION, v)); - return headers; + private RestResponse> executeQueryRequestWithStreamResponse( + final String ksql, + final Optional previousCommandSeqNum, + final Function mapper + ) { + final KsqlRequest ksqlRequest = createKsqlRequest(ksql, previousCommandSeqNum); + final AtomicReference> pubRef = new AtomicReference<>(); + return executeSync(HttpMethod.POST, QUERY_PATH, ksqlRequest, resp -> pubRef.get(), + (resp, vcf) -> { + if (resp.statusCode() == 200) { + pubRef.set(new StreamPublisher<>(Vertx.currentContext(), + resp, mapper, vcf)); + vcf.complete(new ResponseWithBody(resp)); + } else { + resp.bodyHandler(body -> vcf.complete(new ResponseWithBody(resp, body))); + } + }); } - private static boolean shouldRetry( - final Optional readTimeoutMs, - final ProcessingException e + private RestResponse executeSync( + final HttpMethod httpMethod, + final String path, + final Object requestBody, + final Function mapper, + final BiConsumer> responseHandler + ) { + final CompletableFuture vcf = + execute(httpMethod, path, requestBody, responseHandler); + + final ResponseWithBody response; + try { + response = vcf.get(); + } catch (Exception e) { + throw new KsqlRestClientException( + "Error issuing " + httpMethod + " to KSQL server. path:" + path, e); + } + return KsqlClientUtil.toRestResponse(response, path, mapper); + } + + private CompletableFuture execute( + final HttpMethod httpMethod, + final String path, + final Object requestBody, + final BiConsumer> responseHandler ) { - return readTimeoutMs.map(timeout -> timeout < MAX_TIMEOUT).orElse(false) - && e.getCause() instanceof SocketTimeoutException; + final VertxCompletableFuture vcf = new VertxCompletableFuture<>(); + + final HttpClientRequest httpClientRequest = httpClient.request(httpMethod, + socketAddress, socketAddress.port(), socketAddress.host(), + path, + resp -> responseHandler.accept(resp, vcf)) + .exceptionHandler(vcf::completeExceptionally); + + httpClientRequest.putHeader("Accept", "application/json"); + authHeader.ifPresent(v -> httpClientRequest.putHeader("Authorization", v)); + + if (requestBody != null) { + httpClientRequest.end(serialize(requestBody)); + } else { + httpClientRequest.end(); + } + + return vcf; } - private static Optional calcReadTimeout(final Optional previousTimeoutMs) { - return previousTimeoutMs.map(timeout -> Math.min(timeout * 2, MAX_TIMEOUT)); + private static List toRows(final ResponseWithBody resp) { + + final List rows = new ArrayList<>(); + final Buffer buff = resp.getBody(); + int begin = 0; + + for (int i = 0; i <= buff.length(); i++) { + if ((i == buff.length() && (i - begin > 1)) || buff.getByte(i) == (byte) '\n') { + if (begin != i) { // Ignore random newlines - the server can send these + final Buffer sliced = buff.slice(begin, i); + final Buffer tidied = StreamPublisher.toJsonMsg(sliced); + final StreamedRow row = deserialize(tidied, StreamedRow.class); + rows.add(row); + } + + begin = i + 1; + } + } + + return rows; } + } diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/QueryStream.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/QueryStream.java deleted file mode 100644 index 9649fd5f755e..000000000000 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/QueryStream.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Copyright 2019 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.rest.client; - -import com.fasterxml.jackson.databind.ObjectMapper; -import io.confluent.ksql.json.JsonMapper; -import io.confluent.ksql.rest.entity.StreamedRow; -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.net.SocketTimeoutException; -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.Scanner; -import java.util.concurrent.TimeUnit; -import javax.ws.rs.core.Response; -import org.apache.commons.compress.utils.IOUtils; -import org.apache.commons.lang3.StringUtils; - -public final class QueryStream implements Closeable, Iterator { - - static final int READ_TIMEOUT_MS = (int) TimeUnit.SECONDS.toMillis(2); - - private final Response response; - private final ObjectMapper objectMapper; - private final Scanner responseScanner; - private final InputStreamReader isr; - - private StreamedRow bufferedRow; - private volatile boolean closed = false; - - QueryStream(final Response response) { - this.response = response; - - this.objectMapper = JsonMapper.INSTANCE.mapper; - this.isr = new InputStreamReader( - (InputStream) response.getEntity(), - StandardCharsets.UTF_8 - ); - this.responseScanner = new Scanner((buf) -> { - while (true) { - try { - return isr.read(buf); - } catch (final SocketTimeoutException e) { - // Read timeout: - if (closed) { - return -1; - } - } catch (final IOException e) { - // Can occur if isr closed: - if (closed) { - return -1; - } - - throw e; - } - } - }); - - this.bufferedRow = null; - } - - @Override - public boolean hasNext() { - if (bufferedRow != null) { - return true; - } - - return bufferNextRow(); - } - - @Override - public StreamedRow next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - final StreamedRow result = bufferedRow; - bufferedRow = null; - return result; - } - - @Override - public void close() { - if (closed) { - return; - } - - synchronized (this) { - closed = true; - } - responseScanner.close(); - response.close(); - IOUtils.closeQuietly(isr); - } - - private boolean bufferNextRow() { - try { - while (responseScanner.hasNextLine()) { - final String responseLine = responseScanner.nextLine().trim(); - - final String jsonMsg = toJsonMsg(responseLine); - - if (jsonMsg.isEmpty()) { - continue; - } - - try { - bufferedRow = objectMapper.readValue(jsonMsg, StreamedRow.class); - } catch (final IOException exception) { - if (closed) { - return false; - } - throw new RuntimeException(exception); - } - return true; - } - - return false; - } catch (final IllegalStateException e) { - // Can happen is scanner is closed: - if (closed) { - return false; - } - - throw e; - } - } - - /** - * Convert the single line within the full response into a valid JSON object. - * - *

The entire response is an array of JSON objects, e.g. in the form: - * - *

-   *   {@code
-   *   [{...stuff...},
-   *    {...stuff...},
-   *    ...more rows....
-   *    {...stuff...}],
-   *   }
-   * 
- * - *

This method trims any leading {@code [} or trailing {@code ,} or {@code ]} - */ - private static String toJsonMsg(final String responseLine) { - String result = StringUtils.removeStart(responseLine, "["); - result = StringUtils.removeEnd(result, "]"); - result = StringUtils.removeEnd(result, ","); - return result.trim(); - } -} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ResponseWithBody.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ResponseWithBody.java new file mode 100644 index 000000000000..f741d079bc71 --- /dev/null +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ResponseWithBody.java @@ -0,0 +1,44 @@ +/* + * Copyright 2020 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.rest.client; + +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpClientResponse; +import java.util.Objects; + +class ResponseWithBody { + + private final HttpClientResponse response; + private final Buffer body; + + ResponseWithBody(final HttpClientResponse response, final Buffer body) { + this.response = Objects.requireNonNull(response); + this.body = Objects.requireNonNull(body); + } + + ResponseWithBody(final HttpClientResponse response) { + this.response = Objects.requireNonNull(response); + this.body = null; + } + + public HttpClientResponse getResponse() { + return response; + } + + public Buffer getBody() { + return body; + } +} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java new file mode 100644 index 000000000000..0f280e828d9a --- /dev/null +++ b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java @@ -0,0 +1,77 @@ +/* + * Copyright 2020 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.rest.client; + +import io.confluent.ksql.reactive.BufferedPublisher; +import io.vertx.core.Context; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpClientResponse; +import io.vertx.core.parsetools.RecordParser; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; + +public class StreamPublisher extends BufferedPublisher { + + private final HttpClientResponse response; + private boolean drainHandlerSet; + + public static Buffer toJsonMsg(final Buffer responseLine) { + + int start = 0; + int end = responseLine.length() - 1; + if (responseLine.getByte(0) == (byte) '[') { + start = 1; + } + if (responseLine.getByte(end) == (byte) ']') { + end -= 1; + } + if (responseLine.getByte(end) == (byte) ',') { + end -= 1; + } + return responseLine.slice(start, end + 1); + } + + StreamPublisher(final Context context, final HttpClientResponse response, + final Function mapper, + final CompletableFuture bodyFuture) { + super(context); + this.response = response; + final RecordParser recordParser = RecordParser.newDelimited("\n", response); + recordParser.exceptionHandler(bodyFuture::completeExceptionally) + .handler(buff -> { + if (buff.length() == 0) { + // Ignore empty buffer - the server can insert random newlines! + return; + } + final Buffer jsonMsg = toJsonMsg(buff); + if (!accept(mapper.apply(jsonMsg))) { + if (!drainHandlerSet) { + recordParser.pause(); + drainHandlerSet = true; + drainHandler(() -> { + drainHandlerSet = false; + recordParser.resume(); + }); + } + } + }) + .endHandler(v -> complete()); + } + + public void close() { + response.request().connection().close(); + } +} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurer.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurer.java deleted file mode 100644 index 1a421e2e7db8..000000000000 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurer.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Copyright 2019 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.rest.client.ssl; - -import java.util.Map; -import javax.ws.rs.client.ClientBuilder; - -public class DefaultSslClientConfigurer implements SslClientConfigurer { - - @Override - public ClientBuilder configureSsl(final ClientBuilder builder, final Map props) { - final String password = SslUtil.getKeyPassword(props); - - SslUtil.loadKeyStore(props) - .ifPresent(ks -> builder.keyStore(ks, password)); - - SslUtil.loadTrustStore(props) - .ifPresent(builder::trustStore); - - SslUtil.getHostNameVerifier(props) - .ifPresent(builder::hostnameVerifier); - - return builder; - } -} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslClientConfigurer.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslClientConfigurer.java deleted file mode 100644 index fc2f4ee3a9db..000000000000 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslClientConfigurer.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright 2019 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.rest.client.ssl; - -import java.util.Map; -import javax.ws.rs.client.ClientBuilder; - -public interface SslClientConfigurer { - - /** - * Configure the client's SSL settings based on the supplied {@code props}. - * - * @param builder the client builder to configure - * @param props the props that drive what to configure. - * @return the builder. - */ - ClientBuilder configureSsl(ClientBuilder builder, Map props); -} diff --git a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslUtil.java b/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslUtil.java deleted file mode 100644 index c09bfec2504e..000000000000 --- a/ksql-rest-client/src/main/java/io/confluent/ksql/rest/client/ssl/SslUtil.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Copyright 2019 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.rest.client.ssl; - -import io.confluent.ksql.util.KsqlException; -import io.confluent.rest.RestConfig; -import java.io.FileInputStream; -import java.security.KeyStore; -import java.util.Map; -import java.util.Optional; -import javax.net.ssl.HostnameVerifier; -import org.apache.http.conn.ssl.NoopHostnameVerifier; -import org.apache.kafka.common.config.ConfigException; - -public final class SslUtil { - - private SslUtil() { - } - - static Optional loadKeyStore(final Map props) { - return load(props, - RestConfig.SSL_KEYSTORE_LOCATION_CONFIG, - RestConfig.SSL_KEYSTORE_PASSWORD_CONFIG, - RestConfig.SSL_KEYSTORE_TYPE_CONFIG); - } - - static Optional loadTrustStore(final Map props) { - return load(props, - RestConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, - RestConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, - RestConfig.SSL_TRUSTSTORE_TYPE_CONFIG); - } - - static String getKeyPassword(final Map props) { - return props.getOrDefault(RestConfig.SSL_KEY_PASSWORD_CONFIG, ""); - } - - static Optional getHostNameVerifier(final Map props) { - final String algo = props.getOrDefault( - RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ""); - - if (algo.isEmpty()) { - return Optional.of(NoopHostnameVerifier.INSTANCE); - } - - if (algo.equalsIgnoreCase("https")) { - return Optional.empty(); - } - - throw new ConfigException( - RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, algo, "Not supported"); - } - - private static Optional load( - final Map props, - final String locationConfig, - final String passwordConfig, - final String typeConfig - ) { - final String location = props.getOrDefault(locationConfig, ""); - if (location.isEmpty()) { - return Optional.empty(); - } - - try (FileInputStream stream = new FileInputStream(location)) { - - final String password = props.getOrDefault(passwordConfig, ""); - final String type = props.getOrDefault(typeConfig, "JKS"); - - final KeyStore keyStore = KeyStore.getInstance(type); - keyStore.load(stream, password.toCharArray()); - return Optional.of(keyStore); - } catch (final Exception e) { - throw new KsqlException("Failed to load keyStore: " + location, e); - } - } -} diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/FakeApiServer.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/FakeApiServer.java new file mode 100644 index 000000000000..164eb244c7b4 --- /dev/null +++ b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/FakeApiServer.java @@ -0,0 +1,165 @@ +/* + * Copyright 2020 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.rest.client; + +import io.vertx.core.AbstractVerticle; +import io.vertx.core.MultiMap; +import io.vertx.core.Promise; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpMethod; +import io.vertx.core.http.HttpServer; +import io.vertx.core.http.HttpServerOptions; +import io.vertx.core.http.HttpServerRequest; +import io.vertx.ext.web.Router; +import io.vertx.ext.web.RoutingContext; +import io.vertx.ext.web.handler.BodyHandler; +import java.util.concurrent.CompletableFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FakeApiServer extends AbstractVerticle { + + private static final Logger log = LoggerFactory.getLogger(FakeApiServer.class); + + private final HttpServerOptions httpServerOptions; + + private HttpServer httpServer; + private volatile int port; + + private HttpMethod httpMethod; + private String path; + private Buffer body; + private MultiMap headers; + + private Object responseObject; + private Buffer responseBuffer; + private volatile CompletableFuture bodyFuture; + private boolean connectionClosed; + private int errorCode = -1; + + public FakeApiServer(final HttpServerOptions httpServerOptions) { + this.httpServerOptions = httpServerOptions; + } + + @Override + public void start(final Promise startPromise) { + httpServer = vertx.createHttpServer(httpServerOptions).requestHandler(setupRouter()) + .exceptionHandler(FakeApiServer::unhandledExceptionHandler); + httpServer.listen(ar -> { + if (ar.succeeded()) { + port = ar.result().actualPort(); + startPromise.complete(); + } else { + startPromise.fail(ar.cause()); + } + }); + } + + @Override + public void stop(final Promise stopPromise) { + if (httpServer == null) { + stopPromise.complete(); + } else { + httpServer.close(stopPromise.future()); + } + } + + private Router setupRouter() { + final Router router = Router.router(vertx); + router.route() + .handler(BodyHandler.create()) + .handler(this::handleRequest); + return router; + } + + private static void unhandledExceptionHandler(Throwable t) { + t.printStackTrace(); + } + + private synchronized void handleRequest(final RoutingContext routingContext) { + HttpServerRequest request = routingContext.request(); + request.connection().closeHandler(v -> connectionClosed()); + + httpMethod = request.method(); + path = request.path(); + headers = request.headers(); + body = routingContext.getBody(); + if (bodyFuture != null) { + bodyFuture.complete(body); + } + if (errorCode != -1) { + request.response().setStatusCode(errorCode); + } + if (responseBuffer != null) { + request.response().end(responseBuffer); + } else if (responseObject != null) { + request.response().end(KsqlClientUtil.serialize(responseObject)); + } else { + request.response().end(); + } + } + + public synchronized HttpMethod getHttpMethod() { + return httpMethod; + } + + public synchronized String getPath() { + return path; + } + + public Buffer waitForRequestBody() throws Exception { + this.bodyFuture = new CompletableFuture<>(); + return bodyFuture.get(); + } + + public synchronized Buffer getBody() { + return body; + } + + public synchronized MultiMap getHeaders() { + return headers; + } + + public synchronized void setResponseObject(final Object responseBody) { + this.responseObject = responseBody; + } + + public synchronized void setResponseBuffer(final Buffer responseBuffer) { + this.responseBuffer = responseBuffer; + } + + public int getPort() { + return port; + } + + private synchronized void connectionClosed() { + connectionClosed = true; + } + + public synchronized boolean isConnectionClosed() { + return connectionClosed; + } + + public synchronized void setErrorCode(int errorCode) { + this.errorCode = errorCode; + } + + synchronized int getErrorCode() { + return errorCode; + } + +} + diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/HttpClientBuilderTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/HttpClientBuilderTest.java deleted file mode 100644 index 50f5750f3190..000000000000 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/HttpClientBuilderTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Copyright 2019 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.rest.client; - -import static org.hamcrest.Matchers.is; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import io.confluent.ksql.rest.client.ssl.SslClientConfigurer; -import io.confluent.rest.RestConfig; -import java.util.HashMap; -import java.util.Map; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class HttpClientBuilderTest { - - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - - @Mock - private ClientBuilder clientBuilder; - @Mock - private SslClientConfigurer sslClientConfigurer; - @Mock - private Client client; - private Map clientProps; - - @Before - public void setUp() { - clientProps = new HashMap<>(); - - when(clientBuilder.build()).thenReturn(client); - } - - @Test - public void shouldConfigureSslOnTheClient() { - // Given: - clientProps.put(RestConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, "/trust/store/path"); - - // When: - HttpClientBuilder.buildClient(clientBuilder, sslClientConfigurer, clientProps); - - // Then: - verify(sslClientConfigurer).configureSsl(clientBuilder, clientProps); - } - - @Test - public void shouldThrowIfFailedToConfigureClient() { - // Given: - when(clientBuilder.register(any(Object.class))).thenThrow(new RuntimeException("boom")); - - // Then: - expectedException.expect(KsqlRestClientException.class); - expectedException.expectMessage("Failed to configure rest client"); - expectedException.expectCause(hasMessage(is("boom"))); - - // When: - HttpClientBuilder.buildClient(clientBuilder, sslClientConfigurer, clientProps); - } -} \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientTest.java new file mode 100644 index 000000000000..9e39cdbc6d03 --- /dev/null +++ b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientTest.java @@ -0,0 +1,824 @@ +/* + * Copyright 2019 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.rest.client; + +import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.properties.LocalProperties; +import io.confluent.ksql.reactive.BaseSubscriber; +import io.confluent.ksql.rest.entity.ClusterStatusResponse; +import io.confluent.ksql.rest.entity.CommandStatus; +import io.confluent.ksql.rest.entity.CommandStatus.Status; +import io.confluent.ksql.rest.entity.CommandStatuses; +import io.confluent.ksql.rest.entity.HealthCheckResponse; +import io.confluent.ksql.rest.entity.HealthCheckResponseDetail; +import io.confluent.ksql.rest.entity.HeartbeatMessage; +import io.confluent.ksql.rest.entity.HeartbeatResponse; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.KsqlEntityList; +import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.confluent.ksql.rest.entity.KsqlHostInfoEntity; +import io.confluent.ksql.rest.entity.KsqlRequest; +import io.confluent.ksql.rest.entity.ServerInfo; +import io.confluent.ksql.rest.entity.StreamedRow; +import io.confluent.ksql.rest.entity.TopicDescription; +import io.confluent.ksql.test.util.secure.ClientTrustStore; +import io.confluent.ksql.test.util.secure.ServerKeyStore; +import io.confluent.ksql.util.VertxCompletableFuture; +import io.vertx.core.Context; +import io.vertx.core.Vertx; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpClientOptions; +import io.vertx.core.http.HttpMethod; +import io.vertx.core.http.HttpServerOptions; +import io.vertx.core.net.JksOptions; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.kafka.common.config.SslConfigs; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.reactivestreams.Subscription; + +public class KsqlClientTest { + + @Rule + public ExpectedException expectedEx = ExpectedException.none(); + + private Vertx vertx; + private FakeApiServer server; + private KsqlClient ksqlClient; + private String deploymentId; + private Map properties; + private URI serverUri; + + @Before + public void setUp() throws Exception { + vertx = Vertx.vertx(); + startServer(new HttpServerOptions().setPort(0).setHost("localhost")); + properties = ImmutableMap.of("auto.offset.reset", "earliest"); + serverUri = URI.create("http://localhost:" + server.getPort()); + createClient(Optional.empty()); + } + + @After + public void tearDown() throws Exception { + ksqlClient.close(); + stopServer(); + vertx.close(); + } + + @Test + public void shouldSendKsqlRequest() { + + // Given: + String ksql = "some ksql"; + Object expectedResponse = setupExpectedResponse(); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse resp = target.postKsqlRequest(ksql, Optional.of(123L)); + + // Then: + assertThat(resp.get(), is(expectedResponse)); + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + assertThat(server.getPath(), is("/ksql")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(getKsqlRequest(), + is(new KsqlRequest(ksql, properties, 123L))); + } + + @Test + public void shouldSendBasicAuthHeader() { + + // Given: + setupExpectedResponse(); + + BasicCredentials credentials = BasicCredentials.of("tim", "socks"); + ksqlClient.close(); + createClient(Optional.of(credentials)); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + target.postKsqlRequest("some ksql", Optional.of(123L)); + + // Then: + assertThat(server.getHeaders().get("Authorization"), is(toAuthHeader(credentials))); + + } + + @Test + public void shouldOverrideAuthHeader() { + + // Given: + setupExpectedResponse(); + + // When: + KsqlTarget target = ksqlClient.target(serverUri).authorizationHeader("other auth"); + target.postKsqlRequest("some ksql", Optional.of(123L)); + + // Then: + assertThat(server.getHeaders().get("Authorization"), is("other auth")); + } + + @Test + public void shouldOverrideProperties() { + + // Given: + setupExpectedResponse(); + Map props = new HashMap<>(); + props.put("enable.auto.commit", true); + + // When: + KsqlTarget target = ksqlClient.target(serverUri).properties(props); + target.postKsqlRequest("some ksql", Optional.of(123L)); + + // Then: + assertThat(getKsqlRequest().getStreamsProperties(), is(props)); + } + + @Test + public void shouldSendHeartbeatRequest() throws Exception { + + // Given: + KsqlHostInfoEntity entity = new KsqlHostInfoEntity(serverUri.getHost(), serverUri.getPort()); + long timestamp = System.currentTimeMillis(); + server.setResponseObject(new HeartbeatResponse(true)); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + target.postAsyncHeartbeatRequest(entity, timestamp); + + Buffer body = server.waitForRequestBody(); + HeartbeatMessage hbm = KsqlClientUtil.deserialize(body, HeartbeatMessage.class); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + assertThat(server.getPath(), is("/heartbeat")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(hbm, is(new HeartbeatMessage(entity, timestamp))); + } + + @Test + public void shouldRequestServerInfo() { + + // Given: + ServerInfo expectedResponse = new ServerInfo("someversion", + "kafkaclusterid", "ksqlserviceid"); + server.setResponseObject(expectedResponse); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerInfo(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(server.getBody().length(), is(0)); + assertThat(server.getPath(), is("/info")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(response.get(), is(expectedResponse)); + } + + @Test + public void shouldRequestServerHealthcheck() { + + // Given: + Map map = new HashMap<>(); + map.put("foo", new HealthCheckResponseDetail(true)); + HealthCheckResponse healthCheckResponse = new HealthCheckResponse(true, map); + server.setResponseObject(healthCheckResponse); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerHealth(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(server.getBody().length(), is(0)); + assertThat(server.getPath(), is("/healthcheck")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(response.get(), is(healthCheckResponse)); + } + + @Test + public void shouldRequestClusterStatus() { + + // Given: + ClusterStatusResponse clusterStatusResponse = new ClusterStatusResponse(new HashMap<>()); + server.setResponseObject(clusterStatusResponse); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getClusterStatus(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(server.getBody().length(), is(0)); + // Yikes - this is camel case! + assertThat(server.getPath(), is("/clusterStatus")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(response.get(), is(clusterStatusResponse)); + } + + @Test + public void shouldRequestStatuses() { + + // Given: + CommandStatuses commandStatuses = new CommandStatuses(new HashMap<>()); + server.setResponseObject(commandStatuses); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getStatuses(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(server.getBody().length(), is(0)); + assertThat(server.getPath(), is("/status")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(response.get(), is(commandStatuses)); + } + + @Test + public void shouldRequestStatus() { + + // Given: + CommandStatus commandStatus = new CommandStatus(Status.SUCCESS, "msg"); + server.setResponseObject(commandStatus); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getStatus("foo"); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(server.getBody().length(), is(0)); + assertThat(server.getPath(), is("/status/foo")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(response.get(), is(commandStatus)); + } + + @Test + public void shouldPostQueryRequest() { + + // Given: + List expectedResponse = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + GenericRow row = GenericRow.genericRow("foo", 123, true); + StreamedRow sr = StreamedRow.row(row); + expectedResponse.add(sr); + } + server.setResponseBuffer(createResponseBuffer(expectedResponse)); + String sql = "some sql"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target.postQueryRequest(sql, Optional.of(321L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + + assertThat(server.getPath(), is("/query")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(getKsqlRequest(), is(new KsqlRequest(sql, properties, 321L))); + assertThat(response.get(), is(expectedResponse)); + } + + @Test + public void shouldPostQueryRequestStreamed() throws Exception { + + // Given: + + int numRows = 10; + List expectedResponse = setQueryStreamResponse(numRows, false); + String sql = "some sql"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target + .postQueryRequestStreamed(sql, Optional.of(321L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + + assertThat(server.getPath(), is("/query")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(getKsqlRequest(), is(new KsqlRequest(sql, properties, 321L))); + + List rows = getElementsFromPublisher(numRows, response.getResponse()); + assertThat(rows, is(expectedResponse)); + } + + @Test + public void shouldPostQueryRequestStreamedWithLimit() throws Exception { + + // Given: + + int numRows = 10; + List expectedResponse = setQueryStreamResponse(numRows, true); + String sql = "whateva"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target + .postQueryRequestStreamed(sql, Optional.of(321L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + + assertThat(server.getPath(), is("/query")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(getKsqlRequest(), is(new KsqlRequest(sql, properties, 321L))); + + List rows = getElementsFromPublisher(numRows + 1, response.getResponse()); + assertThat(rows, is(expectedResponse)); + } + + @Test + public void shouldCloseConnectionWhenQueryStreamIsClosed() throws Exception { + + // Given: + setQueryStreamResponse(1, false); + String sql = "some sql"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target + .postQueryRequestStreamed(sql, Optional.of(321L)); + + // Then: + assertThat(getKsqlRequest(), is(new KsqlRequest(sql, properties, 321L))); + + // When: + response.getResponse().close(); + + // Then: + assertThatEventually(() -> server.isConnectionClosed(), is(true)); + } + + @Test + public void shouldExecutePrintTopic() throws Exception { + + // Given: + int numRows = 10; + List expectedResponse = setupPrintTopicResponse(numRows); + String command = "print topic whatever"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target + .postPrintTopicRequest(command, Optional.of(123L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + + assertThat(server.getPath(), is("/query")); + assertThat(server.getHeaders().get("Accept"), is("application/json")); + assertThat(getKsqlRequest(), is(new KsqlRequest(command, properties, 123L))); + + List lines = getElementsFromPublisher(numRows, response.getResponse()); + assertThat(lines, is(expectedResponse)); + } + + @Test + public void shouldCloseConnectionWhenPrintTopicPublisherIsClosed() throws Exception { + + // Given: + setupPrintTopicResponse(1); + String command = "print topic whatever"; + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse> response = target + .postPrintTopicRequest(command, Optional.of(123L)); + + // Then: + assertThat(getKsqlRequest(), is(new KsqlRequest(command, properties, 123L))); + + // When: + response.getResponse().close(); + + // Then: + assertThatEventually(() -> server.isConnectionClosed(), is(true)); + } + + @Test + public void shouldPerformRequestWithTls() throws Exception { + ksqlClient.close(); + stopServer(); + + // Given: + startServerWithTls(); + startClientWithTls(); + + KsqlEntityList expectedResponse = setupExpectedResponse(); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse resp = target.postKsqlRequest("ssl test", Optional.of(123L)); + + // Then: + assertThat(getKsqlRequest().getKsql(), is("ssl test")); + assertThat(expectedResponse, is(resp.getResponse())); + } + + @Test + public void shouldFailToStartClientRequestWithNullKeystorePassword() throws Exception { + ksqlClient.close(); + stopServer(); + + // Given: + startServerWithTls(); + expectedEx.expect(KsqlRestClientException.class); + expectedEx + .expectMessage( + "java.io.IOException: Keystore was tampered with, or password was incorrect"); + + // When: + startClientWithTlsAndTruststorePassword(null); + } + + + @Test + public void shouldFailToStartClientRequestWithInvalidKeystorePassword() throws Exception { + ksqlClient.close(); + stopServer(); + + // Given: + startServerWithTls(); + expectedEx.expect(KsqlRestClientException.class); + expectedEx + .expectMessage( + "java.io.IOException: Keystore was tampered with, or password was incorrect"); + + // When: + startClientWithTlsAndTruststorePassword("iquwhduiqhwd"); + } + + @Test + public void shouldFailtoMakeHttpRequestWhenServerIsHttps() throws Exception { + ksqlClient.close(); + stopServer(); + + // Given: + startServerWithTls(); + startClientWithTls(); + expectedEx.expect(KsqlRestClientException.class); + expectedEx + .expectMessage("Cannot make request with scheme http as client is configured with tls"); + + // When: + URI uri = URI.create("http://localhost:" + server.getPort()); + KsqlTarget target = ksqlClient.target(uri); + target.postKsqlRequest("ssl test", Optional.of(123L)); + } + + @Test + public void shouldFailtoMakeHttpsRequestWhenServerIsHttp() { + // Given: + expectedEx.expect(KsqlRestClientException.class); + expectedEx + .expectMessage( + "Cannot make request with scheme https as client is configured without tls"); + + // When: + URI uri = URI.create("https://localhost:" + server.getPort()); + KsqlTarget target = ksqlClient.target(uri); + target.postKsqlRequest("ssl test", Optional.of(123L)); + } + + @Test + public void shouldHandleUnauthorizedOnPostRequests() { + + // Given: + server.setErrorCode(401); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.postKsqlRequest("sql", Optional.of(123L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("Could not authenticate successfully with the supplied credentials.")); + } + + @Test + public void shouldHandleUnauthorizedOnGetRequests() { + + // Given: + server.setErrorCode(401); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerInfo(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("Could not authenticate successfully with the supplied credentials.")); + } + + @Test + public void shouldHandleErrorMessageOnPostRequests() { + // Given: + KsqlErrorMessage ksqlErrorMessage = new KsqlErrorMessage(40000, "ouch", + ImmutableList.of("s1", "s2")); + server.setResponseObject(ksqlErrorMessage); + server.setErrorCode(400); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.postKsqlRequest("sql", Optional.of(123L)); + + // Then: + assertThat(response.getStatusCode().getCode(), is(400)); + assertThat(response.getErrorMessage().getErrorCode(), is(40000)); + assertThat(response.getErrorMessage().getMessage(), is("ouch")); + assertThat(response.getErrorMessage().getStackTrace(), is(ImmutableList.of("s1", "s2"))); + } + + @Test + public void shouldHandleErrorMessageOnGetRequests() { + // Given: + server.setResponseObject(new KsqlErrorMessage(40000, "ouch", ImmutableList.of("s1", "s2"))); + server.setErrorCode(400); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerInfo(); + + // Then: + assertThat(response.getStatusCode().getCode(), is(400)); + assertThat(response.getErrorMessage().getErrorCode(), is(40000)); + assertThat(response.getErrorMessage().getMessage(), is("ouch")); + assertThat(response.getErrorMessage().getStackTrace(), is(ImmutableList.of("s1", "s2"))); + } + + @Test + public void shouldHandleForbiddenOnPostRequests() { + + // Given: + server.setErrorCode(403); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.postKsqlRequest("sql", Optional.of(123L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("You are forbidden from using this cluster.")); + } + + @Test + public void shouldHandleForbiddenOnGetRequests() { + + // Given: + server.setErrorCode(403); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerInfo(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("You are forbidden from using this cluster.")); + } + + @Test + public void shouldHandleArbitraryErrorsOnPostRequests() { + + // Given: + server.setErrorCode(417); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.postKsqlRequest("sql", Optional.of(123L)); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.POST)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("The server returned an unexpected error: Expectation Failed")); + } + + @Test + public void shouldHandleArbitraryErrorsOnGetRequests() { + + // Given: + server.setErrorCode(417); + + // When: + KsqlTarget target = ksqlClient.target(serverUri); + RestResponse response = target.getServerInfo(); + + // Then: + assertThat(server.getHttpMethod(), is(HttpMethod.GET)); + assertThat(response.isErroneous(), is(true)); + assertThat(response.getErrorMessage().getMessage(), + is("The server returned an unexpected error: Expectation Failed")); + } + + private List setQueryStreamResponse(int numRows, boolean limitReached) { + List expectedResponse = new ArrayList<>(); + for (int i = 0; i < numRows; i++) { + GenericRow row = GenericRow.genericRow("foo", 123, true); + StreamedRow sr = StreamedRow.row(row); + expectedResponse.add(sr); + } + if (limitReached) { + expectedResponse.add(StreamedRow.finalMessage("Limit reached")); + } + server.setResponseBuffer(createResponseBuffer(expectedResponse)); + return expectedResponse; + } + + private List setupPrintTopicResponse(int numRows) { + List expectedResponse = new ArrayList<>(); + Buffer responseBuffer = Buffer.buffer(); + for (int i = 0; i < numRows; i++) { + String line = "this is row " + i; + expectedResponse.add(line); + if (i % 2 == 0) { + // The server can include empty lines - we need to test this too + responseBuffer.appendString("\n"); + } + responseBuffer.appendString(line).appendString("\n"); + } + server.setResponseBuffer(responseBuffer); + return expectedResponse; + } + + private void startClientWithTls() { + Map props = new HashMap<>(); + props.putAll(ClientTrustStore.trustStoreProps()); + props.put(KsqlClient.TLS_ENABLED_PROP_NAME, "true"); + createClient(props); + } + + private void startClientWithTlsAndTruststorePassword(final String password) { + Map props = new HashMap<>(); + props.putAll(ClientTrustStore.trustStoreProps()); + props.put(KsqlClient.TLS_ENABLED_PROP_NAME, "true"); + props.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, password); + createClient(props); + } + + private void startServerWithTls() throws Exception { + HttpServerOptions serverOptions = new HttpServerOptions().setPort(0) + .setHost("localhost") + .setSsl(true) + .setKeyStoreOptions(new JksOptions() + .setPath(ServerKeyStore.keyStoreProps().get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) + .setPassword( + ServerKeyStore.keyStoreProps().get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG))); + + startServer(serverOptions); + serverUri = URI.create("https://localhost:" + server.getPort()); + } + + private void startServer(HttpServerOptions httpServerOptions) throws Exception { + server = new FakeApiServer(httpServerOptions); + VertxCompletableFuture deployFuture = new VertxCompletableFuture<>(); + vertx.deployVerticle(server, deployFuture); + deploymentId = deployFuture.get(); + } + + private void stopServer() throws Exception { + VertxCompletableFuture undeployFuture = new VertxCompletableFuture<>(); + vertx.undeploy(deploymentId, undeployFuture); + undeployFuture.get(); + } + + private List getElementsFromPublisher(int numElements, StreamPublisher publisher) + throws Exception { + CompletableFuture> future = new CompletableFuture<>(); + CollectSubscriber subscriber = new CollectSubscriber<>(vertx.getOrCreateContext(), + future, numElements); + publisher.subscribe(subscriber); + return future.get(); + } + + private KsqlEntityList setupExpectedResponse() { + TopicDescription topicDescription = new TopicDescription("statement", "name", + "kafkatopic", "format", "schemaString"); + List entities = new ArrayList<>(); + entities.add(topicDescription); + KsqlEntityList expectedResponse = new KsqlEntityList(entities); + server.setResponseObject(expectedResponse); + return expectedResponse; + } + + private void createClient(Optional credentials) { + ksqlClient = new KsqlClient(new HashMap<>(), credentials, + new LocalProperties(properties), + new HttpClientOptions().setVerifyHost(false)); + } + + private void createClient(Map clientProps) { + ksqlClient = new KsqlClient(clientProps, Optional.empty(), + new LocalProperties(properties), + new HttpClientOptions().setVerifyHost(false)); + } + + private String toAuthHeader(BasicCredentials credentials) { + return "Basic " + Base64.getEncoder() + .encodeToString((credentials.username() + + ":" + credentials.password()).getBytes(StandardCharsets.UTF_8)); + } + + private KsqlRequest getKsqlRequest() { + return KsqlClientUtil.deserialize(server.getBody(), KsqlRequest.class); + } + + private static Buffer createResponseBuffer(List rows) { + // The old API returns rows in a strange format - it looks like a JSON array but it isn't. + // There are newlines separating the elements which are illegal in JSON + // And there can be an extra comma after the last element + // There can also be random newlines in the response + Buffer buffer = Buffer.buffer(); + buffer.appendString("["); + for (int i = 0; i < rows.size(); i++) { + buffer.appendBuffer(KsqlClientUtil.serialize(rows.get(i))); + buffer.appendString(",\n"); + if (i == rows.size() / 2) { + // insert a random newline for good measure - the server can actually do this + buffer.appendString("\n"); + } + } + buffer.appendString("]"); + return buffer; + } + + private static class CollectSubscriber extends BaseSubscriber { + + private final CompletableFuture> future; + private final List list = new ArrayList<>(); + private final int numRows; + + public CollectSubscriber(final Context context, CompletableFuture> future, + final int numRows) { + super(context); + this.future = future; + this.numRows = numRows; + } + + @Override + protected void afterSubscribe(final Subscription subscription) { + makeRequest(1); + } + + @Override + protected void handleValue(final T value) { + list.add(value); + if (list.size() == numRows) { + future.complete(new ArrayList<>(list)); + } + makeRequest(1); + } + + @Override + protected void handleComplete() { + future.complete(new ArrayList<>(list)); + } + + @Override + protected void handleError(final Throwable t) { + future.completeExceptionally(t); + } + } + +} \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientUtilTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientUtilTest.java index cf03616a8441..6fa7522ea9f2 100644 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientUtilTest.java +++ b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlClientUtilTest.java @@ -18,6 +18,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.never; @@ -26,9 +27,13 @@ import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.confluent.ksql.rest.entity.KsqlRequest; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.http.HttpClientResponse; +import io.vertx.core.json.JsonObject; +import java.util.HashMap; +import java.util.Map; import java.util.function.Function; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; import org.eclipse.jetty.http.HttpStatus.Code; import org.junit.Before; import org.junit.Test; @@ -36,6 +41,7 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; + @RunWith(MockitoJUnitRunner.class) public class KsqlClientUtilTest { @@ -43,25 +49,24 @@ public class KsqlClientUtilTest { private static final String ERROR_REASON = "something bad"; @Mock - private Response response; - @Mock - private Function mapper; + private ResponseWithBody response; @Mock - private KsqlEntityList entities; + private HttpClientResponse httpClientResponse; @Mock - private KsqlErrorMessage errorMessage; + private Function mapper; @Mock - private Response.StatusType statusInfo; + private KsqlEntityList entities; @Before public void setUp() { + when(response.getResponse()).thenReturn(httpClientResponse); when(mapper.apply(response)).thenReturn(entities); } @Test public void shouldCreateRestResponseFromSuccessfulResponse() { // Given: - when(response.getStatus()).thenReturn(Status.OK.getStatusCode()); + when(httpClientResponse.statusCode()).thenReturn(Code.OK.getCode()); // When: final RestResponse restResponse = @@ -76,8 +81,9 @@ public void shouldCreateRestResponseFromSuccessfulResponse() { @Test public void shouldCreateRestResponseFromUnsuccessfulResponseWithMessage() { // Given: - when(response.getStatus()).thenReturn(Status.BAD_REQUEST.getStatusCode()); - when(response.readEntity(KsqlErrorMessage.class)).thenReturn(errorMessage); + KsqlErrorMessage errorMessage = new KsqlErrorMessage(12345, "foobar"); + when(httpClientResponse.statusCode()).thenReturn(Code.BAD_REQUEST.getCode()); + when(response.getBody()).thenReturn(KsqlClientUtil.serialize(errorMessage)); // When: final RestResponse restResponse = @@ -90,10 +96,11 @@ public void shouldCreateRestResponseFromUnsuccessfulResponseWithMessage() { verify(mapper, never()).apply(any()); } + @Test public void shouldCreateRestResponseFromNotFoundResponse() { // Given: - when(response.getStatus()).thenReturn(Status.NOT_FOUND.getStatusCode()); + when(httpClientResponse.statusCode()).thenReturn(Code.NOT_FOUND.getCode()); // When: final RestResponse restResponse = @@ -103,13 +110,14 @@ public void shouldCreateRestResponseFromNotFoundResponse() { assertThat("is erroneous", restResponse.isErroneous()); assertThat(restResponse.getStatusCode(), is(Code.NOT_FOUND)); assertThat(restResponse.getErrorMessage().getMessage(), - containsString("Check your ksql http url to make sure you are connecting to a ksql server")); + containsString( + "Check your ksql http url to make sure you are connecting to a ksql server")); } @Test public void shouldCreateRestResponseFromUnauthorizedResponse() { // Given: - when(response.getStatus()).thenReturn(Status.UNAUTHORIZED.getStatusCode()); + when(httpClientResponse.statusCode()).thenReturn(Code.UNAUTHORIZED.getCode()); // When: final RestResponse restResponse = @@ -125,7 +133,7 @@ public void shouldCreateRestResponseFromUnauthorizedResponse() { @Test public void shouldCreateRestResponseFromForbiddenResponse() { // Given: - when(response.getStatus()).thenReturn(Status.FORBIDDEN.getStatusCode()); + when(httpClientResponse.statusCode()).thenReturn(Code.FORBIDDEN.getCode()); // When: final RestResponse restResponse = @@ -141,9 +149,8 @@ public void shouldCreateRestResponseFromForbiddenResponse() { @Test public void shouldCreateRestResponseFromUnknownResponse() { // Given: - when(response.getStatus()).thenReturn(Status.INTERNAL_SERVER_ERROR.getStatusCode()); - when(response.getStatusInfo()).thenReturn(statusInfo); - when(statusInfo.getReasonPhrase()).thenReturn(ERROR_REASON); + when(httpClientResponse.statusCode()).thenReturn(Code.INTERNAL_SERVER_ERROR.getCode()); + when(httpClientResponse.statusMessage()).thenReturn(ERROR_REASON); // When: final RestResponse restResponse = @@ -157,4 +164,29 @@ public void shouldCreateRestResponseFromUnknownResponse() { assertThat(restResponse.getErrorMessage().getMessage(), containsString(ERROR_REASON)); } + + @Test + public void shouldSerialiseDeserialise() { + // Given: + Map props = new HashMap<>(); + props.put("auto.offset.reset", "latest"); + KsqlRequest request = new KsqlRequest("some ksql", props, 21345L); + + // When: + Buffer buff = KsqlClientUtil.serialize(request); + + // Then: + assertThat(buff, is(notNullValue())); + String expectedJson = "{\"ksql\":\"some ksql\",\"streamsProperties\":{\"auto.offset.reset\":\"" + + "latest\"},\"commandSequenceNumber\":21345}"; + assertThat(new JsonObject(buff), is(new JsonObject(expectedJson))); + + // When: + KsqlRequest deserialised = KsqlClientUtil + .deserialize(Buffer.buffer(expectedJson), KsqlRequest.class); + + // Then: + assertThat(deserialised, is(request)); + } + } \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientTest.java index b2f27c8ee454..79df0d0395c2 100644 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientTest.java +++ b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlRestClientTest.java @@ -15,7 +15,11 @@ package io.confluent.ksql.rest.client; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + import io.confluent.ksql.properties.LocalProperties; +import java.net.URI; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -43,4 +47,31 @@ public void shouldThrowOnInvalidServerAddress() { // When: new KsqlRestClient(client, "timbuktu", localProps); } + + @Test + public void shouldParseSingleServerAddress() throws Exception { + final String singleServerAddress = "http://singleServer:8088"; + final URI singleServerURI = new URI(singleServerAddress); + + KsqlRestClient ksqlRestClient = new KsqlRestClient(client, singleServerAddress, localProps); + assertThat(ksqlRestClient.getServerAddress(), is(singleServerURI)); + } + + @Test + public void shouldParseMultipleServerAddresses() throws Exception { + final String firstServerAddress = "http://firstServer:8088"; + final String multipleServerAddresses = firstServerAddress + ",http://secondServer:8088"; + final URI firstServerURI = new URI(firstServerAddress); + KsqlRestClient ksqlRestClient = new KsqlRestClient(client, multipleServerAddresses, localProps); + assertThat(ksqlRestClient.getServerAddress(), is(firstServerURI)); + } + + @Test + public void shouldThrowIfAnyServerAddressIsInvalid() { + expectedException.expect(KsqlRestClientException.class); + expectedException + .expectMessage("The supplied serverAddress is invalid: secondBuggyServer.8088"); + new KsqlRestClient(client, "http://firstServer:8088,secondBuggyServer.8088", localProps); + } + } \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlTargetTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlTargetTest.java deleted file mode 100644 index 5ac65fb12f12..000000000000 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/KsqlTargetTest.java +++ /dev/null @@ -1,341 +0,0 @@ -/* - * Copyright 2019 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.rest.client; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.sameInstance; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import io.confluent.ksql.properties.LocalProperties; -import io.confluent.ksql.rest.entity.CommandStatus; -import io.confluent.ksql.rest.entity.CommandStatuses; -import io.confluent.ksql.rest.entity.HealthCheckResponse; -import io.confluent.ksql.rest.entity.KsqlEntityList; -import io.confluent.ksql.rest.entity.KsqlErrorMessage; -import io.confluent.ksql.rest.entity.KsqlRequest; -import io.confluent.ksql.rest.entity.ServerInfo; -import java.io.InputStream; -import java.util.Map; -import java.util.Optional; -import javax.ws.rs.client.Entity; -import javax.ws.rs.client.Invocation.Builder; -import javax.ws.rs.client.WebTarget; -import javax.ws.rs.core.HttpHeaders; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.MultivaluedHashMap; -import javax.ws.rs.core.MultivaluedMap; -import javax.ws.rs.core.Response; -import org.eclipse.jetty.http.HttpStatus.Code; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class KsqlTargetTest { - - private static final Optional NO_PREVIOUS_CMD = Optional.empty(); - - @Mock - private WebTarget webTarget; - @Mock - private Builder invocationBuilder; - @Mock - private Response response; - @Mock - private LocalProperties localProperties; - @Mock - private Map localPropertiesAsMap; - private KsqlTarget target; - - @Before - public void setUp() { - target = new KsqlTarget(webTarget, localProperties, Optional.empty()); - - when(webTarget.path(any())).thenReturn(webTarget); - when(webTarget.request(any(MediaType.class))).thenReturn(invocationBuilder); - when(invocationBuilder.headers(any())).thenReturn(invocationBuilder); - when(invocationBuilder.property(any(), any())).thenReturn(invocationBuilder); - when(invocationBuilder.get()).thenReturn(response); - when(invocationBuilder.post(any())).thenReturn(response); - when(response.getStatus()).thenReturn(Code.OK.getCode()); - - when(localProperties.toMap()).thenReturn(localPropertiesAsMap); - } - - @Test - public void shouldIssueGetRequests() { - // Given: - when(response.readEntity(ServerInfo.class)).thenReturn(mock(ServerInfo.class)); - - // When: - target.getServerInfo(); - - // Then: - verify(webTarget).path(any()); - verify(webTarget).request(MediaType.APPLICATION_JSON_TYPE); - verify(invocationBuilder).headers(any()); - verify(invocationBuilder).get(); - } - - @Test - public void shouldIssueGetRequestWithAuthHeader() { - // Given: - when(response.readEntity(ServerInfo.class)).thenReturn(mock(ServerInfo.class)); - - // When: - target - .authorizationHeader("Auth header") - .getServerInfo(); - - // Then: - verify(invocationBuilder).headers(authHeaders("Auth header")); - } - - @Test - public void shouldHandleGetFailure() { - // Given: - when(response.getStatus()).thenReturn(Code.BAD_REQUEST.getCode()); - when(response.getStatusInfo()).thenReturn(Response.Status.BAD_REQUEST); - - // When: - final RestResponse result = target.getServerInfo(); - - // Then: - assertThat("is erroneous", result.isErroneous()); - assertThat(result.get(), is(instanceOf(KsqlErrorMessage.class))); - assertThat(result.getStatusCode(), is(Code.BAD_REQUEST)); - } - - @Test - public void shouldIssuePostRequests() { - // When: - target.postPrintTopicRequest("statement", Optional.empty()); - - // Then: - verify(webTarget).path(any()); - verify(webTarget).request(MediaType.APPLICATION_JSON_TYPE); - verify(invocationBuilder).property(any(), any()); - verify(invocationBuilder).headers(any()); - verify(invocationBuilder).post(any()); - } - - @Test - public void shouldIssuePostRequestsWithAuthHeaders() { - // When: - target - .authorizationHeader("BASIC skhfhsknks") - .postPrintTopicRequest("statement", Optional.empty()); - - // Then: - verify(invocationBuilder).headers(authHeaders("BASIC skhfhsknks")); - } - - @Test - public void shouldHandlePostFailure() { - // Given: - when(response.getStatus()).thenReturn(Code.BAD_REQUEST.getCode()); - when(response.getStatusInfo()).thenReturn(Response.Status.BAD_REQUEST); - - // When: - final RestResponse result = target.postPrintTopicRequest("request", Optional.empty()); - - // Then: - assertThat("is erroneous", result.isErroneous()); - assertThat(result.get(), is(instanceOf(KsqlErrorMessage.class))); - assertThat(result.getStatusCode(), is(Code.BAD_REQUEST)); - } - - @Test - public void shouldGetServerInfo() { - // Given: - final ServerInfo serverInfo = mock(ServerInfo.class); - when(response.readEntity(ServerInfo.class)).thenReturn(serverInfo); - - // When: - final RestResponse result = target.getServerInfo(); - - // Then: - verify(webTarget).path("/info"); - verify(invocationBuilder).get(); - verify(response).close(); - assertThat(result.get(), is(sameInstance(serverInfo))); - } - - @Test - public void shouldGetServerHealth() { - // Given: - final HealthCheckResponse serverHealth = mock(HealthCheckResponse.class); - when(response.readEntity(HealthCheckResponse.class)).thenReturn(serverHealth); - - // When: - final RestResponse result = target.getServerHealth(); - - // Then: - verify(webTarget).path("/healthcheck"); - verify(invocationBuilder).get(); - verify(response).close(); - assertThat(result.get(), is(sameInstance(serverHealth))); - } - - @Test - public void shouldGetStatuses() { - // Given: - final CommandStatuses statuses = mock(CommandStatuses.class); - when(response.readEntity(CommandStatuses.class)).thenReturn(statuses); - - // When: - final RestResponse result = target.getStatuses(); - - // Then: - verify(webTarget).path("/status"); - verify(invocationBuilder).get(); - verify(response).close(); - assertThat(result.get(), is(sameInstance(statuses))); - } - - @Test - public void shouldGetStatus() { - // Given: - final CommandStatus status = mock(CommandStatus.class); - when(response.readEntity(CommandStatus.class)).thenReturn(status); - - // When: - final RestResponse result = target.getStatus("cmdId"); - - // Then: - verify(webTarget).path("/status/cmdId"); - verify(invocationBuilder).get(); - verify(response).close(); - assertThat(result.get(), is(sameInstance(status))); - } - - @Test - public void shouldPostKsqlRequest() { - // Given: - final KsqlEntityList entityList = mock(KsqlEntityList.class); - when(response.readEntity(KsqlEntityList.class)).thenReturn(entityList); - - // When: - final RestResponse result = target - .postKsqlRequest("Test request", NO_PREVIOUS_CMD); - - // Then: - verify(webTarget).path("/ksql"); - verify(invocationBuilder).post(jsonKsqlRequest("Test request", NO_PREVIOUS_CMD)); - verify(response).close(); - assertThat(result.get(), is(sameInstance(entityList))); - } - - @Test - public void shouldPostKsqlRequestWithPreviousCmdSeqNum() { - // Given: - final KsqlEntityList entityList = mock(KsqlEntityList.class); - when(response.readEntity(KsqlEntityList.class)).thenReturn(entityList); - - // When: - target - .postKsqlRequest("ksql request", Optional.of(24L)); - - // Then: - verify(invocationBuilder).post(jsonKsqlRequest("ksql request", Optional.of(24L))); - } - - @Test - public void shouldPostQueryRequest() { - // Given: - final InputStream is = mock(InputStream.class); - when(response.getEntity()).thenReturn(is); - - // When: - target - .postQueryRequest("query request", NO_PREVIOUS_CMD); - - // Then: - verify(webTarget).path("/query"); - verify(invocationBuilder).post(jsonKsqlRequest("query request", NO_PREVIOUS_CMD)); - verify(response, never()).close(); - } - - @Test - public void shouldPostQueryRequestWithPreviousCmdSeqNum() { - // Given: - final InputStream is = mock(InputStream.class); - when(response.getEntity()).thenReturn(is); - - // When: - target - .postQueryRequest("query request", Optional.of(42L)); - - // Then: - verify(invocationBuilder).post(jsonKsqlRequest("query request", Optional.of(42L))); - } - - @Test - public void shouldPostPrintTopicRequest() { - // Given: - final InputStream is = mock(InputStream.class); - when(response.getEntity()).thenReturn(is); - - // When: - final RestResponse result = target - .postPrintTopicRequest("print request", NO_PREVIOUS_CMD); - - // Then: - verify(webTarget).path("/query"); - verify(invocationBuilder).post(jsonKsqlRequest("print request", NO_PREVIOUS_CMD)); - verify(response, never()).close(); - assertThat(result.get(), is(is)); - } - - @Test - public void shouldPostPrintTopicRequestWithPreviousCmdSeqNum() { - // Given: - final InputStream is = mock(InputStream.class); - when(response.getEntity()).thenReturn(is); - - // When: - target - .postPrintTopicRequest("print request", Optional.of(42L)); - - // Then: - verify(invocationBuilder).post(jsonKsqlRequest("print request", Optional.of(42L))); - } - - private Entity jsonKsqlRequest( - final String ksql, - final Optional previousCmdSeqNum - ) { - return Entity.json(new KsqlRequest( - ksql, - localPropertiesAsMap, - previousCmdSeqNum.orElse(null) - )); - } - - private static MultivaluedMap authHeaders(final String value) { - final MultivaluedMap headers = new MultivaluedHashMap<>(); - headers.add(HttpHeaders.AUTHORIZATION, value); - return headers; - } -} \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurerTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurerTest.java deleted file mode 100644 index 7de4e62b6c8c..000000000000 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/DefaultSslClientConfigurerTest.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright 2019 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.rest.client.ssl; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.verify; - -import io.confluent.ksql.test.util.secure.ClientTrustStore; -import io.confluent.ksql.test.util.secure.ServerKeyStore; -import io.confluent.rest.RestConfig; -import java.util.HashMap; -import java.util.Map; -import javax.ws.rs.client.ClientBuilder; -import org.apache.kafka.common.config.SslConfigs; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class DefaultSslClientConfigurerTest { - - @Mock - private ClientBuilder clientBuilder; - private Map clientProps; - private DefaultSslClientConfigurer configurer; - - @Before - public void setUp() { - clientProps = new HashMap<>(); - configurer = new DefaultSslClientConfigurer(); - } - - @Test - public void shouldConfigureKeyStoreIfLocationSet() { - // Given: - clientProps.putAll(ServerKeyStore.keyStoreProps()); - final String keyPassword = clientProps.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG); - - // When: - configurer.configureSsl(clientBuilder, clientProps); - - // Then: - verify(clientBuilder).keyStore(any(), eq(keyPassword)); - } - - @Test - public void shouldConfigureTrustStoreIfLocationSet() { - // Given: - clientProps.putAll(ClientTrustStore.trustStoreProps()); - - // When: - configurer.configureSsl(clientBuilder, clientProps); - - // Then: - verify(clientBuilder).trustStore(any()); - } - - @Test - public void shouldConfigureHostNameVerifierSet() { - // Given: - clientProps.put(RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ""); - - // When: - configurer.configureSsl(clientBuilder, clientProps); - - // Then: - verify(clientBuilder).hostnameVerifier(any()); - } -} \ No newline at end of file diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/SslUtilTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/SslUtilTest.java deleted file mode 100644 index 0637022deb41..000000000000 --- a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/client/ssl/SslUtilTest.java +++ /dev/null @@ -1,234 +0,0 @@ -/* - * Copyright 2019 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.rest.client.ssl; - -import static java.util.Collections.emptyMap; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; - -import com.google.common.collect.ImmutableMap; -import io.confluent.ksql.test.util.secure.ClientTrustStore; -import io.confluent.ksql.test.util.secure.ServerKeyStore; -import io.confluent.ksql.util.KsqlException; -import io.confluent.rest.RestConfig; -import java.security.KeyStore; -import java.util.Map; -import java.util.Optional; -import org.apache.http.conn.ssl.NoopHostnameVerifier; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.SslConfigs; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -public class SslUtilTest { - - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - - @Test - public void shouldNotLoadKeyStoreByDefault() { - // When: - final Optional result = SslUtil.loadKeyStore(emptyMap()); - - // Then: - assertThat(result, is(Optional.empty())); - } - - @Test - public void shouldNotLoadTrustStoreByDefault() { - // When: - final Optional result = SslUtil.loadTrustStore(emptyMap()); - - // Then: - assertThat(result, is(Optional.empty())); - } - - @Test - public void shouldLoadKeyStore() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_KEYSTORE_LOCATION_CONFIG, - keyStoreProp(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG), - RestConfig.SSL_KEYSTORE_PASSWORD_CONFIG, - keyStoreProp(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) - ); - - // When: - final Optional result = SslUtil.loadKeyStore(props); - - // Then: - assertThat(result, is(not(Optional.empty()))); - } - - @Test - public void shouldLoadTrustStore() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, - trustStoreProp(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), - RestConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, - trustStoreProp(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) - ); - - // When: - final Optional result = SslUtil.loadTrustStore(props); - - // Then: - assertThat(result, is(not(Optional.empty()))); - } - - @Test - public void shouldThrowIfKeyStoreNotFound() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_KEYSTORE_LOCATION_CONFIG, "/will/not/find/me" - ); - - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("Failed to load keyStore: /will/not/find/me"); - - // When: - SslUtil.loadKeyStore(props); - } - - @Test - public void shouldThrowIfTrustStoreNotFound() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, "/will/not/find/me" - ); - - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("Failed to load keyStore: /will/not/find/me"); - - // When: - SslUtil.loadTrustStore(props); - } - - @Test - public void shouldThrowIfKeyStorePasswordWrong() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_KEYSTORE_LOCATION_CONFIG, - keyStoreProp(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG), - RestConfig.SSL_KEYSTORE_PASSWORD_CONFIG, - "wrong!" - ); - - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("Failed to load keyStore:"); - expectedException.expectCause(hasMessage(is( - "Keystore was tampered with, or password was incorrect"))); - - // When: - SslUtil.loadKeyStore(props); - } - - @Test - public void shouldThrowIfTrustStorePasswordWrong() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, - trustStoreProp(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), - RestConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, - "wrong!" - ); - - // Then: - expectedException.expect(KsqlException.class); - expectedException.expectMessage("Failed to load keyStore:"); - expectedException.expectCause(hasMessage(is( - "Keystore was tampered with, or password was incorrect"))); - - // When: - SslUtil.loadTrustStore(props); - } - - @Test - public void shouldDefaultToNoKeyPassword() { - assertThat(SslUtil.getKeyPassword(emptyMap()), is("")); - } - - @Test - public void shouldExtractKeyPassword() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_KEY_PASSWORD_CONFIG, "let me in" - ); - - // Then: - assertThat(SslUtil.getKeyPassword(props), is("let me in")); - } - - @Test - public void shouldDefaultToNoopHostNameVerification() { - assertThat(SslUtil.getHostNameVerifier(emptyMap()), - is(Optional.of(NoopHostnameVerifier.INSTANCE))); - } - - @Test - public void shouldSupportNoOpHostNameVerifier() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "" - ); - - // Then: - assertThat(SslUtil.getHostNameVerifier(props), - is(Optional.of(NoopHostnameVerifier.INSTANCE))); - } - - @Test - public void shouldSupportHttpsHostNameVerifier() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "httpS" - ); - - // Then: - assertThat(SslUtil.getHostNameVerifier(props), is(Optional.empty())); - } - - @Test - public void shouldThrowOnUnsupportedHostNameVerifier() { - // Given: - final Map props = ImmutableMap.of( - RestConfig.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "what?" - ); - - // Then: - expectedException.expect(ConfigException.class); - expectedException.expectMessage( - "Invalid value what? for configuration ssl.endpoint.identification.algorithm: Not supported"); - - // When: - SslUtil.getHostNameVerifier(props); - } - - private static String keyStoreProp(final String config) { - return ServerKeyStore.keyStoreProps().get(config); - } - - private static String trustStoreProp(final String config) { - return ClientTrustStore.trustStoreProps().get(config); - } -} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/AsyncAssert.java b/ksql-test-util/src/main/java/io/confluent/ksql/test/util/AsyncAssert.java similarity index 87% rename from ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/AsyncAssert.java rename to ksql-test-util/src/main/java/io/confluent/ksql/test/util/AsyncAssert.java index 13097a9158a0..fa76f5a7a89a 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/api/utils/AsyncAssert.java +++ b/ksql-test-util/src/main/java/io/confluent/ksql/test/util/AsyncAssert.java @@ -13,7 +13,7 @@ * specific language governing permissions and limitations under the License. */ -package io.confluent.ksql.api.utils; +package io.confluent.ksql.test.util; import static org.hamcrest.MatcherAssert.assertThat; @@ -23,8 +23,7 @@ public class AsyncAssert { private AssertionError error; - public synchronized void assertAsync(T t, - Matcher expected) { + public synchronized void assertAsync(final T t, final Matcher expected) { try { assertThat(t, expected); } catch (AssertionError e) { diff --git a/pom.xml b/pom.xml index 70c2359c55f9..b20a9e0f721c 100644 --- a/pom.xml +++ b/pom.xml @@ -120,6 +120,8 @@ 2.24.0 2.9.5 65.1 + 3.8.5 + 1.0.3 true true 5.5.0-ccs-SNAPSHOT