From 461c00aec7e633e8dd0a6fc8c3865e659e3636df Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Sun, 29 Mar 2020 16:37:54 -0700 Subject: [PATCH 01/19] feat: java client push/pull query support --- ksqldb-api-client/pom.xml | 89 +++++ .../io/confluent/ksql/api/client/Client.java | 69 ++++ .../ksql/api/client/ClientOptions.java | 63 ++++ .../confluent/ksql/api/client/InsertAck.java | 22 ++ .../ksql/api/client/QueryResult.java | 55 ++++ .../io/confluent/ksql/api/client/Row.java | 126 +++++++ .../ksql/api/client/impl/ClientImpl.java | 207 ++++++++++++ .../api/client/impl/ClientOptionsImpl.java | 190 +++++++++++ .../api/client/impl/PollableSubscriber.java | 113 +++++++ .../client/impl/PullQueryResponseHandler.java | 66 ++++ .../client/impl/PushQueryResponseHandler.java | 77 +++++ .../api/client/impl/QueryResponseHandler.java | 84 +++++ .../ksql/api/client/impl/QueryResultImpl.java | 102 ++++++ .../ksql/api/client/impl/RowImpl.java | 149 +++++++++ .../confluent/ksql/api/client/ClientTest.java | 308 ++++++++++++++++++ .../api/client/impl/QueryResultImplTest.java | 63 ++++ .../ksql/api/client/impl/RowImplTest.java | 74 +++++ .../protocol/QueryResponseMetadata.java | 7 +- .../io/confluent/ksql/api/BaseApiTest.java | 23 +- pom.xml | 1 + 20 files changed, 1876 insertions(+), 12 deletions(-) create mode 100644 ksqldb-api-client/pom.xml create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/InsertAck.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Row.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java create mode 100644 ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java create mode 100644 ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/QueryResultImplTest.java create mode 100644 ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java diff --git a/ksqldb-api-client/pom.xml b/ksqldb-api-client/pom.xml new file mode 100644 index 000000000000..5a616a00d3e4 --- /dev/null +++ b/ksqldb-api-client/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + + + io.confluent.ksql + ksqldb-parent + 6.0.0-SNAPSHOT + + + ksqldb-api-client + + + + io.confluent.ksql + ksqldb-rest-app + + + + com.fasterxml.jackson.datatype + jackson-datatype-jdk8 + ${jackson.version} + + + + io.vertx + vertx-core + ${vertx.version} + + + + org.reactivestreams + reactive-streams + ${reactive-streams.version} + + + + + + io.confluent.ksql + ksqldb-rest-app + ${project.parent.version} + test-jar + test + + + + io.confluent.ksql + ksqldb-test-util + ${project.version} + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java new file mode 100644 index 000000000000..bd830ea0a4c6 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java @@ -0,0 +1,69 @@ +/* + * 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.api.client; + +import io.confluent.ksql.api.client.impl.ClientImpl; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import org.reactivestreams.Publisher; + +public interface Client { + + /** + * Execute a push query. + * + * @param sql statement of query to execute. + * @return query result. + */ + CompletableFuture streamQuery(String sql); + + /** + * Execute a push query. + * + * @param sql statement of query to execute. + * @param properties query properties. + * @return query result. + */ + CompletableFuture streamQuery(String sql, Map properties); + + /** + * Execute a pull query. + * + * @param sql statement of query to execute. + * @return query result. + */ + CompletableFuture> executeQuery(String sql); + + /** + * Execute a pull query. + * + * @param sql statement of query to execute. + * @param properties query properties. + * @return query result. + */ + CompletableFuture> executeQuery(String sql, Map properties); + + CompletableFuture insertInto(String streamName, Map row); + + Publisher streamInserts(String streamName, Publisher> insertsPublisher); + + void close(); + + static Client create(ClientOptions clientOptions) { + return new ClientImpl(clientOptions); + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java new file mode 100644 index 000000000000..736099114e88 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java @@ -0,0 +1,63 @@ +/* + * 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.api.client; + +public interface ClientOptions { + + ClientOptions setHost(String host); + + ClientOptions setPort(int port); + + ClientOptions setUseTls(boolean useTls); + + ClientOptions setUseClientAuth(boolean useClientAuth); + + ClientOptions setTrustStore(String trustStorePath); + + ClientOptions setTrustStorePassword(String trustStorePassword); + + ClientOptions setKeyStore(String keyStorePath); + + ClientOptions setKeyStorePassword(String keyStorePassword); + + ClientOptions setBasicAuthCredentials(String username, String password); + + ClientOptions unsetBasicAuthCredentials(); + + String getHost(); + + int getPort(); + + boolean isUseTls(); + + boolean isUseClientAuth(); + + boolean isUseBasicAuth(); + + String getTrustStore(); + + String getTrustStorePassword(); + + String getKeyStore(); + + String getKeyStorePassword(); + + String getBasicAuthUsername(); + + String getBasicAuthPassword(); + + ClientOptions copy(); +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/InsertAck.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/InsertAck.java new file mode 100644 index 000000000000..8beca33e9e79 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/InsertAck.java @@ -0,0 +1,22 @@ +/* + * 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.api.client; + +public interface InsertAck { + + int getNum(); + +} diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java new file mode 100644 index 000000000000..849e798934ba --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java @@ -0,0 +1,55 @@ +/* + * 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.api.client; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.reactivestreams.Publisher; + +/** + * The result of a streaming (push) query. Records may be consumed by either subscribing to the + * publisher or polling (blocking) for one record at a time. These two methods of consumption are + * mutually exclusive; only one method may be used (per QueryResult). + */ +public interface QueryResult extends Publisher { + + List columnNames(); + + List columnTypes(); + + String queryID(); + + /** + * Block until a row becomes available. + * + * @return the row. + */ + Row poll(); + + /** + * Block until a row becomes available or the timeout has elapsed. + * + * @param timeout amount of to wait for a row. Non-positive values are interpreted as no timeout. + * @param timeUnit unit for timeout param. + * @return the row, if available; else, null. + */ + Row poll(long timeout, TimeUnit timeUnit); + + boolean isComplete(); + + void close(); + +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Row.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Row.java new file mode 100644 index 000000000000..f1cedf07d38f --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Row.java @@ -0,0 +1,126 @@ +/* + * 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.api.client; + +import java.util.List; + +/** + * A single record, returned as part of a query result. + */ +public interface Row { + + List columnNames(); + + List columnTypes(); + + List values(); + + /** + * Get the value for a particular column of the Row as an Object. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + Object getObject(int columnIndex); + + /** + * Get the value for a particular column of the Row as an Object. + * + * @param columnName name of column. + * @return column value. + */ + Object getObject(String columnName); + + /** + * Get the value for a particular column of the Row as a string. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + String getString(int columnIndex); + + /** + * Get the value for a particular column of the Row as a string. + * + * @param columnName name of column. + * @return column value. + */ + String getString(String columnName); + + /** + * Get the value for a particular column of the Row as an integer. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + Integer getInt(int columnIndex); + + /** + * Get the value for a particular column of the Row as an integer. + * + * @param columnName name of column. + * @return column value. + */ + Integer getInt(String columnName); + + /** + * Get the value for a particular column of the Row as a long. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + Long getLong(int columnIndex); + + /** + * Get the value for a particular column of the Row as a long. + * + * @param columnName name of column. + * @return column value. + */ + Long getLong(String columnName); + + /** + * Get the value for a particular column of the Row as a double. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + Double getDouble(int columnIndex); + + /** + * Get the value for a particular column of the Row as a double. + * + * @param columnName name of column. + * @return column value. + */ + Double getDouble(String columnName); + + /** + * Get the value for a particular column of the Row as a boolean. + * + * @param columnIndex index of column (1-indexed). + * @return column value. + */ + Boolean getBoolean(int columnIndex); + + /** + * Get the value for a particular column of the Row as a boolean. + * + * @param columnName name of column. + * @return column value. + */ + Boolean getBoolean(String columnName); +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java new file mode 100644 index 000000000000..feb9e3d82c5d --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -0,0 +1,207 @@ +/* + * 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.api.client.impl; + +import static io.netty.handler.codec.http.HttpHeaderNames.AUTHORIZATION; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; + +import io.confluent.ksql.api.client.Client; +import io.confluent.ksql.api.client.ClientOptions; +import io.confluent.ksql.api.client.InsertAck; +import io.confluent.ksql.api.client.QueryResult; +import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.rest.client.KsqlRestClientException; +import io.vertx.core.Context; +import io.vertx.core.Vertx; +import io.vertx.core.http.HttpClient; +import io.vertx.core.http.HttpClientOptions; +import io.vertx.core.http.HttpClientRequest; +import io.vertx.core.http.HttpClientResponse; +import io.vertx.core.http.HttpMethod; +import io.vertx.core.http.HttpVersion; +import io.vertx.core.json.JsonObject; +import io.vertx.core.net.JksOptions; +import io.vertx.core.net.SocketAddress; +import io.vertx.core.parsetools.RecordParser; +import java.nio.charset.Charset; +import java.util.Base64; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import org.reactivestreams.Publisher; + +public class ClientImpl implements Client { + + private final ClientOptions clientOptions; + private final Vertx vertx; + private final HttpClient httpClient; + private final SocketAddress serverSocketAddress; + private final boolean ownedVertx; + + public ClientImpl(final ClientOptions clientOptions) { + this(clientOptions, Vertx.vertx(), true); + } + + public ClientImpl(final ClientOptions clientOptions, final Vertx vertx) { + this(clientOptions, vertx, false); + } + + private ClientImpl(final ClientOptions clientOptions, final Vertx vertx, + final boolean ownedVertx) { + this.clientOptions = clientOptions.copy(); + this.vertx = vertx; + this.ownedVertx = ownedVertx; + this.httpClient = createHttpClient(vertx, clientOptions); + this.serverSocketAddress = io.vertx.core.net.SocketAddress + .inetSocketAddress(clientOptions.getPort(), clientOptions.getHost()); + } + + @Override + public CompletableFuture streamQuery(final String sql) { + return streamQuery(sql, Collections.emptyMap()); + } + + @Override + public CompletableFuture streamQuery( + final String sql, + final Map properties + ) { + return makeQueryRequest(sql, properties, PushQueryResponseHandler::new); + } + + @Override + public CompletableFuture> executeQuery(final String sql) { + return executeQuery(sql, Collections.emptyMap()); + } + + @Override + public CompletableFuture> executeQuery( + final String sql, + final Map properties + ) { + return makeQueryRequest(sql, properties, PullQueryResponseHandler::new); + } + + @Override + public CompletableFuture insertInto( + final String streamName, final Map row) { + return null; // not yet implemented + } + + @Override + public Publisher streamInserts( + String streamName, Publisher> insertsPublisher) { + return null; // not yet implemented + } + + @Override + public void close() { + httpClient.close(); + if (ownedVertx) { + vertx.close(); + } + } + + @FunctionalInterface + private interface ResponseHandlerSupplier { + QueryResponseHandler get(Context ctx, RecordParser recordParser, CompletableFuture cf); + } + + private CompletableFuture makeQueryRequest( + final String sql, + final Map properties, + final ResponseHandlerSupplier responseHandlerSupplier + ) { + + final JsonObject requestBody = new JsonObject().put("sql", sql).put("properties", properties); + + final CompletableFuture cf = new CompletableFuture<>(); + + HttpClientRequest request = httpClient.request(HttpMethod.POST, + serverSocketAddress, clientOptions.getPort(), clientOptions.getHost(), + "/query-stream", + response -> handleResponse(response, cf, responseHandlerSupplier)) + .exceptionHandler(e -> handleRequestException(e, cf)); + if (clientOptions.isUseBasicAuth()) { + request = configureBasicAuth(request); + } + request.end(requestBody.toBuffer()); + + return cf; + } + + private HttpClientRequest configureBasicAuth(final HttpClientRequest request) { + final String creds = clientOptions.getBasicAuthUsername() + + ":" + + clientOptions.getBasicAuthPassword(); + final String base64creds = + Base64.getEncoder().encodeToString(creds.getBytes(Charset.defaultCharset())); + return request.putHeader(AUTHORIZATION.toString(), "Basic " + base64creds); + } + + private static void handleRequestException(final Throwable t, final CompletableFuture cf) { + cf.completeExceptionally(t); + } + + private static void handleResponse( + final HttpClientResponse response, + final CompletableFuture cf, + final ResponseHandlerSupplier responseHandlerSupplier) { + if (response.statusCode() == OK.code()) { + final RecordParser recordParser = RecordParser.newDelimited("\n", response); + final QueryResponseHandler responseHandler = + responseHandlerSupplier.get(Vertx.currentContext(), recordParser, cf); + + recordParser.handler(responseHandler::handleBodyBuffer); + recordParser.endHandler(responseHandler::handleBodyEnd); + } else { + response.bodyHandler(buffer -> { + final JsonObject errorResponse = buffer.toJsonObject(); + cf.completeExceptionally(new KsqlRestClientException(String.format( + "Received %d response from server: %s. Error code: %d", + response.statusCode(), + errorResponse.getString("message"), + errorResponse.getInteger("errorCode") + ))); + }); + } + } + + private static HttpClient createHttpClient(final Vertx vertx, final ClientOptions clientOptions) { + HttpClientOptions options = new HttpClientOptions() + .setSsl(clientOptions.isUseTls()) + .setUseAlpn(true) + .setProtocolVersion(HttpVersion.HTTP_2) + .setDefaultHost(clientOptions.getHost()) + .setDefaultPort(clientOptions.getPort()); + if (clientOptions.isUseTls() && !clientOptions.getTrustStore().isEmpty()) { + options = options.setTrustStoreOptions( + new JksOptions() + .setPath(clientOptions.getTrustStore()) + .setPassword(clientOptions.getTrustStorePassword()) + ); + } + if (clientOptions.isUseClientAuth()) { + options = options.setKeyStoreOptions( + new JksOptions() + .setPath(clientOptions.getKeyStore()) + .setPassword(clientOptions.getKeyStorePassword()) + ); + } + return vertx.createHttpClient(options); + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java new file mode 100644 index 000000000000..3af60b1ce6e3 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java @@ -0,0 +1,190 @@ +/* + * 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.api.client.impl; + +import io.confluent.ksql.api.client.ClientOptions; +import java.util.Objects; + +public class ClientOptionsImpl implements ClientOptions { + + private String host = "localhost"; + private int port = 8088; + private boolean useTls = false; + private boolean useClientAuth = false; + private boolean useBasicAuth = false; + private String trustStorePath = ""; + private String trustStorePassword = ""; + private String keyStorePath = ""; + private String keyStorePassword = ""; + private String basicAuthUsername = ""; + private String basicAuthPassword = ""; + + public ClientOptionsImpl() { + } + + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck + private ClientOptionsImpl( + // CHECKSTYLE_RULES.ON: ParameterNumberCheck + final String host, final int port, + final boolean useTls, final boolean useClientAuth, final boolean useBasicAuth, + final String trustStorePath, final String trustStorePassword, + final String keyStorePath, final String keyStorePassword, + final String basicAuthUsername, final String basicAuthPassword) { + this.host = Objects.requireNonNull(host); + this.port = port; + this.useTls = useTls; + this.useClientAuth = useClientAuth; + this.useBasicAuth = useBasicAuth; + this.trustStorePath = Objects.requireNonNull(trustStorePath); + this.trustStorePassword = Objects.requireNonNull(trustStorePassword); + this.keyStorePath = Objects.requireNonNull(keyStorePath); + this.keyStorePassword = Objects.requireNonNull(keyStorePassword); + this.basicAuthUsername = Objects.requireNonNull(basicAuthUsername); + this.basicAuthPassword = Objects.requireNonNull(basicAuthPassword); + } + + @Override + public ClientOptions setHost(final String host) { + this.host = Objects.requireNonNull(host); + return this; + } + + @Override + public ClientOptions setPort(final int port) { + this.port = port; + return this; + } + + @Override + public ClientOptions setUseTls(final boolean useTls) { + this.useTls = useTls; + return this; + } + + @Override + public ClientOptions setUseClientAuth(final boolean useClientAuth) { + this.useClientAuth = useClientAuth; + return this; + } + + @Override + public ClientOptions setTrustStore(final String trustStorePath) { + this.trustStorePath = Objects.requireNonNull(trustStorePath); + return this; + } + + @Override + public ClientOptions setTrustStorePassword(final String trustStorePassword) { + this.trustStorePassword = Objects.requireNonNull(trustStorePassword); + return this; + } + + @Override + public ClientOptions setKeyStore(final String keyStorePath) { + this.keyStorePath = Objects.requireNonNull(keyStorePath); + return this; + } + + @Override + public ClientOptions setKeyStorePassword(final String keyStorePassword) { + this.keyStorePassword = Objects.requireNonNull(keyStorePassword); + return this; + } + + @Override + public ClientOptions setBasicAuthCredentials(final String username, final String password) { + Objects.requireNonNull(username); + Objects.requireNonNull(password); + + this.useBasicAuth = true; + this.basicAuthUsername = username; + this.basicAuthPassword = password; + return this; + } + + @Override + public ClientOptions unsetBasicAuthCredentials() { + this.useBasicAuth = false; + this.basicAuthUsername = ""; + this.basicAuthPassword = ""; + return this; + } + + @Override + public String getHost() { + return host; + } + + @Override + public int getPort() { + return port; + } + + @Override + public boolean isUseTls() { + return useTls; + } + + @Override + public boolean isUseClientAuth() { + return useClientAuth; + } + + @Override + public boolean isUseBasicAuth() { + return useClientAuth; + } + + @Override + public String getTrustStore() { + return trustStorePath; + } + + @Override + public String getTrustStorePassword() { + return trustStorePassword; + } + + @Override + public String getKeyStore() { + return keyStorePath; + } + + @Override + public String getKeyStorePassword() { + return keyStorePassword; + } + + @Override + public String getBasicAuthUsername() { + return basicAuthUsername; + } + + @Override + public String getBasicAuthPassword() { + return basicAuthPassword; + } + + @Override + public ClientOptions copy() { + return new ClientOptionsImpl( + host, port, + useTls, useClientAuth, useBasicAuth, + trustStorePath, trustStorePassword, + keyStorePath, keyStorePassword, + basicAuthUsername, basicAuthPassword); + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java new file mode 100644 index 000000000000..41175c86f842 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java @@ -0,0 +1,113 @@ +/* + * 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.api.client.impl; + +import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.reactive.BaseSubscriber; +import io.vertx.core.Context; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import org.reactivestreams.Subscription; + +public class PollableSubscriber extends BaseSubscriber { + + private static final int REQUEST_BATCH_SIZE = 100; + // 100ms in ns + private static final long MAX_POLL_NANOS = TimeUnit.MILLISECONDS.toNanos(100); + + private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private int tokens; + private Throwable error; + private volatile boolean closed; + + public PollableSubscriber(final Context context) { + super(context); + } + + @Override + protected void afterSubscribe(final Subscription subscription) { + checkRequestTokens(); + } + + @Override + protected void handleValue(final Row row) { + queue.add(row); + } + + @Override + protected synchronized void handleError(final Throwable t) { + System.out.println("pollable subscriber encountered error: " + t); + error = t; + } + + @Override + protected synchronized void handleComplete() { + close(); + } + + public synchronized Row poll(final long timeout, final TimeUnit timeUnit) throws Throwable { + if (closed) { + return null; + } + final long timeoutNs = timeUnit.toNanos(timeout); + final long end; + long remainingTime; + if (timeoutNs > 0) { + end = System.nanoTime() + timeoutNs; + remainingTime = timeoutNs; + } else { + end = Long.MAX_VALUE; + remainingTime = Long.MAX_VALUE; + } + do { + // Poll in smaller units so we can exit on close + final long pollTime = Math.min(remainingTime, MAX_POLL_NANOS); + try { + final Row row = queue.poll(pollTime, TimeUnit.NANOSECONDS); + if (row != null) { + tokens--; + checkRequestTokens(); + return row; + } + if (error != null) { + final Throwable error = this.error; + this.error = null; + throw error; + } + } catch (InterruptedException e) { + return null; + } + remainingTime = end - System.nanoTime(); + } while (!closed && remainingTime > 0); + return null; + } + + public void close() { + closed = true; + } + + synchronized boolean isClosed() { + return closed; + } + + private void checkRequestTokens() { + if (tokens == 0) { + tokens += REQUEST_BATCH_SIZE; + makeRequest(REQUEST_BATCH_SIZE); + } + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java new file mode 100644 index 000000000000..50fb62e72f58 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java @@ -0,0 +1,66 @@ +/* + * 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.api.client.impl; + +import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; +import io.confluent.ksql.rest.client.KsqlRestClientException; +import io.vertx.core.Context; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.json.JsonArray; +import io.vertx.core.parsetools.RecordParser; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public class PullQueryResponseHandler extends QueryResponseHandler> { + + private final List rows; + private List columnNames; + private List columnTypes; + + PullQueryResponseHandler(final Context context, final RecordParser recordParser, + final CompletableFuture> cf) { + super(context, recordParser, cf); + this.rows = new ArrayList<>(); + } + + @Override + protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) { + if (queryResponseMetadata.queryId != null && !queryResponseMetadata.queryId.isEmpty()) { + cf.completeExceptionally( + new KsqlRestClientException("Expecting pull query but was push query")); + } + + columnNames = queryResponseMetadata.columnNames; + columnTypes = queryResponseMetadata.columnTypes; + } + + @Override + protected void handleRow(final Buffer buff) { + final JsonArray values = new JsonArray(buff); + rows.add(new RowImpl(columnNames, columnTypes, values)); + } + + @Override + protected void handleBodyEnd() { + if (!hasReadArguments) { + throw new IllegalStateException("Body ended before metadata received"); + } + + cf.complete(rows); + } +} diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java new file mode 100644 index 000000000000..a69bc17f0b7a --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.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.api.client.impl; + +import io.confluent.ksql.api.client.QueryResult; +import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; +import io.confluent.ksql.rest.client.KsqlRestClientException; +import io.vertx.core.Context; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.json.JsonArray; +import io.vertx.core.parsetools.RecordParser; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +public class PushQueryResponseHandler extends QueryResponseHandler { + + private QueryResultImpl queryResult; + private boolean paused; + + PushQueryResponseHandler(final Context context, final RecordParser recordParser, + final CompletableFuture cf) { + super(context, recordParser, cf); + } + + @Override + protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) { + if (queryResponseMetadata.queryId == null || queryResponseMetadata.queryId.isEmpty()) { + cf.completeExceptionally( + new KsqlRestClientException("Expecting push query but was pull query")); + return; + } + + this.queryResult = new QueryResultImpl(context, queryResponseMetadata.queryId, + Collections.unmodifiableList(queryResponseMetadata.columnNames), + Collections.unmodifiableList(queryResponseMetadata.columnTypes)); + cf.complete(queryResult); + } + + @Override + protected void handleRow(final Buffer buff) { + if (queryResult == null) { + throw new IllegalStateException("handleRow called before metadata processed"); + } + + final JsonArray values = new JsonArray(buff); + final Row row = new RowImpl(queryResult.columnNames(), queryResult.columnTypes(), values); + final boolean full = queryResult.accept(row); + if (full && !paused) { + recordParser.pause(); + queryResult.drainHandler(this::publisherReceptive); + paused = true; + } + } + + @Override + protected void handleBodyEnd() { + } + + private void publisherReceptive() { + paused = false; + recordParser.resume(); + } +} diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java new file mode 100644 index 000000000000..766d387abe4b --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -0,0 +1,84 @@ +/* + * 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.api.client.impl; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.guava.GuavaModule; +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; +import io.confluent.ksql.util.VertxUtils; +import io.vertx.core.Context; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.json.jackson.DatabindCodec; +import io.vertx.core.parsetools.RecordParser; +import java.util.concurrent.CompletableFuture; + +abstract class QueryResponseHandler { + + protected final Context context; + protected final RecordParser recordParser; + protected final CompletableFuture cf; + protected boolean hasReadArguments; + + QueryResponseHandler(final Context context, final RecordParser recordParser, + final CompletableFuture cf) { + this.context = context; + this.recordParser = recordParser; + this.cf = cf; + } + + public void handleBodyBuffer(final Buffer buff) { + checkContext(); + if (!hasReadArguments) { + handleArgs(buff); + } else { + handleRow(buff); + } + } + + public void handleBodyEnd(final Void v) { + checkContext(); + handleBodyEnd(); + } + + protected abstract void handleBodyEnd(); + + protected abstract void handleMetadata(final QueryResponseMetadata queryResponseMetadata); + + protected abstract void handleRow(final Buffer buff); + + private void handleArgs(final Buffer buff) { + hasReadArguments = true; + + final QueryResponseMetadata queryResponseMetadata; + final ObjectMapper objectMapper = DatabindCodec.mapper(); + objectMapper.registerModule(new GuavaModule()); + objectMapper.registerSubtypes(ImmutableList.class); + try { + queryResponseMetadata = objectMapper + .readValue(buff.getBytes(), QueryResponseMetadata.class); + } catch (Exception e) { + cf.completeExceptionally(e); + return; + } + + handleMetadata(queryResponseMetadata); + } + + private void checkContext() { + VertxUtils.checkContext(context); + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java new file mode 100644 index 000000000000..65f9be9ca84f --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java @@ -0,0 +1,102 @@ +/* + * 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.api.client.impl; + +import io.confluent.ksql.api.client.QueryResult; +import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.reactive.BufferedPublisher; +import io.vertx.core.Context; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.reactivestreams.Subscriber; + +class QueryResultImpl extends BufferedPublisher implements QueryResult { + + private final String queryId; + private final List columnNames; + private final List columnTypes; + private final PollableSubscriber pollableSubscriber; + private boolean polling; + private boolean subscribing; + + QueryResultImpl(final Context context, final String queryId, final List columnNames, + final List columnTypes) { + super(context); + this.queryId = queryId; + this.columnNames = columnNames; + this.columnTypes = columnTypes; + this.pollableSubscriber = new PollableSubscriber(ctx); + } + + @Override + public List columnNames() { + return columnNames; + } + + @Override + public List columnTypes() { + return columnTypes; + } + + @Override + public String queryID() { + return queryId; + } + + @Override + public synchronized void subscribe(final Subscriber subscriber) { + if (polling) { + throw new IllegalStateException("Cannot set subscriber if polling"); + } + subscribing = true; + super.subscribe(subscriber); + } + + @Override + public Row poll() { + return poll(0, TimeUnit.MILLISECONDS); + } + + @Override + public synchronized Row poll(final long timeout, final TimeUnit timeUnit) { + if (subscribing) { + throw new IllegalStateException("Cannot poll if subscriber has been set"); + } + if (!polling) { + subscribe(pollableSubscriber); + subscribing = false; + polling = true; + } + try { + return pollableSubscriber.poll(timeout, timeUnit); + } catch (final Throwable t) { + sendError(new Exception("Failed to poll", t)); + return null; + } + } + + @Override + public boolean isComplete() { + return false; + } + + @Override + public void close() { + pollableSubscriber.close(); + } + + +} \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java new file mode 100644 index 000000000000..c35f252897c6 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java @@ -0,0 +1,149 @@ +/* + * 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.api.client.impl; + +import io.confluent.ksql.api.client.Row; +import io.vertx.core.json.JsonArray; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class RowImpl implements Row { + + private final List columnNames; + private final List columnTypes; + private final List values; + private final Map columnNameToIndex; + + @SuppressWarnings("unchecked") + public RowImpl( + final List columnNames, + final List columnTypes, + final JsonArray values) { + this.columnNames = columnNames; + this.columnTypes = columnTypes; + this.values = values.getList(); + + this.columnNameToIndex = valueToIndexMap(columnNames); + } + + @Override + public List columnNames() { + return columnNames; + } + + @Override + public List columnTypes() { + return columnTypes; + } + + @Override + public List values() { + return values; + } + + @Override + public Object getObject(final int columnIndex) { + return values.get(columnIndex - 1); + } + + @Override + public Object getObject(final String columnName) { + return getObject(indexFromName(columnName)); + } + + @Override + public String getString(final int columnIndex) { + return (String)getObject(columnIndex); + } + + @Override + public String getString(final String columnName) { + return getString(indexFromName(columnName)); + } + + @Override + public Integer getInt(final int columnIndex) { + Number number = (Number)getObject(columnIndex); + if (number == null) { + return null; + } else { + return number instanceof Integer ? (Integer)number : number.intValue(); + } + } + + @Override + public Integer getInt(final String columnName) { + return getInt(indexFromName(columnName)); + } + + @Override + public Long getLong(final int columnIndex) { + Number number = (Number)getObject(columnIndex); + if (number == null) { + return null; + } else { + return number instanceof Long ? (Long)number : number.longValue(); + } + } + + @Override + public Long getLong(final String columnName) { + return getLong(indexFromName(columnName)); + } + + @Override + public Double getDouble(final int columnIndex) { + Number number = (Number)getObject(columnIndex); + if (number == null) { + return null; + } else { + return number instanceof Double ? (Double)number : number.doubleValue(); + } + } + + @Override + public Double getDouble(final String columnName) { + return getDouble(indexFromName(columnName)); + } + + @Override + public Boolean getBoolean(final int columnIndex) { + return (Boolean)getObject(columnIndex); + } + + @Override + public Boolean getBoolean(final String columnName) { + return getBoolean(indexFromName(columnName)); + } + + private int indexFromName(final String columnName) { + final Integer index = columnNameToIndex.get(columnName); + if (index == null) { + throw new IllegalArgumentException("No column exists with name: " + columnName); + } + return index; + } + + // Given list of values, return map of value to index in list. Returned indices are 1-indexed. + private static Map valueToIndexMap(final List values) { + final Map valueToIndex = new HashMap<>(); + for (int i = 0; i < values.size(); i++) { + valueToIndex.put(values.get(i), i + 1); + } + return valueToIndex; + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java new file mode 100644 index 000000000000..e56165c1b063 --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -0,0 +1,308 @@ +/* + * 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.api.client; + +import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +import io.confluent.ksql.api.BaseApiTest; +import io.confluent.ksql.api.client.impl.ClientImpl; +import io.confluent.ksql.api.client.impl.ClientOptionsImpl; +import io.confluent.ksql.api.server.PushQueryId; +import io.confluent.ksql.parser.exception.ParseFailedException; +import io.vertx.core.Context; +import io.vertx.ext.web.client.WebClient; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.junit.Test; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; + +public class ClientTest extends BaseApiTest { + + @SuppressWarnings("unchecked") + protected static final List DEFAULT_COLUMN_NAMES = BaseApiTest.DEFAULT_COLUMN_NAMES.getList(); + @SuppressWarnings("unchecked") + protected static final List DEFAULT_COLUMN_TYPES = BaseApiTest.DEFAULT_COLUMN_TYPES.getList(); + protected static final Map DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES = + BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES.getMap(); + + protected Context context; + protected Client client; + + @Override + public void setUp() { + super.setUp(); + + context = vertx.getOrCreateContext(); + } + + @Override + protected WebClient createClient() { + // Use Java client for these tests, rather than a vanilla WebClient + this.client = createJavaClient(); + + return null; + } + + @Override + protected void stopClient() { + if (client != null) { + try { + client.close(); + } catch (Exception e) { + log.error("Failed to close client", e); + } + } + } + + @Test + public void shouldStreamQueryAsync() throws Exception { + // When + final CompletableFuture cf = client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES); + assertThatEventually(cf::isDone, equalTo(true)); + assertThat(cf.isCompletedExceptionally(), equalTo(false)); + final QueryResult queryResult = cf.get(); + + // Then + verifyPushQueryMetadata(queryResult); + + shouldDeliver(queryResult, DEFAULT_ROWS.size()); + } + + @Test + public void shouldStreamQuerySync() throws Exception { + // When + final CompletableFuture cf = client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES); + assertThatEventually(cf::isDone, equalTo(true)); + assertThat(cf.isCompletedExceptionally(), equalTo(false)); + final QueryResult queryResult = cf.get(); + + // Then + verifyPushQueryMetadata(queryResult); + + for (int i = 0; i < DEFAULT_ROWS.size(); i++) { + final Row row = queryResult.poll(); + assertThat(row.values(), equalTo(rowWithIndex(i).getList())); + assertThat(row.columnNames(), equalTo(DEFAULT_COLUMN_NAMES)); + assertThat(row.columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); + } + } + + @Test + public void shouldHandleErrorResponseFromStreamQuery() throws Exception { + // Given + ParseFailedException pfe = new ParseFailedException("invalid query blah"); + testEndpoints.setCreateQueryPublisherException(pfe); + + // When/Then + assertErrorWhen( + () -> client.streamQuery("bad query", DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES), + "Received 400 response from server", + "invalid query blah" + ); + } + + @Test + public void shouldFailStreamQueryOnPullQuery() throws Exception { + assertErrorWhen( + () -> client.streamQuery(DEFAULT_PULL_QUERY), + "Expecting push query but was pull query" + ); + } + + @Test + public void shouldExecutePullQuery() throws Exception { + // When + final CompletableFuture> cf = client.executeQuery(DEFAULT_PULL_QUERY); + assertThatEventually(cf::isDone, equalTo(true)); + assertThat(cf.isCompletedExceptionally(), equalTo(false)); + final List rows = cf.get(); + + // Then + assertThat(testEndpoints.getLastSql(), is(DEFAULT_PULL_QUERY)); + assertThat(testEndpoints.getLastProperties().getMap(), is(Collections.emptyMap())); + + assertThat(rows, hasSize(DEFAULT_ROWS.size())); + for (int i = 0; i < DEFAULT_ROWS.size(); i++) { + assertThat(rows.get(i).values(), equalTo(rowWithIndex(i).getList())); + assertThat(rows.get(i).columnNames(), equalTo(DEFAULT_COLUMN_NAMES)); + assertThat(rows.get(i).columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); + } + + assertThat(server.getQueryIDs(), hasSize(0)); + } + + @Test + public void shouldHandleErrorResponseFromExecuteQuery() throws Exception { + // Given + ParseFailedException pfe = new ParseFailedException("invalid query blah"); + testEndpoints.setCreateQueryPublisherException(pfe); + + // When/Then + assertErrorWhen( + () -> client.executeQuery("bad query"), + "Received 400 response from server", + "invalid query blah" + ); + } + + @Test + public void shouldFailExecuteQueryOnPushQuery() throws Exception { + assertErrorWhen( + () -> client.executeQuery(DEFAULT_PUSH_QUERY), + "Expecting pull query but was push query" + ); + } + + protected Client createJavaClient() { + return new ClientImpl(createJavaClientOptions(), vertx); + } + + protected ClientOptions createJavaClientOptions() { + return new ClientOptionsImpl() + .setHost("localhost") + .setPort(server.getListeners().get(0).getPort()) + .setUseTls(false); + } + + private void verifyPushQueryMetadata(final QueryResult queryResult) { + assertThat(testEndpoints.getLastSql(), is(DEFAULT_PUSH_QUERY)); + assertThat(testEndpoints.getLastProperties(), is(BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES)); + + assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); + assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); + + String queryId = queryResult.queryID(); + assertThat(queryId, is(notNullValue())); + assertThat(server.getQueryIDs(), hasSize(1)); + assertThat(server.getQueryIDs().contains(new PushQueryId(queryId)), is(true)); + } + + private void shouldDeliver(final Publisher publisher, final int numRows) { + TestSubscriber subscriber = new TestSubscriber() { + @Override + public synchronized void onSubscribe(final Subscription sub) { + super.onSubscribe(sub); + sub.request(numRows); + } + }; + publisher.subscribe(subscriber); + assertThatEventually(subscriber::getValues, hasSize(numRows)); + for (int i = 0; i < numRows; i++) { + assertThat(subscriber.getValues().get(i).values(), equalTo(rowWithIndex(i).getList())); + } + assertThat(subscriber.isCompleted(), equalTo(false)); + assertThat(subscriber.getError(), is(nullValue())); + } + + private void assertErrorWhen( + final Supplier> queryRequest, + final String... errorMessages + ) throws Exception { + // Given + CountDownLatch latch = new CountDownLatch(1); + + // When + final CompletableFuture cf = queryRequest.get() + .exceptionally(error -> { + + // Then + assertThat(error, notNullValue()); + for (final String msg : errorMessages) { + assertThat(error.getMessage(), containsString(msg)); + } + + latch.countDown(); + return null; + }); + awaitLatch(latch, cf); + } + + private static void awaitLatch(CountDownLatch latch, CompletableFuture cf) throws Exception { + // Log reason for any failures, else output of failed tests is uninformative + cf.exceptionally(failure -> { + System.out.println("Failure reason: " + failure.getMessage()); + return null; + }); + + awaitLatch(latch); + } + + private static void awaitLatch(CountDownLatch latch) throws Exception { + assertThat(latch.await(2000, TimeUnit.MILLISECONDS), is(true)); + } + + private static class TestSubscriber implements Subscriber { + + private Subscription sub; + private boolean completed; + private Throwable error; + private final List values = new ArrayList<>(); + + public TestSubscriber() { + } + + @Override + public synchronized void onSubscribe(final Subscription sub) { + this.sub = sub; + } + + @Override + public synchronized void onNext(final T value) { + values.add(value); + } + + @Override + public synchronized void onError(final Throwable t) { + this.error = t; + } + + @Override + public synchronized void onComplete() { + this.completed = true; + } + + public boolean isCompleted() { + return completed; + } + + public Throwable getError() { + return error; + } + + public List getValues() { + return values; + } + + public Subscription getSub() { + return sub; + } + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/QueryResultImplTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/QueryResultImplTest.java new file mode 100644 index 000000000000..683111649eec --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/QueryResultImplTest.java @@ -0,0 +1,63 @@ +/* + * 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.api.client.impl; + +import static org.junit.Assert.assertThrows; + +import io.confluent.ksql.api.client.Row; +import io.vertx.core.Context; +import java.util.Collections; +import java.util.concurrent.TimeUnit; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; +import org.reactivestreams.Subscriber; + +@RunWith(MockitoJUnitRunner.class) +public class QueryResultImplTest { + + @Mock + private Context context; + @Mock + private Subscriber subscriber; + + private QueryResultImpl queryResult; + + @Before + public void setUp() { + queryResult = new QueryResultImpl(context, "queryId", Collections.emptyList(), Collections.emptyList()); + } + + @Test + public void shouldNotSubscribeIfPolling() { + // Given + queryResult.poll(1, TimeUnit.NANOSECONDS); + + // When / Then + assertThrows(IllegalStateException.class, () -> queryResult.subscribe(subscriber)); + } + + @Test + public void shouldNotPollIfSubscribed() { + // Given + queryResult.subscribe(subscriber); + + // When / Then + assertThrows(IllegalStateException.class, () -> queryResult.poll()); + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java new file mode 100644 index 000000000000..e03cafcf306b --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java @@ -0,0 +1,74 @@ +/* + * 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.api.client.impl; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableList; +import io.vertx.core.json.JsonArray; +import java.util.List; +import org.junit.Before; +import org.junit.Test; + +public class RowImplTest { + + private static final List COLUMN_NAMES = ImmutableList.of("f_str", "f_int", "f_long", "f_double", "f_bool"); + private static final List COLUMN_TYPES = ImmutableList.of("STRING", "INTEGER", "BIGINT", "DOUBLE", "BOOLEAN"); + private static final JsonArray VALUES = new JsonArray(ImmutableList.of("foo", 2, 1234L, 34.43, false)); + + private RowImpl row; + + @Before + public void setUp() { + row = new RowImpl(COLUMN_NAMES, COLUMN_TYPES, VALUES); + } + + @Test + public void shouldOneIndexColumnNames() { + assertThat(row.getObject(1), is("foo")); + assertThat(row.getObject(2), is(2)); + assertThat(row.getObject(3), is(1234L)); + assertThat(row.getObject(4), is(34.43)); + assertThat(row.getObject(5), is(false)); + } + + @Test + public void shouldGetString() { + assertThat(row.getString("f_str"), is("foo")); + } + + @Test + public void shouldGetInt() { + assertThat(row.getInt("f_int"), is(2)); + } + + @Test + public void shouldGetLong() { + assertThat(row.getLong("f_long"), is(1234L)); + assertThat(row.getLong("f_int"), is(2L)); + } + + @Test + public void shouldGetDouble() { + assertThat(row.getDouble("f_double"), is(34.43)); + } + + @Test + public void shouldGetBoolean() { + assertThat(row.getBoolean("f_bool"), is(false)); + } +} \ No newline at end of file diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/protocol/QueryResponseMetadata.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/protocol/QueryResponseMetadata.java index 504aec7b14cd..2c781e61d46b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/protocol/QueryResponseMetadata.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/protocol/QueryResponseMetadata.java @@ -15,6 +15,7 @@ package io.confluent.ksql.api.server.protocol; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.Immutable; import java.util.List; @@ -30,8 +31,10 @@ public class QueryResponseMetadata extends SerializableObject { public final ImmutableList columnNames; public final ImmutableList columnTypes; - public QueryResponseMetadata(final String queryId, final List columnNames, - final List columnTypes) { + public QueryResponseMetadata( + final @JsonProperty(value = "queryId") String queryId, + final @JsonProperty(value = "columnNames") List columnNames, + final @JsonProperty(value = "columnTypes") List columnTypes) { this.queryId = queryId; this.columnNames = ImmutableList.copyOf(Objects.requireNonNull(columnNames)); this.columnTypes = ImmutableList.copyOf(Objects.requireNonNull(columnTypes)); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java index 124d36d6b283..989de3106869 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java @@ -213,6 +213,19 @@ protected static void validateErrorCommon(final int errorCode, final String mess assertThat(error.getString("message"), is(message)); } + protected static JsonArray rowWithIndex(final int index) { + return new JsonArray().add("foo" + index).add(index).add(index % 2 == 0); + } + + private static List generateRows() { + List rows = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + JsonArray row = rowWithIndex(i); + rows.add(row); + } + return rows; + } + @SuppressWarnings("unchecked") protected void setDefaultRowGenerator() { List rows = new ArrayList<>(); @@ -225,14 +238,4 @@ protected void setDefaultRowGenerator() { DEFAULT_COLUMN_TYPES.getList(), rows)); } - - private static List generateRows() { - List rows = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - JsonArray row = new JsonArray().add("foo" + i).add(i).add(i % 2 == 0); - rows.add(row); - } - return rows; - } - } diff --git a/pom.xml b/pom.xml index 17d46c0a8c35..258db143c581 100644 --- a/pom.xml +++ b/pom.xml @@ -64,6 +64,7 @@ ksqldb-rocksdb-config-setter ksqldb-docker ksqldb-api-reactive-streams-tck + ksqldb-api-client From 6a169a04042a9650daf9286706a7deff4bffba9d Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 14:42:48 -0700 Subject: [PATCH 02/19] chore: checkstyle --- .../java/io/confluent/ksql/api/client/impl/ClientImpl.java | 2 +- .../ksql/api/client/impl/QueryResponseHandler.java | 4 ++-- .../java/io/confluent/ksql/api/client/impl/RowImpl.java | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index feb9e3d82c5d..9e60c51c246f 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -104,7 +104,7 @@ public CompletableFuture insertInto( @Override public Publisher streamInserts( - String streamName, Publisher> insertsPublisher) { + final String streamName, final Publisher> insertsPublisher) { return null; // not yet implemented } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index 766d387abe4b..9f8b6b2b662d 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -56,9 +56,9 @@ public void handleBodyEnd(final Void v) { protected abstract void handleBodyEnd(); - protected abstract void handleMetadata(final QueryResponseMetadata queryResponseMetadata); + protected abstract void handleMetadata(QueryResponseMetadata queryResponseMetadata); - protected abstract void handleRow(final Buffer buff); + protected abstract void handleRow(Buffer buff); private void handleArgs(final Buffer buff) { hasReadArguments = true; diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java index c35f252897c6..5930a6be8772 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java @@ -77,7 +77,7 @@ public String getString(final String columnName) { @Override public Integer getInt(final int columnIndex) { - Number number = (Number)getObject(columnIndex); + final Number number = (Number)getObject(columnIndex); if (number == null) { return null; } else { @@ -92,7 +92,7 @@ public Integer getInt(final String columnName) { @Override public Long getLong(final int columnIndex) { - Number number = (Number)getObject(columnIndex); + final Number number = (Number)getObject(columnIndex); if (number == null) { return null; } else { @@ -107,7 +107,7 @@ public Long getLong(final String columnName) { @Override public Double getDouble(final int columnIndex) { - Number number = (Number)getObject(columnIndex); + final Number number = (Number)getObject(columnIndex); if (number == null) { return null; } else { From c2c2c55911ad6b77697c31d9da6c0726511b8058 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 15:06:28 -0700 Subject: [PATCH 03/19] chore: allow push and pull queries via both streaming and exec --- .../io/confluent/ksql/api/client/Client.java | 10 +- .../ksql/api/client/QueryResult.java | 6 +- .../ksql/api/client/impl/ClientImpl.java | 4 +- ....java => ExecuteQueryResponseHandler.java} | 10 +- ...r.java => StreamQueryResponseHandler.java} | 11 +- .../confluent/ksql/api/client/ClientTest.java | 129 ++++++++++++------ .../io/confluent/ksql/api/TestEndpoints.java | 19 ++- .../ksql/api/TestQueryPublisher.java | 7 +- 8 files changed, 124 insertions(+), 72 deletions(-) rename ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/{PullQueryResponseHandler.java => ExecuteQueryResponseHandler.java} (80%) rename ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/{PushQueryResponseHandler.java => StreamQueryResponseHandler.java} (82%) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java index bd830ea0a4c6..a841de9ccad3 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java @@ -24,7 +24,7 @@ public interface Client { /** - * Execute a push query. + * Execute a query (push or pull) and receive the results one row at a time. * * @param sql statement of query to execute. * @return query result. @@ -32,7 +32,7 @@ public interface Client { CompletableFuture streamQuery(String sql); /** - * Execute a push query. + * Execute a query (push or pull) and receive the results one row at a time. * * @param sql statement of query to execute. * @param properties query properties. @@ -41,7 +41,8 @@ public interface Client { CompletableFuture streamQuery(String sql, Map properties); /** - * Execute a pull query. + * Execute a query (push or pull) and receive all result rows together, once the query has + * completed. * * @param sql statement of query to execute. * @return query result. @@ -49,7 +50,8 @@ public interface Client { CompletableFuture> executeQuery(String sql); /** - * Execute a pull query. + * Execute a query (push or pull) and receive all result rows together, once the query has + * completed. * * @param sql statement of query to execute. * @param properties query properties. diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java index 849e798934ba..6c231ce004cc 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/QueryResult.java @@ -20,9 +20,9 @@ import org.reactivestreams.Publisher; /** - * The result of a streaming (push) query. Records may be consumed by either subscribing to the - * publisher or polling (blocking) for one record at a time. These two methods of consumption are - * mutually exclusive; only one method may be used (per QueryResult). + * The result of a query (push or pull), streamed one row at time. Records may be consumed by either + * subscribing to the publisher or polling (blocking) for one record at a time. These two methods of + * consumption are mutually exclusive; only one method may be used (per QueryResult). */ public interface QueryResult extends Publisher { diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index 9e60c51c246f..8064b23b15f2 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -80,7 +80,7 @@ public CompletableFuture streamQuery( final String sql, final Map properties ) { - return makeQueryRequest(sql, properties, PushQueryResponseHandler::new); + return makeQueryRequest(sql, properties, StreamQueryResponseHandler::new); } @Override @@ -93,7 +93,7 @@ public CompletableFuture> executeQuery( final String sql, final Map properties ) { - return makeQueryRequest(sql, properties, PullQueryResponseHandler::new); + return makeQueryRequest(sql, properties, ExecuteQueryResponseHandler::new); } @Override diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java similarity index 80% rename from ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java rename to ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java index 50fb62e72f58..a6fbae62b1a3 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PullQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java @@ -17,7 +17,6 @@ import io.confluent.ksql.api.client.Row; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; -import io.confluent.ksql.rest.client.KsqlRestClientException; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; import io.vertx.core.json.JsonArray; @@ -26,13 +25,13 @@ import java.util.List; import java.util.concurrent.CompletableFuture; -public class PullQueryResponseHandler extends QueryResponseHandler> { +public class ExecuteQueryResponseHandler extends QueryResponseHandler> { private final List rows; private List columnNames; private List columnTypes; - PullQueryResponseHandler(final Context context, final RecordParser recordParser, + ExecuteQueryResponseHandler(final Context context, final RecordParser recordParser, final CompletableFuture> cf) { super(context, recordParser, cf); this.rows = new ArrayList<>(); @@ -40,11 +39,6 @@ public class PullQueryResponseHandler extends QueryResponseHandler> { @Override protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) { - if (queryResponseMetadata.queryId != null && !queryResponseMetadata.queryId.isEmpty()) { - cf.completeExceptionally( - new KsqlRestClientException("Expecting pull query but was push query")); - } - columnNames = queryResponseMetadata.columnNames; columnTypes = queryResponseMetadata.columnTypes; } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java similarity index 82% rename from ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java rename to ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java index a69bc17f0b7a..28f2e2c7e7b3 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PushQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java @@ -18,7 +18,6 @@ import io.confluent.ksql.api.client.QueryResult; import io.confluent.ksql.api.client.Row; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; -import io.confluent.ksql.rest.client.KsqlRestClientException; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; import io.vertx.core.json.JsonArray; @@ -26,24 +25,18 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; -public class PushQueryResponseHandler extends QueryResponseHandler { +public class StreamQueryResponseHandler extends QueryResponseHandler { private QueryResultImpl queryResult; private boolean paused; - PushQueryResponseHandler(final Context context, final RecordParser recordParser, + StreamQueryResponseHandler(final Context context, final RecordParser recordParser, final CompletableFuture cf) { super(context, recordParser, cf); } @Override protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) { - if (queryResponseMetadata.queryId == null || queryResponseMetadata.queryId.isEmpty()) { - cf.completeExceptionally( - new KsqlRestClientException("Expecting push query but was pull query")); - return; - } - this.queryResult = new QueryResultImpl(context, queryResponseMetadata.queryId, Collections.unmodifiableList(queryResponseMetadata.columnNames), Collections.unmodifiableList(queryResponseMetadata.columnTypes)); diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index e56165c1b063..55ec7d98a0e5 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -52,6 +52,7 @@ public class ClientTest extends BaseApiTest { protected static final List DEFAULT_COLUMN_TYPES = BaseApiTest.DEFAULT_COLUMN_TYPES.getList(); protected static final Map DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES = BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES.getMap(); + protected static final String DEFAULT_PUSH_QUERY_WITH_LIMIT = "select * from foo emit changes limit 10;"; protected Context context; protected Client client; @@ -83,29 +84,68 @@ protected void stopClient() { } @Test - public void shouldStreamQueryAsync() throws Exception { + public void shouldStreamPushQueryAsync() throws Exception { // When - final CompletableFuture cf = client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES); - assertThatEventually(cf::isDone, equalTo(true)); - assertThat(cf.isCompletedExceptionally(), equalTo(false)); - final QueryResult queryResult = cf.get(); + final QueryResult queryResult = + client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); // Then - verifyPushQueryMetadata(queryResult); + assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); + assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); + + shouldDeliver(queryResult, DEFAULT_ROWS.size()); + + String queryId = queryResult.queryID(); + assertThat(queryId, is(notNullValue())); + verifyPushQueryServerState(DEFAULT_PUSH_QUERY, queryId); + } + + @Test + public void shouldStreamPushQuerySync() throws Exception { + // When + final QueryResult queryResult = + client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); + + // Then + assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); + assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); + + for (int i = 0; i < DEFAULT_ROWS.size(); i++) { + final Row row = queryResult.poll(); + assertThat(row.values(), equalTo(rowWithIndex(i).getList())); + assertThat(row.columnNames(), equalTo(DEFAULT_COLUMN_NAMES)); + assertThat(row.columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); + } + + String queryId = queryResult.queryID(); + assertThat(queryId, is(notNullValue())); + verifyPushQueryServerState(DEFAULT_PUSH_QUERY, queryId); + } + + @Test + public void shouldStreamPullQueryAsync() throws Exception { + // When + final QueryResult queryResult = + client.streamQuery(DEFAULT_PULL_QUERY).get(); + + // Then + assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); + assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); shouldDeliver(queryResult, DEFAULT_ROWS.size()); + + verifyPullQueryServerState(); } @Test - public void shouldStreamQuerySync() throws Exception { + public void shouldStreamPullQuerySync() throws Exception { // When - final CompletableFuture cf = client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES); - assertThatEventually(cf::isDone, equalTo(true)); - assertThat(cf.isCompletedExceptionally(), equalTo(false)); - final QueryResult queryResult = cf.get(); + final QueryResult queryResult = + client.streamQuery(DEFAULT_PULL_QUERY).get(); // Then - verifyPushQueryMetadata(queryResult); + assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); + assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); for (int i = 0; i < DEFAULT_ROWS.size(); i++) { final Row row = queryResult.poll(); @@ -113,6 +153,8 @@ public void shouldStreamQuerySync() throws Exception { assertThat(row.columnNames(), equalTo(DEFAULT_COLUMN_NAMES)); assertThat(row.columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); } + + verifyPullQueryServerState(); } @Test @@ -130,25 +172,28 @@ public void shouldHandleErrorResponseFromStreamQuery() throws Exception { } @Test - public void shouldFailStreamQueryOnPullQuery() throws Exception { - assertErrorWhen( - () -> client.streamQuery(DEFAULT_PULL_QUERY), - "Expecting push query but was pull query" - ); + public void shouldExecutePullQuery() throws Exception { + // When + final List rows = client.executeQuery(DEFAULT_PULL_QUERY).get(); + + // Then + assertThat(rows, hasSize(DEFAULT_ROWS.size())); + for (int i = 0; i < DEFAULT_ROWS.size(); i++) { + assertThat(rows.get(i).values(), equalTo(rowWithIndex(i).getList())); + assertThat(rows.get(i).columnNames(), equalTo(DEFAULT_COLUMN_NAMES)); + assertThat(rows.get(i).columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); + } + + verifyPullQueryServerState(); } @Test - public void shouldExecutePullQuery() throws Exception { + public void shouldExecutePushQuery() throws Exception { // When - final CompletableFuture> cf = client.executeQuery(DEFAULT_PULL_QUERY); - assertThatEventually(cf::isDone, equalTo(true)); - assertThat(cf.isCompletedExceptionally(), equalTo(false)); - final List rows = cf.get(); + final List rows = + client.executeQuery(DEFAULT_PUSH_QUERY_WITH_LIMIT, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); // Then - assertThat(testEndpoints.getLastSql(), is(DEFAULT_PULL_QUERY)); - assertThat(testEndpoints.getLastProperties().getMap(), is(Collections.emptyMap())); - assertThat(rows, hasSize(DEFAULT_ROWS.size())); for (int i = 0; i < DEFAULT_ROWS.size(); i++) { assertThat(rows.get(i).values(), equalTo(rowWithIndex(i).getList())); @@ -156,7 +201,7 @@ public void shouldExecutePullQuery() throws Exception { assertThat(rows.get(i).columnTypes(), equalTo(DEFAULT_COLUMN_TYPES)); } - assertThat(server.getQueryIDs(), hasSize(0)); + verifyPushQueryServerState(DEFAULT_PUSH_QUERY_WITH_LIMIT); } @Test @@ -173,14 +218,6 @@ public void shouldHandleErrorResponseFromExecuteQuery() throws Exception { ); } - @Test - public void shouldFailExecuteQueryOnPushQuery() throws Exception { - assertErrorWhen( - () -> client.executeQuery(DEFAULT_PUSH_QUERY), - "Expecting pull query but was push query" - ); - } - protected Client createJavaClient() { return new ClientImpl(createJavaClientOptions(), vertx); } @@ -192,17 +229,25 @@ protected ClientOptions createJavaClientOptions() { .setUseTls(false); } - private void verifyPushQueryMetadata(final QueryResult queryResult) { - assertThat(testEndpoints.getLastSql(), is(DEFAULT_PUSH_QUERY)); + private void verifyPushQueryServerState(final String sql) { + verifyPushQueryServerState(sql, null); + } + + private void verifyPushQueryServerState(final String sql, final String queryId) { + assertThat(testEndpoints.getLastSql(), is(sql)); assertThat(testEndpoints.getLastProperties(), is(BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES)); - assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); - assertThat(queryResult.columnTypes(), is(DEFAULT_COLUMN_TYPES)); + if (queryId != null) { + assertThat(server.getQueryIDs(), hasSize(1)); + assertThat(server.getQueryIDs().contains(new PushQueryId(queryId)), is(true)); + } + } - String queryId = queryResult.queryID(); - assertThat(queryId, is(notNullValue())); - assertThat(server.getQueryIDs(), hasSize(1)); - assertThat(server.getQueryIDs().contains(new PushQueryId(queryId)), is(true)); + private void verifyPullQueryServerState() { + assertThat(testEndpoints.getLastSql(), is(DEFAULT_PULL_QUERY)); + assertThat(testEndpoints.getLastProperties().getMap(), is(Collections.emptyMap())); + + assertThat(server.getQueryIDs(), hasSize(0)); } private void shouldDeliver(final Publisher publisher, final int numRows) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java index 0a2bd6adf97c..7bf263a2d6f1 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java @@ -66,11 +66,13 @@ public synchronized CompletableFuture createQueryPublisher(final this.lastSql = sql; this.lastProperties = properties; this.lastApiSecurityContext = apiSecurityContext; - boolean push = sql.toLowerCase().contains("emit changes"); - TestQueryPublisher queryPublisher = new TestQueryPublisher(context, + final boolean push = sql.toLowerCase().contains("emit changes"); + final int limit = extractLimit(sql); + final TestQueryPublisher queryPublisher = new TestQueryPublisher(context, rowGeneratorFactory.get(), rowsBeforePublisherError, - push); + push, + limit); queryPublishers.add(queryPublisher); completableFuture.complete(queryPublisher); } @@ -223,5 +225,16 @@ public synchronized void setRowsBeforePublisherError(final int rowsBeforePublish public synchronized void setCreateQueryPublisherException(final RuntimeException exception) { this.createQueryPublisherException = exception; } + + private static int extractLimit(final String sql) { + final int ind = sql.toLowerCase().indexOf("limit"); + if (ind == -1) { + return -1; + } + + // extract the string between "limit" and the following semicolon + final String limit = sql.substring(ind + 5, ind + sql.substring(ind).indexOf(";")).trim(); + return Integer.parseInt(limit); + } } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java index ad224d7454d0..e257e710de2c 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestQueryPublisher.java @@ -27,14 +27,16 @@ public class TestQueryPublisher extends BasePublisher implements Que private final RowGenerator rowGenerator; private final int rowsBeforePublisherError; private final boolean push; + private final int limit; private int rowsSent; public TestQueryPublisher(final Context ctx, final RowGenerator rowGenerator, - final int rowsBeforePublisherError, final boolean push) { + final int rowsBeforePublisherError, final boolean push, final int limit) { super(ctx); this.rowGenerator = rowGenerator; this.rowsBeforePublisherError = rowsBeforePublisherError; this.push = push; + this.limit = limit; } synchronized boolean hasSubscriber() { @@ -64,6 +66,9 @@ private void doSend(long amount) { } else { rowsSent++; getSubscriber().onNext(row); + if (rowsSent == limit) { + sendComplete(); + } } } } From ec989636d680b75ef8e9f22016781a1c5acb714a Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 16:12:01 -0700 Subject: [PATCH 04/19] chore: feedback --- .../io/confluent/ksql/api/client/Client.java | 5 +++ .../ksql/api/client/impl/ClientImpl.java | 6 +--- .../impl/ExecuteQueryResponseHandler.java | 6 +++- .../api/client/impl/QueryResponseHandler.java | 8 ++--- .../ksql/api/client/impl/RowImpl.java | 23 ++++-------- .../impl/StreamQueryResponseHandler.java | 13 ++++++- .../ksql/api/client/util/RowUtil.java | 35 +++++++++++++++++++ .../confluent/ksql/api/client/ClientTest.java | 2 +- .../ksql/api/client/impl/RowImplTest.java | 5 ++- 9 files changed, 74 insertions(+), 29 deletions(-) create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java index a841de9ccad3..5cacefd7ca47 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java @@ -16,6 +16,7 @@ package io.confluent.ksql.api.client; import io.confluent.ksql.api.client.impl.ClientImpl; +import io.vertx.core.Vertx; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -68,4 +69,8 @@ public interface Client { static Client create(ClientOptions clientOptions) { return new ClientImpl(clientOptions); } + + static Client create(ClientOptions clientOptions, Vertx vertx) { + return new ClientImpl(clientOptions, vertx); + } } \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index 8064b23b15f2..cc365e7a4a5b 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -135,7 +135,7 @@ private CompletableFuture makeQueryRequest( serverSocketAddress, clientOptions.getPort(), clientOptions.getHost(), "/query-stream", response -> handleResponse(response, cf, responseHandlerSupplier)) - .exceptionHandler(e -> handleRequestException(e, cf)); + .exceptionHandler(cf::completeExceptionally); if (clientOptions.isUseBasicAuth()) { request = configureBasicAuth(request); } @@ -153,10 +153,6 @@ private HttpClientRequest configureBasicAuth(final HttpClientRequest request) { return request.putHeader(AUTHORIZATION.toString(), "Basic " + base64creds); } - private static void handleRequestException(final Throwable t, final CompletableFuture cf) { - cf.completeExceptionally(t); - } - private static void handleResponse( final HttpClientResponse response, final CompletableFuture cf, diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java index a6fbae62b1a3..ba79e7d9ac61 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java @@ -16,6 +16,7 @@ package io.confluent.ksql.api.client.impl; import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.api.client.util.RowUtil; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; @@ -23,6 +24,7 @@ import io.vertx.core.parsetools.RecordParser; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; public class ExecuteQueryResponseHandler extends QueryResponseHandler> { @@ -30,6 +32,7 @@ public class ExecuteQueryResponseHandler extends QueryResponseHandler> private final List rows; private List columnNames; private List columnTypes; + private Map columnNameToIndex; ExecuteQueryResponseHandler(final Context context, final RecordParser recordParser, final CompletableFuture> cf) { @@ -41,12 +44,13 @@ public class ExecuteQueryResponseHandler extends QueryResponseHandler> protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) { columnNames = queryResponseMetadata.columnNames; columnTypes = queryResponseMetadata.columnTypes; + columnNameToIndex = RowUtil.valueToIndexMap(columnNames); } @Override protected void handleRow(final Buffer buff) { final JsonArray values = new JsonArray(buff); - rows.add(new RowImpl(columnNames, columnTypes, values)); + rows.add(new RowImpl(columnNames, columnTypes, values, columnNameToIndex)); } @Override diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index 9f8b6b2b662d..cfeabc77e103 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -60,6 +60,10 @@ public void handleBodyEnd(final Void v) { protected abstract void handleRow(Buffer buff); + protected void checkContext() { + VertxUtils.checkContext(context); + } + private void handleArgs(final Buffer buff) { hasReadArguments = true; @@ -77,8 +81,4 @@ private void handleArgs(final Buffer buff) { handleMetadata(queryResponseMetadata); } - - private void checkContext() { - VertxUtils.checkContext(context); - } } \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java index 5930a6be8772..81213cd94e5c 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/RowImpl.java @@ -17,9 +17,9 @@ import io.confluent.ksql.api.client.Row; import io.vertx.core.json.JsonArray; -import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; public class RowImpl implements Row { @@ -32,12 +32,12 @@ public class RowImpl implements Row { public RowImpl( final List columnNames, final List columnTypes, - final JsonArray values) { - this.columnNames = columnNames; - this.columnTypes = columnTypes; - this.values = values.getList(); - - this.columnNameToIndex = valueToIndexMap(columnNames); + final JsonArray values, + final Map columnNameToIndex) { + this.columnNames = Objects.requireNonNull(columnNames); + this.columnTypes = Objects.requireNonNull(columnTypes); + this.values = Objects.requireNonNull(values).getList(); + this.columnNameToIndex = Objects.requireNonNull(columnNameToIndex); } @Override @@ -137,13 +137,4 @@ private int indexFromName(final String columnName) { } return index; } - - // Given list of values, return map of value to index in list. Returned indices are 1-indexed. - private static Map valueToIndexMap(final List values) { - final Map valueToIndex = new HashMap<>(); - for (int i = 0; i < values.size(); i++) { - valueToIndex.put(values.get(i), i + 1); - } - return valueToIndex; - } } \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java index 28f2e2c7e7b3..b9c44dc55e9d 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java @@ -17,17 +17,20 @@ import io.confluent.ksql.api.client.QueryResult; import io.confluent.ksql.api.client.Row; +import io.confluent.ksql.api.client.util.RowUtil; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; import io.vertx.core.json.JsonArray; import io.vertx.core.parsetools.RecordParser; import java.util.Collections; +import java.util.Map; import java.util.concurrent.CompletableFuture; public class StreamQueryResponseHandler extends QueryResponseHandler { private QueryResultImpl queryResult; + private Map columnNameToIndex; private boolean paused; StreamQueryResponseHandler(final Context context, final RecordParser recordParser, @@ -40,6 +43,7 @@ protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) this.queryResult = new QueryResultImpl(context, queryResponseMetadata.queryId, Collections.unmodifiableList(queryResponseMetadata.columnNames), Collections.unmodifiableList(queryResponseMetadata.columnTypes)); + this.columnNameToIndex = RowUtil.valueToIndexMap(queryResponseMetadata.columnNames); cf.complete(queryResult); } @@ -50,7 +54,12 @@ protected void handleRow(final Buffer buff) { } final JsonArray values = new JsonArray(buff); - final Row row = new RowImpl(queryResult.columnNames(), queryResult.columnTypes(), values); + final Row row = new RowImpl( + queryResult.columnNames(), + queryResult.columnTypes(), + values, + columnNameToIndex + ); final boolean full = queryResult.accept(row); if (full && !paused) { recordParser.pause(); @@ -64,6 +73,8 @@ protected void handleBodyEnd() { } private void publisherReceptive() { + checkContext(); + paused = false; recordParser.resume(); } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java new file mode 100644 index 000000000000..ee963c7701bf --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java @@ -0,0 +1,35 @@ +/* + * 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.api.client.util; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public final class RowUtil { + + private RowUtil() { + } + + // Given list of values, return map of value to index in list. Returned indices are 1-indexed. + public static Map valueToIndexMap(final List values) { + final Map valueToIndex = new HashMap<>(); + for (int i = 0; i < values.size(); i++) { + valueToIndex.put(values.get(i), i + 1); + } + return valueToIndex; + } +} \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 55ec7d98a0e5..e2aee5c1335a 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -219,7 +219,7 @@ public void shouldHandleErrorResponseFromExecuteQuery() throws Exception { } protected Client createJavaClient() { - return new ClientImpl(createJavaClientOptions(), vertx); + return Client.create(createJavaClientOptions(), vertx); } protected ClientOptions createJavaClientOptions() { diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java index e03cafcf306b..171b8278ac61 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/RowImplTest.java @@ -19,8 +19,10 @@ import static org.hamcrest.Matchers.is; import com.google.common.collect.ImmutableList; +import io.confluent.ksql.api.client.util.RowUtil; import io.vertx.core.json.JsonArray; import java.util.List; +import java.util.Map; import org.junit.Before; import org.junit.Test; @@ -28,13 +30,14 @@ public class RowImplTest { private static final List COLUMN_NAMES = ImmutableList.of("f_str", "f_int", "f_long", "f_double", "f_bool"); private static final List COLUMN_TYPES = ImmutableList.of("STRING", "INTEGER", "BIGINT", "DOUBLE", "BOOLEAN"); + private static final Map COLUMN_NAME_TO_INDEX = RowUtil.valueToIndexMap(COLUMN_NAMES); private static final JsonArray VALUES = new JsonArray(ImmutableList.of("foo", 2, 1234L, 34.43, false)); private RowImpl row; @Before public void setUp() { - row = new RowImpl(COLUMN_NAMES, COLUMN_TYPES, VALUES); + row = new RowImpl(COLUMN_NAMES, COLUMN_TYPES, VALUES, COLUMN_NAME_TO_INDEX); } @Test From 5771d712958007e1dbbcc5ac613b4c74ea8c02f6 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 16:20:46 -0700 Subject: [PATCH 05/19] chore: allow nulls in ClientOptions --- .../ksql/api/client/ClientOptions.java | 2 - .../api/client/impl/ClientOptionsImpl.java | 61 ++++++++----------- 2 files changed, 25 insertions(+), 38 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java index 736099114e88..73e05c72b117 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java @@ -35,8 +35,6 @@ public interface ClientOptions { ClientOptions setBasicAuthCredentials(String username, String password); - ClientOptions unsetBasicAuthCredentials(); - String getHost(); int getPort(); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java index 3af60b1ce6e3..24c8ccfd230b 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java @@ -25,12 +25,12 @@ public class ClientOptionsImpl implements ClientOptions { private boolean useTls = false; private boolean useClientAuth = false; private boolean useBasicAuth = false; - private String trustStorePath = ""; - private String trustStorePassword = ""; - private String keyStorePath = ""; - private String keyStorePassword = ""; - private String basicAuthUsername = ""; - private String basicAuthPassword = ""; + private String trustStorePath; + private String trustStorePassword; + private String keyStorePath; + private String keyStorePassword; + private String basicAuthUsername; + private String basicAuthPassword; public ClientOptionsImpl() { } @@ -48,17 +48,17 @@ private ClientOptionsImpl( this.useTls = useTls; this.useClientAuth = useClientAuth; this.useBasicAuth = useBasicAuth; - this.trustStorePath = Objects.requireNonNull(trustStorePath); - this.trustStorePassword = Objects.requireNonNull(trustStorePassword); - this.keyStorePath = Objects.requireNonNull(keyStorePath); - this.keyStorePassword = Objects.requireNonNull(keyStorePassword); - this.basicAuthUsername = Objects.requireNonNull(basicAuthUsername); - this.basicAuthPassword = Objects.requireNonNull(basicAuthPassword); + this.trustStorePath = trustStorePath; + this.trustStorePassword = trustStorePassword; + this.keyStorePath = keyStorePath; + this.keyStorePassword = keyStorePassword; + this.basicAuthUsername = basicAuthUsername; + this.basicAuthPassword = basicAuthPassword; } @Override public ClientOptions setHost(final String host) { - this.host = Objects.requireNonNull(host); + this.host = host; return this; } @@ -82,50 +82,39 @@ public ClientOptions setUseClientAuth(final boolean useClientAuth) { @Override public ClientOptions setTrustStore(final String trustStorePath) { - this.trustStorePath = Objects.requireNonNull(trustStorePath); + this.trustStorePath = trustStorePath; return this; } @Override public ClientOptions setTrustStorePassword(final String trustStorePassword) { - this.trustStorePassword = Objects.requireNonNull(trustStorePassword); + this.trustStorePassword = trustStorePassword; return this; } @Override public ClientOptions setKeyStore(final String keyStorePath) { - this.keyStorePath = Objects.requireNonNull(keyStorePath); + this.keyStorePath = keyStorePath; return this; } @Override public ClientOptions setKeyStorePassword(final String keyStorePassword) { - this.keyStorePassword = Objects.requireNonNull(keyStorePassword); + this.keyStorePassword = keyStorePassword; return this; } @Override public ClientOptions setBasicAuthCredentials(final String username, final String password) { - Objects.requireNonNull(username); - Objects.requireNonNull(password); - - this.useBasicAuth = true; + this.useBasicAuth = username == null && password == null; this.basicAuthUsername = username; this.basicAuthPassword = password; return this; } - @Override - public ClientOptions unsetBasicAuthCredentials() { - this.useBasicAuth = false; - this.basicAuthUsername = ""; - this.basicAuthPassword = ""; - return this; - } - @Override public String getHost() { - return host; + return host == null ? "" : host; } @Override @@ -150,32 +139,32 @@ public boolean isUseBasicAuth() { @Override public String getTrustStore() { - return trustStorePath; + return trustStorePath == null ? "" : trustStorePath; } @Override public String getTrustStorePassword() { - return trustStorePassword; + return trustStorePassword == null ? "" : trustStorePassword; } @Override public String getKeyStore() { - return keyStorePath; + return keyStorePath == null ? "" : keyStorePath; } @Override public String getKeyStorePassword() { - return keyStorePassword; + return keyStorePassword == null ? "" : keyStorePassword; } @Override public String getBasicAuthUsername() { - return basicAuthUsername; + return basicAuthUsername == null ? "" : basicAuthUsername; } @Override public String getBasicAuthPassword() { - return basicAuthPassword; + return basicAuthPassword == null ? "" : basicAuthPassword; } @Override From 02fbc871e20d8493ab574d96ead47348903084f1 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 23:25:44 -0700 Subject: [PATCH 06/19] chore: static json mapper --- .../api/client/impl/QueryResponseHandler.java | 10 +---- .../ksql/api/client/util/JsonMapper.java | 38 +++++++++++++++++++ 2 files changed, 40 insertions(+), 8 deletions(-) create mode 100644 ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/JsonMapper.java diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index cfeabc77e103..4ed417f07ec5 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -15,14 +15,11 @@ package io.confluent.ksql.api.client.impl; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.datatype.guava.GuavaModule; -import com.google.common.collect.ImmutableList; +import io.confluent.ksql.api.client.util.JsonMapper; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; import io.confluent.ksql.util.VertxUtils; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; -import io.vertx.core.json.jackson.DatabindCodec; import io.vertx.core.parsetools.RecordParser; import java.util.concurrent.CompletableFuture; @@ -68,11 +65,8 @@ private void handleArgs(final Buffer buff) { hasReadArguments = true; final QueryResponseMetadata queryResponseMetadata; - final ObjectMapper objectMapper = DatabindCodec.mapper(); - objectMapper.registerModule(new GuavaModule()); - objectMapper.registerSubtypes(ImmutableList.class); try { - queryResponseMetadata = objectMapper + queryResponseMetadata = JsonMapper.get() .readValue(buff.getBytes(), QueryResponseMetadata.class); } catch (Exception e) { cf.completeExceptionally(e); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/JsonMapper.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/JsonMapper.java new file mode 100644 index 000000000000..b98621d7098a --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/JsonMapper.java @@ -0,0 +1,38 @@ +/* + * 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.api.client.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.guava.GuavaModule; +import com.google.common.collect.ImmutableList; +import io.vertx.core.json.jackson.DatabindCodec; + +public final class JsonMapper { + + private static final ObjectMapper MAPPER = DatabindCodec.mapper(); + + static { + MAPPER.registerModule(new GuavaModule()); + MAPPER.registerSubtypes(ImmutableList.class); + } + + private JsonMapper() { + } + + public static ObjectMapper get() { + return MAPPER; + } +} \ No newline at end of file From d4f7f413563fcf200c7d989dc3494b7d5943a326 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Tue, 28 Apr 2020 23:55:39 -0700 Subject: [PATCH 07/19] chore: fix synchronization in PollableSubscriber --- .../api/client/impl/PollableSubscriber.java | 22 +++++++++---------- .../ksql/api/client/impl/QueryResultImpl.java | 9 ++------ 2 files changed, 12 insertions(+), 19 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java index 41175c86f842..dd4c0e48431a 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/PollableSubscriber.java @@ -18,9 +18,11 @@ import io.confluent.ksql.api.client.Row; import io.confluent.ksql.reactive.BaseSubscriber; import io.vertx.core.Context; +import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import org.reactivestreams.Subscription; public class PollableSubscriber extends BaseSubscriber { @@ -30,12 +32,14 @@ public class PollableSubscriber extends BaseSubscriber { private static final long MAX_POLL_NANOS = TimeUnit.MILLISECONDS.toNanos(100); private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private final Consumer errorHandler; private int tokens; - private Throwable error; private volatile boolean closed; - public PollableSubscriber(final Context context) { + public PollableSubscriber(final Context context, final Consumer errorHandler) { super(context); + + this.errorHandler = Objects.requireNonNull(errorHandler); } @Override @@ -49,17 +53,16 @@ protected void handleValue(final Row row) { } @Override - protected synchronized void handleError(final Throwable t) { - System.out.println("pollable subscriber encountered error: " + t); - error = t; + protected void handleError(final Throwable t) { + errorHandler.accept(new Exception(t)); } @Override - protected synchronized void handleComplete() { + protected void handleComplete() { close(); } - public synchronized Row poll(final long timeout, final TimeUnit timeUnit) throws Throwable { + public synchronized Row poll(final long timeout, final TimeUnit timeUnit) { if (closed) { return null; } @@ -83,11 +86,6 @@ public synchronized Row poll(final long timeout, final TimeUnit timeUnit) throws checkRequestTokens(); return row; } - if (error != null) { - final Throwable error = this.error; - this.error = null; - throw error; - } } catch (InterruptedException e) { return null; } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java index 65f9be9ca84f..05e23dc872d0 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java @@ -38,7 +38,7 @@ class QueryResultImpl extends BufferedPublisher implements QueryResult { this.queryId = queryId; this.columnNames = columnNames; this.columnTypes = columnTypes; - this.pollableSubscriber = new PollableSubscriber(ctx); + this.pollableSubscriber = new PollableSubscriber(ctx, this::sendError); } @Override @@ -80,12 +80,7 @@ public synchronized Row poll(final long timeout, final TimeUnit timeUnit) { subscribing = false; polling = true; } - try { - return pollableSubscriber.poll(timeout, timeUnit); - } catch (final Throwable t) { - sendError(new Exception("Failed to poll", t)); - return null; - } + return pollableSubscriber.poll(timeout, timeUnit); } @Override From 21c59fdc456d423088d56fc425189a1deb26d1ac Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 29 Apr 2020 00:05:32 -0700 Subject: [PATCH 08/19] test: clean up negative tests in ClientTest --- .../confluent/ksql/api/client/ClientTest.java | 74 +++++-------------- 1 file changed, 20 insertions(+), 54 deletions(-) diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index e2aee5c1335a..713e825d70d5 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -23,9 +23,9 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThrows; import io.confluent.ksql.api.BaseApiTest; -import io.confluent.ksql.api.client.impl.ClientImpl; import io.confluent.ksql.api.client.impl.ClientOptionsImpl; import io.confluent.ksql.api.server.PushQueryId; import io.confluent.ksql.parser.exception.ParseFailedException; @@ -35,10 +35,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; +import java.util.concurrent.ExecutionException; import org.junit.Test; import org.reactivestreams.Publisher; import org.reactivestreams.Subscriber; @@ -158,17 +155,20 @@ public void shouldStreamPullQuerySync() throws Exception { } @Test - public void shouldHandleErrorResponseFromStreamQuery() throws Exception { + public void shouldHandleErrorResponseFromStreamQuery() { // Given ParseFailedException pfe = new ParseFailedException("invalid query blah"); testEndpoints.setCreateQueryPublisherException(pfe); - // When/Then - assertErrorWhen( - () -> client.streamQuery("bad query", DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES), - "Received 400 response from server", - "invalid query blah" + // When + final Exception e = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.streamQuery("bad query", DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get() ); + + // Then + assertThat(e.getCause().getMessage(), containsString("Received 400 response from server")); + assertThat(e.getCause().getMessage(), containsString("invalid query blah")); } @Test @@ -205,17 +205,20 @@ public void shouldExecutePushQuery() throws Exception { } @Test - public void shouldHandleErrorResponseFromExecuteQuery() throws Exception { + public void shouldHandleErrorResponseFromExecuteQuery() { // Given ParseFailedException pfe = new ParseFailedException("invalid query blah"); testEndpoints.setCreateQueryPublisherException(pfe); - // When/Then - assertErrorWhen( - () -> client.executeQuery("bad query"), - "Received 400 response from server", - "invalid query blah" + // When + final Exception e = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.executeQuery("bad query").get() ); + + // Then + assertThat(e.getCause().getMessage(), containsString("Received 400 response from server")); + assertThat(e.getCause().getMessage(), containsString("invalid query blah")); } protected Client createJavaClient() { @@ -267,43 +270,6 @@ public synchronized void onSubscribe(final Subscription sub) { assertThat(subscriber.getError(), is(nullValue())); } - private void assertErrorWhen( - final Supplier> queryRequest, - final String... errorMessages - ) throws Exception { - // Given - CountDownLatch latch = new CountDownLatch(1); - - // When - final CompletableFuture cf = queryRequest.get() - .exceptionally(error -> { - - // Then - assertThat(error, notNullValue()); - for (final String msg : errorMessages) { - assertThat(error.getMessage(), containsString(msg)); - } - - latch.countDown(); - return null; - }); - awaitLatch(latch, cf); - } - - private static void awaitLatch(CountDownLatch latch, CompletableFuture cf) throws Exception { - // Log reason for any failures, else output of failed tests is uninformative - cf.exceptionally(failure -> { - System.out.println("Failure reason: " + failure.getMessage()); - return null; - }); - - awaitLatch(latch); - } - - private static void awaitLatch(CountDownLatch latch) throws Exception { - assertThat(latch.await(2000, TimeUnit.MILLISECONDS), is(true)); - } - private static class TestSubscriber implements Subscriber { private Subscription sub; From 40e3582db859d97db5dd9d4a49a541bd4a1ce541 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 29 Apr 2020 00:10:42 -0700 Subject: [PATCH 09/19] test: more ClientTest cleanup --- .../confluent/ksql/api/client/ClientTest.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 713e825d70d5..569217bc2711 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -29,7 +29,6 @@ import io.confluent.ksql.api.client.impl.ClientOptionsImpl; import io.confluent.ksql.api.server.PushQueryId; import io.confluent.ksql.parser.exception.ParseFailedException; -import io.vertx.core.Context; import io.vertx.ext.web.client.WebClient; import java.util.ArrayList; import java.util.Collections; @@ -51,21 +50,19 @@ public class ClientTest extends BaseApiTest { BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES.getMap(); protected static final String DEFAULT_PUSH_QUERY_WITH_LIMIT = "select * from foo emit changes limit 10;"; - protected Context context; protected Client client; @Override public void setUp() { super.setUp(); - context = vertx.getOrCreateContext(); + // Use Java client for these tests, rather than WebClient as in BaseApiTest + this.client = createJavaClient(); } @Override protected WebClient createClient() { - // Use Java client for these tests, rather than a vanilla WebClient - this.client = createJavaClient(); - + // Ensure these tests use Java client rather than WebClient return null; } @@ -300,19 +297,19 @@ public synchronized void onComplete() { this.completed = true; } - public boolean isCompleted() { + public synchronized boolean isCompleted() { return completed; } - public Throwable getError() { + public synchronized Throwable getError() { return error; } - public List getValues() { + public synchronized List getValues() { return values; } - public Subscription getSub() { + public synchronized Subscription getSub() { return sub; } } From 13722fe463ba1d19069fd722fb11ba24e9078773 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 29 Apr 2020 00:40:50 -0700 Subject: [PATCH 10/19] chore: creator for ClientOptions --- .../java/io/confluent/ksql/api/client/ClientOptions.java | 6 ++++++ .../test/java/io/confluent/ksql/api/client/ClientTest.java | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java index 73e05c72b117..092c0b241ca3 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java @@ -15,6 +15,8 @@ package io.confluent.ksql.api.client; +import io.confluent.ksql.api.client.impl.ClientOptionsImpl; + public interface ClientOptions { ClientOptions setHost(String host); @@ -58,4 +60,8 @@ public interface ClientOptions { String getBasicAuthPassword(); ClientOptions copy(); + + static ClientOptions create() { + return new ClientOptionsImpl(); + } } \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 569217bc2711..23a363c06ad3 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -223,7 +223,7 @@ protected Client createJavaClient() { } protected ClientOptions createJavaClientOptions() { - return new ClientOptionsImpl() + return ClientOptions.create() .setHost("localhost") .setPort(server.getListeners().get(0).getPort()) .setUseTls(false); From 553ecf2846c2c938f8b0a3a6afdd4438afec6207 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Wed, 29 Apr 2020 00:58:11 -0700 Subject: [PATCH 11/19] chore: don't create basic auth header on each request --- .../ksql/api/client/impl/ClientImpl.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index cc365e7a4a5b..2ba828f79fdf 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -50,6 +50,7 @@ public class ClientImpl implements Client { private final Vertx vertx; private final HttpClient httpClient; private final SocketAddress serverSocketAddress; + private final String basicAuthHeader; private final boolean ownedVertx; public ClientImpl(final ClientOptions clientOptions) { @@ -66,6 +67,7 @@ private ClientImpl(final ClientOptions clientOptions, final Vertx vertx, this.vertx = vertx; this.ownedVertx = ownedVertx; this.httpClient = createHttpClient(vertx, clientOptions); + this.basicAuthHeader = createBasicAuthHeader(clientOptions); this.serverSocketAddress = io.vertx.core.net.SocketAddress .inetSocketAddress(clientOptions.getPort(), clientOptions.getHost()); } @@ -145,12 +147,7 @@ private CompletableFuture makeQueryRequest( } private HttpClientRequest configureBasicAuth(final HttpClientRequest request) { - final String creds = clientOptions.getBasicAuthUsername() - + ":" - + clientOptions.getBasicAuthPassword(); - final String base64creds = - Base64.getEncoder().encodeToString(creds.getBytes(Charset.defaultCharset())); - return request.putHeader(AUTHORIZATION.toString(), "Basic " + base64creds); + return request.putHeader(AUTHORIZATION.toString(), basicAuthHeader); } private static void handleResponse( @@ -200,4 +197,17 @@ private static HttpClient createHttpClient(final Vertx vertx, final ClientOption } return vertx.createHttpClient(options); } + + private static String createBasicAuthHeader(final ClientOptions clientOptions) { + if (!clientOptions.isUseBasicAuth()) { + return ""; + } + + final String creds = clientOptions.getBasicAuthUsername() + + ":" + + clientOptions.getBasicAuthPassword(); + final String base64creds = + Base64.getEncoder().encodeToString(creds.getBytes(Charset.defaultCharset())); + return "Basic " + base64creds; + } } \ No newline at end of file From c58782c7180fc30987d72cca62aa2195a82737aa Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Sun, 3 May 2020 12:48:14 -0700 Subject: [PATCH 12/19] fix: basic auth --- ksqldb-api-client/pom.xml | 8 +++ .../api/client/impl/ClientOptionsImpl.java | 4 +- .../ksql/api/client/ClientBasicAuthTest.java | 67 +++++++++++++++++++ .../confluent/ksql/api/client/ClientTest.java | 5 +- 4 files changed, 81 insertions(+), 3 deletions(-) create mode 100644 ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientBasicAuthTest.java diff --git a/ksqldb-api-client/pom.xml b/ksqldb-api-client/pom.xml index 5a616a00d3e4..8096a02b1a36 100644 --- a/ksqldb-api-client/pom.xml +++ b/ksqldb-api-client/pom.xml @@ -69,6 +69,14 @@ test + + + org.eclipse.jetty + jetty-jaas + ${jetty.version} + test + + diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java index 24c8ccfd230b..06e1e7a86866 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java @@ -106,7 +106,7 @@ public ClientOptions setKeyStorePassword(final String keyStorePassword) { @Override public ClientOptions setBasicAuthCredentials(final String username, final String password) { - this.useBasicAuth = username == null && password == null; + this.useBasicAuth = username != null || password != null; this.basicAuthUsername = username; this.basicAuthPassword = password; return this; @@ -134,7 +134,7 @@ public boolean isUseClientAuth() { @Override public boolean isUseBasicAuth() { - return useClientAuth; + return useBasicAuth; } @Override diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientBasicAuthTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientBasicAuthTest.java new file mode 100644 index 000000000000..6bcf3fb94fcb --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientBasicAuthTest.java @@ -0,0 +1,67 @@ +/* + * 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.api.client; + +import io.confluent.ksql.api.server.ApiServerConfig; +import io.confluent.ksql.test.util.TestBasicJaasConfig; +import java.util.Map; +import org.junit.ClassRule; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClientBasicAuthTest extends ClientTest { + + protected static final Logger log = LoggerFactory.getLogger(ClientBasicAuthTest.class); + + private static final String PROPS_JAAS_REALM = "KsqlServer-Props"; + private static final String KSQL_RESOURCE = "ksql-user"; + private static final String USER_WITH_ACCESS = "harry"; + private static final String USER_WITH_ACCESS_PWD = "changeme"; + + @ClassRule + public static final TestBasicJaasConfig JAAS_CONFIG = TestBasicJaasConfig + .builder(PROPS_JAAS_REALM) + .addUser(USER_WITH_ACCESS, USER_WITH_ACCESS_PWD, KSQL_RESOURCE) + .build(); + + @Override + protected ApiServerConfig createServerConfig() { + ApiServerConfig config = super.createServerConfig(); + Map origs = config.originals(); + origs.put( + ApiServerConfig.AUTHENTICATION_METHOD_CONFIG, + ApiServerConfig.AUTHENTICATION_METHOD_BASIC); + origs.put( + ApiServerConfig.AUTHENTICATION_REALM_CONFIG, + PROPS_JAAS_REALM + ); + origs.put( + ApiServerConfig.AUTHENTICATION_ROLES_CONFIG, + KSQL_RESOURCE + ); + return new ApiServerConfig(origs); + } + + @Override + protected ClientOptions createJavaClientOptions() { + return ClientOptions.create() + .setHost("localhost") + .setPort(server.getListeners().get(0).getPort()) + .setUseTls(false) + .setBasicAuthCredentials(USER_WITH_ACCESS, USER_WITH_ACCESS_PWD); + } + +} diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 23a363c06ad3..4dba6ad84da7 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertThrows; import io.confluent.ksql.api.BaseApiTest; -import io.confluent.ksql.api.client.impl.ClientOptionsImpl; import io.confluent.ksql.api.server.PushQueryId; import io.confluent.ksql.parser.exception.ParseFailedException; import io.vertx.ext.web.client.WebClient; @@ -39,9 +38,13 @@ import org.reactivestreams.Publisher; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ClientTest extends BaseApiTest { + protected static final Logger log = LoggerFactory.getLogger(ClientTest.class); + @SuppressWarnings("unchecked") protected static final List DEFAULT_COLUMN_NAMES = BaseApiTest.DEFAULT_COLUMN_NAMES.getList(); @SuppressWarnings("unchecked") From 1073dbbdbd9557cb04c488ffa89aac4eb542df6e Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Sun, 3 May 2020 23:23:18 -0700 Subject: [PATCH 13/19] chore: handle record parser exception --- .../io/confluent/ksql/api/client/impl/ClientImpl.java | 1 + .../api/client/impl/ExecuteQueryResponseHandler.java | 10 ++++++++++ .../ksql/api/client/impl/QueryResponseHandler.java | 11 +++++++++++ .../ksql/api/client/impl/QueryResultImpl.java | 4 ++++ .../api/client/impl/StreamQueryResponseHandler.java | 5 +++++ 5 files changed, 31 insertions(+) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index 2ba828f79fdf..84d645345ce0 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -161,6 +161,7 @@ private static void handleResponse( recordParser.handler(responseHandler::handleBodyBuffer); recordParser.endHandler(responseHandler::handleBodyEnd); + recordParser.exceptionHandler(responseHandler::handleException); } else { response.bodyHandler(buffer -> { final JsonObject errorResponse = buffer.toJsonObject(); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java index ba79e7d9ac61..b74ba45f7a88 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java @@ -26,9 +26,13 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ExecuteQueryResponseHandler extends QueryResponseHandler> { + private static final Logger log = LoggerFactory.getLogger(ExecuteQueryResponseHandler.class); + private final List rows; private List columnNames; private List columnTypes; @@ -61,4 +65,10 @@ protected void handleBodyEnd() { cf.complete(rows); } + + @Override + public void handleExceptionAfterFutureCompleted(final Throwable t) { + // This should not happen + log.error("Exceptions should not occur after the future has been completed", t); + } } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index 4ed417f07ec5..4acc7adaf853 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -51,12 +51,23 @@ public void handleBodyEnd(final Void v) { handleBodyEnd(); } + public void handleException(final Throwable t) { + checkContext(); + if (!cf.isDone()) { + cf.completeExceptionally(t); + } else { + handleExceptionAfterFutureCompleted(t); + } + } + protected abstract void handleBodyEnd(); protected abstract void handleMetadata(QueryResponseMetadata queryResponseMetadata); protected abstract void handleRow(Buffer buff); + protected abstract void handleExceptionAfterFutureCompleted(Throwable t); + protected void checkContext() { VertxUtils.checkContext(context); } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java index 05e23dc872d0..ba6471ef0c9e 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java @@ -88,6 +88,10 @@ public boolean isComplete() { return false; } + public void handleError(final Exception e) { + sendError(e); + } + @Override public void close() { pollableSubscriber.close(); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java index b9c44dc55e9d..4623f560ab36 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java @@ -72,6 +72,11 @@ protected void handleRow(final Buffer buff) { protected void handleBodyEnd() { } + @Override + public void handleExceptionAfterFutureCompleted(final Throwable t) { + queryResult.handleError(new Exception(t)); + } + private void publisherReceptive() { checkContext(); From 0f990c9346bc00f934c88fee56e7965121a8f791 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Mon, 4 May 2020 00:05:17 -0700 Subject: [PATCH 14/19] fix: synchronization in QueryResultImpl --- .../io/confluent/ksql/api/client/impl/ClientImpl.java | 4 ++-- .../ksql/api/client/impl/QueryResultImpl.java | 11 ++++++----- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index 84d645345ce0..8c70d11ffeb8 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -68,8 +68,8 @@ private ClientImpl(final ClientOptions clientOptions, final Vertx vertx, this.ownedVertx = ownedVertx; this.httpClient = createHttpClient(vertx, clientOptions); this.basicAuthHeader = createBasicAuthHeader(clientOptions); - this.serverSocketAddress = io.vertx.core.net.SocketAddress - .inetSocketAddress(clientOptions.getPort(), clientOptions.getHost()); + this.serverSocketAddress = + SocketAddress.inetSocketAddress(clientOptions.getPort(), clientOptions.getHost()); } @Override diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java index ba6471ef0c9e..81b6d9fbf5ac 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java @@ -29,7 +29,7 @@ class QueryResultImpl extends BufferedPublisher implements QueryResult { private final List columnNames; private final List columnTypes; private final PollableSubscriber pollableSubscriber; - private boolean polling; + private volatile boolean polling; private boolean subscribing; QueryResultImpl(final Context context, final String queryId, final List columnNames, @@ -57,12 +57,14 @@ public String queryID() { } @Override - public synchronized void subscribe(final Subscriber subscriber) { + public void subscribe(final Subscriber subscriber) { if (polling) { throw new IllegalStateException("Cannot set subscriber if polling"); } - subscribing = true; - super.subscribe(subscriber); + synchronized (this) { + subscribing = true; + super.subscribe(subscriber); + } } @Override @@ -97,5 +99,4 @@ public void close() { pollableSubscriber.close(); } - } \ No newline at end of file From 904476714df0a29fc17e6cf8ebf38f3e898105f8 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Mon, 4 May 2020 00:06:48 -0700 Subject: [PATCH 15/19] feat: tls tests --- .../ksql/api/client/ClientOptions.java | 8 +++ .../ksql/api/client/impl/ClientImpl.java | 2 + .../api/client/impl/ClientOptionsImpl.java | 34 +++++++++- .../ksql/api/client/ClientTlsTest.java | 63 +++++++++++++++++++ 4 files changed, 105 insertions(+), 2 deletions(-) create mode 100644 ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTlsTest.java diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java index 092c0b241ca3..a4d8b1023bb3 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java @@ -27,6 +27,10 @@ public interface ClientOptions { ClientOptions setUseClientAuth(boolean useClientAuth); + ClientOptions setVerifyHost(boolean verifyHost); + + ClientOptions setTrustAll(boolean trustAll); + ClientOptions setTrustStore(String trustStorePath); ClientOptions setTrustStorePassword(String trustStorePassword); @@ -45,6 +49,10 @@ public interface ClientOptions { boolean isUseClientAuth(); + boolean isVerifyHost(); + + boolean isTrustAll(); + boolean isUseBasicAuth(); String getTrustStore(); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index 8c70d11ffeb8..fbf9a43a4eae 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -180,6 +180,8 @@ private static HttpClient createHttpClient(final Vertx vertx, final ClientOption .setSsl(clientOptions.isUseTls()) .setUseAlpn(true) .setProtocolVersion(HttpVersion.HTTP_2) + .setVerifyHost(clientOptions.isVerifyHost()) + .setTrustAll(clientOptions.isTrustAll()) .setDefaultHost(clientOptions.getHost()) .setDefaultPort(clientOptions.getPort()); if (clientOptions.isUseTls() && !clientOptions.getTrustStore().isEmpty()) { diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java index 06e1e7a86866..8b10d397af11 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java @@ -24,6 +24,8 @@ public class ClientOptionsImpl implements ClientOptions { private int port = 8088; private boolean useTls = false; private boolean useClientAuth = false; + private boolean verifyHost = true; + private boolean trustAll = false; private boolean useBasicAuth = false; private String trustStorePath; private String trustStorePassword; @@ -39,7 +41,9 @@ public ClientOptionsImpl() { private ClientOptionsImpl( // CHECKSTYLE_RULES.ON: ParameterNumberCheck final String host, final int port, - final boolean useTls, final boolean useClientAuth, final boolean useBasicAuth, + final boolean useTls, final boolean useClientAuth, + final boolean verifyHost, final boolean trustAll, + final boolean useBasicAuth, final String trustStorePath, final String trustStorePassword, final String keyStorePath, final String keyStorePassword, final String basicAuthUsername, final String basicAuthPassword) { @@ -47,6 +51,8 @@ private ClientOptionsImpl( this.port = port; this.useTls = useTls; this.useClientAuth = useClientAuth; + this.verifyHost = verifyHost; + this.trustAll = trustAll; this.useBasicAuth = useBasicAuth; this.trustStorePath = trustStorePath; this.trustStorePassword = trustStorePassword; @@ -80,6 +86,18 @@ public ClientOptions setUseClientAuth(final boolean useClientAuth) { return this; } + @Override + public ClientOptions setVerifyHost(final boolean verifyHost) { + this.verifyHost = verifyHost; + return this; + } + + @Override + public ClientOptions setTrustAll(final boolean trustAll) { + this.trustAll = trustAll; + return this; + } + @Override public ClientOptions setTrustStore(final String trustStorePath) { this.trustStorePath = trustStorePath; @@ -132,6 +150,16 @@ public boolean isUseClientAuth() { return useClientAuth; } + @Override + public boolean isVerifyHost() { + return verifyHost; + } + + @Override + public boolean isTrustAll() { + return trustAll; + } + @Override public boolean isUseBasicAuth() { return useBasicAuth; @@ -171,7 +199,9 @@ public String getBasicAuthPassword() { public ClientOptions copy() { return new ClientOptionsImpl( host, port, - useTls, useClientAuth, useBasicAuth, + useTls, useClientAuth, + verifyHost, trustAll, + useBasicAuth, trustStorePath, trustStorePassword, keyStorePath, keyStorePassword, basicAuthUsername, basicAuthPassword); diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTlsTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTlsTest.java new file mode 100644 index 000000000000..bffeb06bfccf --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTlsTest.java @@ -0,0 +1,63 @@ +/* + * 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.api.client; + +import io.confluent.ksql.api.server.ApiServerConfig; +import io.confluent.ksql.test.util.secure.ServerKeyStore; +import java.util.HashMap; +import java.util.Map; +import org.apache.kafka.common.config.SslConfigs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClientTlsTest extends ClientTest { + + protected static final Logger log = LoggerFactory.getLogger(ClientTlsTest.class); + + @Override + protected ApiServerConfig createServerConfig() { + + String keyStorePath = ServerKeyStore.keyStoreProps() + .get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + String keyStorePassword = ServerKeyStore.keyStoreProps() + .get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + String trustStorePath = ServerKeyStore.keyStoreProps() + .get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); + String trustStorePassword = ServerKeyStore.keyStoreProps() + .get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG); + + Map config = new HashMap<>(); + config.put(ApiServerConfig.LISTENERS, "https://localhost:0"); + config.put(ApiServerConfig.TLS_KEY_STORE_PATH, keyStorePath); + config.put(ApiServerConfig.TLS_KEY_STORE_PASSWORD, keyStorePassword); + config.put(ApiServerConfig.TLS_TRUST_STORE_PATH, trustStorePath); + config.put(ApiServerConfig.TLS_TRUST_STORE_PASSWORD, trustStorePassword); + config.put(ApiServerConfig.VERTICLE_INSTANCES, 4); + + return new ApiServerConfig(config); + } + + @Override + protected ClientOptions createJavaClientOptions() { + return ClientOptions.create() + .setHost("localhost") + .setPort(server.getListeners().get(0).getPort()) + .setUseTls(true) + .setVerifyHost(false) + .setTrustAll(true); + } + +} From 8ad66e85a9aa2ba6f4b00e9f87625b7b164860e7 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Fri, 8 May 2020 16:47:04 -0700 Subject: [PATCH 16/19] chore: checkstyle --- .../ksql/api/client/impl/QueryResponseHandler.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index 4acc7adaf853..179f6c0dce3c 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -46,11 +46,6 @@ public void handleBodyBuffer(final Buffer buff) { } } - public void handleBodyEnd(final Void v) { - checkContext(); - handleBodyEnd(); - } - public void handleException(final Throwable t) { checkContext(); if (!cf.isDone()) { @@ -60,6 +55,11 @@ public void handleException(final Throwable t) { } } + public void handleBodyEnd(final Void v) { + checkContext(); + handleBodyEnd(); + } + protected abstract void handleBodyEnd(); protected abstract void handleMetadata(QueryResponseMetadata queryResponseMetadata); From dff2ffbf0b715d722df53f148e9cad7d9c17aca5 Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Fri, 8 May 2020 17:06:52 -0700 Subject: [PATCH 17/19] fix: implement isComplete() on QueryResult --- .../ksql/api/client/impl/QueryResultImpl.java | 2 +- .../ksql/reactive/BufferedPublisher.java | 18 ++++++++++++------ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java index 81b6d9fbf5ac..4bb6c7c5fd2e 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResultImpl.java @@ -87,7 +87,7 @@ public synchronized Row poll(final long timeout, final TimeUnit timeUnit) { @Override public boolean isComplete() { - return false; + return super.isComplete(); } public void handleError(final Exception e) { diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java b/ksqldb-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java index da283b93a444..afe089271f83 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/reactive/BufferedPublisher.java @@ -39,8 +39,8 @@ public class BufferedPublisher extends BasePublisher { private final Queue buffer = new ArrayDeque<>(); private final int bufferMaxSize; private Runnable drainHandler; + private boolean shouldSendComplete; private boolean complete; - private boolean completing; /** * Construct a BufferedPublisher @@ -63,6 +63,7 @@ public BufferedPublisher(final Context ctx, final Collection initialBuffer) { this(ctx); this.buffer.addAll(initialBuffer); complete = true; + shouldSendComplete = true; } /** @@ -89,7 +90,7 @@ public BufferedPublisher(final Context ctx, final int bufferMaxSize) { */ public boolean accept(final T t) { checkContext(); - if (completing) { + if (complete) { throw new IllegalStateException("Cannot call accept after complete is called"); } if (!isCancelled()) { @@ -124,17 +125,21 @@ public void drainHandler(final Runnable handler) { */ public void complete() { checkContext(); - if (isCancelled() || completing) { + if (isCancelled() || complete) { return; } - completing = true; + complete = true; if (buffer.isEmpty() && getSubscriber() != null) { sendComplete(); } else { - complete = true; + shouldSendComplete = true; } } + protected boolean isComplete() { + return complete; + } + protected void maybeSend() { int numSent = 0; while (!isCancelled() && getDemand() > 0 && !buffer.isEmpty()) { @@ -150,8 +155,9 @@ protected void maybeSend() { } if (buffer.isEmpty() && !isCancelled()) { - if (complete) { + if (shouldSendComplete) { sendComplete(); + shouldSendComplete = false; } else if (getDemand() > 0 && drainHandler != null) { final Runnable handler = drainHandler; ctx.runOnContext(v -> handler.run()); From 1ac08f6a52421c86b838d5d5488ef87f32240d7b Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Fri, 8 May 2020 17:28:58 -0700 Subject: [PATCH 18/19] chore: limit number of rows that may be returned from executeQuery() --- .../ksql/api/client/ClientOptions.java | 4 ++++ .../ksql/api/client/impl/ClientImpl.java | 7 ++++++- .../api/client/impl/ClientOptionsImpl.java | 19 +++++++++++++++++-- .../impl/ExecuteQueryResponseHandler.java | 19 ++++++++++++++++--- 4 files changed, 43 insertions(+), 6 deletions(-) diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java index a4d8b1023bb3..93c9d80ad289 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/ClientOptions.java @@ -41,6 +41,8 @@ public interface ClientOptions { ClientOptions setBasicAuthCredentials(String username, String password); + ClientOptions setExecuteQueryMaxResultRows(int maxRows); + String getHost(); int getPort(); @@ -67,6 +69,8 @@ public interface ClientOptions { String getBasicAuthPassword(); + int getExecuteQueryMaxResultRows(); + ClientOptions copy(); static ClientOptions create() { diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index fbf9a43a4eae..5a7b0ceb3de8 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -95,7 +95,12 @@ public CompletableFuture> executeQuery( final String sql, final Map properties ) { - return makeQueryRequest(sql, properties, ExecuteQueryResponseHandler::new); + return makeQueryRequest( + sql, + properties, + (context, recordParser, cf) -> new ExecuteQueryResponseHandler( + context, recordParser, cf, clientOptions.getExecuteQueryMaxResultRows()) + ); } @Override diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java index 8b10d397af11..393a3e6d73e6 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientOptionsImpl.java @@ -33,6 +33,7 @@ public class ClientOptionsImpl implements ClientOptions { private String keyStorePassword; private String basicAuthUsername; private String basicAuthPassword; + private int executeQueryMaxResultRows = 10000; public ClientOptionsImpl() { } @@ -46,7 +47,8 @@ private ClientOptionsImpl( final boolean useBasicAuth, final String trustStorePath, final String trustStorePassword, final String keyStorePath, final String keyStorePassword, - final String basicAuthUsername, final String basicAuthPassword) { + final String basicAuthUsername, final String basicAuthPassword, + final int executeQueryMaxResultRows) { this.host = Objects.requireNonNull(host); this.port = port; this.useTls = useTls; @@ -60,6 +62,7 @@ private ClientOptionsImpl( this.keyStorePassword = keyStorePassword; this.basicAuthUsername = basicAuthUsername; this.basicAuthPassword = basicAuthPassword; + this.executeQueryMaxResultRows = executeQueryMaxResultRows; } @Override @@ -130,6 +133,12 @@ public ClientOptions setBasicAuthCredentials(final String username, final String return this; } + @Override + public ClientOptions setExecuteQueryMaxResultRows(final int maxRows) { + this.executeQueryMaxResultRows = maxRows; + return this; + } + @Override public String getHost() { return host == null ? "" : host; @@ -195,6 +204,11 @@ public String getBasicAuthPassword() { return basicAuthPassword == null ? "" : basicAuthPassword; } + @Override + public int getExecuteQueryMaxResultRows() { + return executeQueryMaxResultRows; + } + @Override public ClientOptions copy() { return new ClientOptionsImpl( @@ -204,6 +218,7 @@ public ClientOptions copy() { useBasicAuth, trustStorePath, trustStorePassword, keyStorePath, keyStorePassword, - basicAuthUsername, basicAuthPassword); + basicAuthUsername, basicAuthPassword, + executeQueryMaxResultRows); } } \ No newline at end of file diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java index b74ba45f7a88..0d0ef6c3f096 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java @@ -18,6 +18,7 @@ import io.confluent.ksql.api.client.Row; import io.confluent.ksql.api.client.util.RowUtil; import io.confluent.ksql.api.server.protocol.QueryResponseMetadata; +import io.confluent.ksql.rest.client.KsqlRestClientException; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; import io.vertx.core.json.JsonArray; @@ -34,13 +35,18 @@ public class ExecuteQueryResponseHandler extends QueryResponseHandler> private static final Logger log = LoggerFactory.getLogger(ExecuteQueryResponseHandler.class); private final List rows; + private final int maxRows; private List columnNames; private List columnTypes; private Map columnNameToIndex; - ExecuteQueryResponseHandler(final Context context, final RecordParser recordParser, - final CompletableFuture> cf) { + ExecuteQueryResponseHandler( + final Context context, + final RecordParser recordParser, + final CompletableFuture> cf, + final int maxRows) { super(context, recordParser, cf); + this.maxRows = maxRows; this.rows = new ArrayList<>(); } @@ -54,7 +60,14 @@ protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) @Override protected void handleRow(final Buffer buff) { final JsonArray values = new JsonArray(buff); - rows.add(new RowImpl(columnNames, columnTypes, values, columnNameToIndex)); + if (rows.size() < maxRows) { + rows.add(new RowImpl(columnNames, columnTypes, values, columnNameToIndex)); + } else { + throw new KsqlRestClientException( + "Reached max number of rows that may be returned by executeQuery(). " + + "Increase the limit via ClientOptions#setExecuteQueryMaxResultRows(). " + + "Current limit: " + maxRows); + } } @Override From bb5ac44de13c569fd327bcc6731f56ec30d2d7eb Mon Sep 17 00:00:00 2001 From: Victoria Xia Date: Fri, 8 May 2020 22:48:40 -0700 Subject: [PATCH 19/19] chore: findbugs --- .../confluent/ksql/api/client/ClientTest.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 4dba6ad84da7..add22a3a0c57 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -53,27 +53,26 @@ public class ClientTest extends BaseApiTest { BaseApiTest.DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES.getMap(); protected static final String DEFAULT_PUSH_QUERY_WITH_LIMIT = "select * from foo emit changes limit 10;"; - protected Client client; + protected Client javaClient; @Override public void setUp() { super.setUp(); - // Use Java client for these tests, rather than WebClient as in BaseApiTest - this.client = createJavaClient(); + this.javaClient = createJavaClient(); } @Override protected WebClient createClient() { - // Ensure these tests use Java client rather than WebClient + // Ensure these tests use Java client rather than WebClient (as in BaseApiTest) return null; } @Override protected void stopClient() { - if (client != null) { + if (javaClient != null) { try { - client.close(); + javaClient.close(); } catch (Exception e) { log.error("Failed to close client", e); } @@ -84,7 +83,7 @@ protected void stopClient() { public void shouldStreamPushQueryAsync() throws Exception { // When final QueryResult queryResult = - client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); + javaClient.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); // Then assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); @@ -101,7 +100,7 @@ public void shouldStreamPushQueryAsync() throws Exception { public void shouldStreamPushQuerySync() throws Exception { // When final QueryResult queryResult = - client.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); + javaClient.streamQuery(DEFAULT_PUSH_QUERY, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); // Then assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); @@ -123,7 +122,7 @@ public void shouldStreamPushQuerySync() throws Exception { public void shouldStreamPullQueryAsync() throws Exception { // When final QueryResult queryResult = - client.streamQuery(DEFAULT_PULL_QUERY).get(); + javaClient.streamQuery(DEFAULT_PULL_QUERY).get(); // Then assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); @@ -138,7 +137,7 @@ public void shouldStreamPullQueryAsync() throws Exception { public void shouldStreamPullQuerySync() throws Exception { // When final QueryResult queryResult = - client.streamQuery(DEFAULT_PULL_QUERY).get(); + javaClient.streamQuery(DEFAULT_PULL_QUERY).get(); // Then assertThat(queryResult.columnNames(), is(DEFAULT_COLUMN_NAMES)); @@ -163,7 +162,7 @@ public void shouldHandleErrorResponseFromStreamQuery() { // When final Exception e = assertThrows( ExecutionException.class, // thrown from .get() when the future completes exceptionally - () -> client.streamQuery("bad query", DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get() + () -> javaClient.streamQuery("bad query", DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get() ); // Then @@ -174,7 +173,7 @@ public void shouldHandleErrorResponseFromStreamQuery() { @Test public void shouldExecutePullQuery() throws Exception { // When - final List rows = client.executeQuery(DEFAULT_PULL_QUERY).get(); + final List rows = javaClient.executeQuery(DEFAULT_PULL_QUERY).get(); // Then assertThat(rows, hasSize(DEFAULT_ROWS.size())); @@ -191,7 +190,7 @@ public void shouldExecutePullQuery() throws Exception { public void shouldExecutePushQuery() throws Exception { // When final List rows = - client.executeQuery(DEFAULT_PUSH_QUERY_WITH_LIMIT, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); + javaClient.executeQuery(DEFAULT_PUSH_QUERY_WITH_LIMIT, DEFAULT_PUSH_QUERY_REQUEST_PROPERTIES).get(); // Then assertThat(rows, hasSize(DEFAULT_ROWS.size())); @@ -213,7 +212,7 @@ public void shouldHandleErrorResponseFromExecuteQuery() { // When final Exception e = assertThrows( ExecutionException.class, // thrown from .get() when the future completes exceptionally - () -> client.executeQuery("bad query").get() + () -> javaClient.executeQuery("bad query").get() ); // Then