From 54c5139ed3fd6de1fe933b1fb8040f163776510b Mon Sep 17 00:00:00 2001 From: Andy Coates <8012398+big-andy-coates@users.noreply.github.com> Date: Fri, 13 Sep 2019 16:13:08 +0100 Subject: [PATCH] feat(static): initial drop of static query functionality (#3340) * feat(static): initial drop of static query functionality The change brings in the first batch of functionality for static query. It allows static queries on both windowed and non-windowed aggregate tables, i.e. tables build with a `GROUP BY` clause. For non-windowed queries the following form is accepted: ``` SELECT * FROM X WHERE ROWKEY=Y; ``` - The only filter allowed is an equals filter on ROWKEY. - Only a single ROWKEY is supported at the moment. - Only `SELECT *` is supported at the moment. For windowed queries the following forms is accepted: ``` -- exact window start: SELECT * FROM X WHERE ROWKEY=Y AND WINDOWSTART=X; SELECT * FROM X WHERE ROWKEY=Y AND WINDOWSTART='2020-02-23T23:45:12.000'; -- windwo start range: SELECT * FROM X WHERE ROWKEY=Y AND WINDOWSTART>A AND WINDOWSTART<=B; SELECT * FROM X WHERE ROWKEY=Y AND '2020-02-23T23:45:12.000' < WINDOWSTART AND WINDOWSTART <= '2020-02-24T23:45:12.000'; ``` All types of windowed tables are supported. At the moment, static query requests are served over the standard `/ksql` HTTP POST request. But this may change to being a websocket request. * chore: fix test * chore: reviewer's requested changes --- .../main/java/io/confluent/ksql/cli/Cli.java | 18 +- .../confluent/ksql/cli/console/Console.java | 4 + .../ksql/cli/console/table/Table.java | 29 +- .../builder/QueryResultTableBuilder.java | 97 +++ .../java/io/confluent/ksql/cli/CliTest.java | 60 +- .../builder/QueryResultTableBuilderTest.java | 247 +++++++ .../io/confluent/ksql/util/KsqlConstants.java | 7 +- .../materialization/KsqlMaterialization.java | 135 ++++ .../KsqlMaterializationFactory.java | 189 ++++++ .../ksql/materialization/Locator.java | 54 ++ .../ksql/materialization/Materialization.java | 48 ++ .../MaterializationException.java | 30 + .../materialization/MaterializationInfo.java | 100 +++ .../MaterializationProvider.java | 29 + .../MaterializationTimeOutException.java | 23 + .../materialization/MaterializedTable.java | 34 + .../MaterializedWindowedTable.java | 38 ++ .../materialization/NotRunningException.java | 23 + .../ksql/materialization/Window.java | 86 +++ .../ksql/materialization/ks/KsLocator.java | 117 ++++ .../materialization/ks/KsMaterialization.java | 89 +++ .../ks/KsMaterializationFactory.java | 144 +++++ .../ks/KsMaterializedSessionTable.java | 96 +++ .../ks/KsMaterializedTable.java | 51 ++ .../ks/KsMaterializedWindowTable.java | 71 ++ .../ksql/materialization/ks/KsStateStore.java | 87 +++ .../ksql/physical/PhysicalPlanBuilder.java | 132 +++- .../ksql/planner/plan/AggregateNode.java | 45 +- .../ksql/planner/plan/DataSourceNode.java | 3 +- .../ksql/structured/SqlPredicate.java | 6 +- .../ksql/util/PersistentQueryMetadata.java | 15 + .../confluent/ksql/engine/KsqlEngineTest.java | 20 + .../ksql/integration/TestKsqlContext.java | 20 +- .../KsqlMaterializationFactoryTest.java | 235 +++++++ .../KsqlMaterializationTest.java | 346 ++++++++++ .../ksql/materialization/WindowTest.java | 56 ++ .../materialization/ks/KsLocatorTest.java | 206 ++++++ .../ks/KsMaterializationFactoryTest.java | 168 +++++ .../ks/KsMaterializationFunctionalTest.java | 593 +++++++++++++++++ .../ks/KsMaterializationTest.java | 140 ++++ .../ks/KsMaterializedSessionTableTest.java | 241 +++++++ .../ks/KsMaterializedTableTest.java | 140 ++++ .../ks/KsMaterializedWindowTableTest.java | 206 ++++++ .../materialization/ks/KsStateStoreTest.java | 222 +++++++ .../ksql/test/rest/RestTestExecutor.java | 130 +++- .../hopping-windows.json | 23 + ...materialized-aggregate-static-queries.json | 394 +++++++++++ .../rewrite/StatementRewriteForRowtime.java | 24 +- ksql-rest-app/pom.xml | 11 +- .../rest/entity/QueryResultEntityFactory.java | 112 ++++ .../server/execution/CustomExecutors.java | 3 + .../server/execution/StaticQueryExecutor.java | 611 ++++++++++++++++++ .../server/validation/CustomValidators.java | 3 +- .../server/validation/QueryValidator.java | 35 - .../entity/QueryDescriptionFactoryTest.java | 2 + .../entity/QueryResultEntityFactoryTest.java | 193 ++++++ .../StaticQueryFunctionalTest.java | 226 +++++++ .../computation/StatementExecutorTest.java | 2 +- .../server/resources/KsqlResourceTest.java | 16 +- .../rest/entity/QueryResultEntityTest.java | 271 ++++++++ .../java/io/confluent/ksql/rest/Errors.java | 2 +- .../ksql/rest/entity/KsqlEntity.java | 3 +- .../ksql/rest/entity/QueryResultEntity.java | 226 +++++++ .../execution/streams/SelectValueMapper.java | 2 +- 64 files changed, 6844 insertions(+), 145 deletions(-) create mode 100644 ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilder.java create mode 100644 ksql-cli/src/test/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilderTest.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterialization.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterializationFactory.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/Locator.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/Materialization.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationException.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationInfo.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationProvider.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationTimeOutException.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedTable.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedWindowedTable.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/NotRunningException.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/Window.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsLocator.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterialization.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializationFactory.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTable.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedTable.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTable.java create mode 100644 ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsStateStore.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationFactoryTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/WindowTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsLocatorTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFactoryTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTableTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedTableTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTableTest.java create mode 100644 ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsStateStoreTest.java create mode 100644 ksql-functional-tests/src/test/resources/rest-query-validation-tests/materialized-aggregate-static-queries.json create mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntityFactory.java create mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/StaticQueryExecutor.java delete mode 100644 ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/QueryValidator.java create mode 100644 ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityFactoryTest.java create mode 100644 ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/StaticQueryFunctionalTest.java create mode 100644 ksql-rest-client/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityTest.java create mode 100644 ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntity.java diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java index 190866789e89..37d6394d7191 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/Cli.java @@ -26,6 +26,7 @@ import io.confluent.ksql.parser.DefaultKsqlParser; import io.confluent.ksql.parser.KsqlParser.ParsedStatement; import io.confluent.ksql.parser.SqlBaseParser; +import io.confluent.ksql.parser.SqlBaseParser.QueryStatementContext; import io.confluent.ksql.parser.SqlBaseParser.SingleStatementContext; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.client.KsqlRestClient; @@ -257,8 +258,21 @@ private void handleStatements(final String line) final SingleStatementContext statementContext = statement.getStatement(); final String statementText = statement.getStatementText(); - if (statementContext.statement() instanceof SqlBaseParser.QueryStatementContext - || statementContext.statement() instanceof SqlBaseParser.PrintTopicContext) { + if (statementContext.statement() instanceof SqlBaseParser.QueryStatementContext) { + + final QueryStatementContext queryContext = + (QueryStatementContext) statementContext.statement(); + + if (queryContext.query().EMIT() != null) { + consecutiveStatements = printOrDisplayQueryResults( + consecutiveStatements, + statementContext, + statementText + ); + } else { + makeKsqlRequest(statementText); + } + } else if (statementContext.statement() instanceof SqlBaseParser.PrintTopicContext) { consecutiveStatements = printOrDisplayQueryResults( consecutiveStatements, statementContext, diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index f5e250625e99..463474e08a42 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -36,6 +36,7 @@ import io.confluent.ksql.cli.console.table.builder.KafkaTopicsListTableBuilder; import io.confluent.ksql.cli.console.table.builder.PropertiesListTableBuilder; import io.confluent.ksql.cli.console.table.builder.QueriesTableBuilder; +import io.confluent.ksql.cli.console.table.builder.QueryResultTableBuilder; import io.confluent.ksql.cli.console.table.builder.StreamsListTableBuilder; import io.confluent.ksql.cli.console.table.builder.TableBuilder; import io.confluent.ksql.cli.console.table.builder.TablesListTableBuilder; @@ -66,6 +67,7 @@ import io.confluent.ksql.rest.entity.QueryDescription; import io.confluent.ksql.rest.entity.QueryDescriptionEntity; import io.confluent.ksql.rest.entity.QueryDescriptionList; +import io.confluent.ksql.rest.entity.QueryResultEntity; import io.confluent.ksql.rest.entity.RunningQuery; import io.confluent.ksql.rest.entity.SourceDescription; import io.confluent.ksql.rest.entity.SourceDescriptionEntity; @@ -161,6 +163,8 @@ public class Console implements Closeable { tablePrinter(TypeList.class, TypeListTableBuilder::new)) .put(ErrorEntity.class, tablePrinter(ErrorEntity.class, ErrorEntityTableBuilder::new)) + .put(QueryResultEntity.class, + tablePrinter(QueryResultEntity.class, QueryResultTableBuilder::new)) .build(); private static Handler1 tablePrinter( diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/Table.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/Table.java index c1bb801f24bc..cc7bfc03f3ba 100644 --- a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/Table.java +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/Table.java @@ -15,6 +15,9 @@ package io.confluent.ksql.cli.console.table; +import static java.util.Objects.requireNonNull; + +import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.cli.console.Console; import java.util.Arrays; import java.util.LinkedList; @@ -30,12 +33,26 @@ public final class Table { private final List header; private final List footer; - private Table(final List columnHeaders, final List> rowValues, - final List header, final List footer) { - this.columnHeaders = columnHeaders; - this.rowValues = rowValues; - this.header = header; - this.footer = footer; + private Table( + final List columnHeaders, + final List> rowValues, + final List header, + final List footer + ) { + this.columnHeaders = requireNonNull(columnHeaders, "columnHeaders"); + this.rowValues = requireNonNull(rowValues, "rowValues"); + this.header = requireNonNull(header, "header"); + this.footer = requireNonNull(footer, "footer"); + } + + @VisibleForTesting + public List headers() { + return columnHeaders; + } + + @VisibleForTesting + public List> rows() { + return rowValues; } public static final class Builder { diff --git a/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilder.java b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilder.java new file mode 100644 index 000000000000..8135d78014c6 --- /dev/null +++ b/ksql-cli/src/main/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilder.java @@ -0,0 +1,97 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.cli.console.table.builder; + +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.cli.console.table.Table; +import io.confluent.ksql.cli.console.table.Table.Builder; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.rest.entity.QueryResultEntity; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.schema.ksql.Column; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class QueryResultTableBuilder implements TableBuilder { + + private static final List TIME_WINDOW_HEADINGS = ImmutableList + .of("WINDOWSTART BIGINT"); + + private static final List SESSION_WINDOW_HEADINGS = ImmutableList.builder() + .addAll(TIME_WINDOW_HEADINGS) + .add("WINDOWEND BIGINT") + .build(); + + @Override + public Table buildTable(final QueryResultEntity entity) { + + final List headers = buildHeadings(entity); + + final Stream> rows = entity + .getRows() + .stream() + .map(r -> buildRow(r, entity.getSchema().value())); + + return new Builder() + .withColumnHeaders(headers) + .withRows(rows) + .build(); + } + + private static List buildHeadings(final QueryResultEntity entity) { + final LogicalSchema schema = entity.getSchema(); + + final Stream keys = schema.key().stream() + .map(f -> f.fullName() + " " + f.type() + " KEY"); + + final Stream window = entity.getWindowType() + .map(wt -> wt == WindowType.SESSION + ? SESSION_WINDOW_HEADINGS + : TIME_WINDOW_HEADINGS) + .orElse(ImmutableList.of()) + .stream(); + + final Stream values = schema.value().stream() + .map(f -> f.fullName() + " " + f.type()); + + return Stream.concat(keys, Stream.concat(window, values)) + .collect(Collectors.toList()); + } + + private static List buildRow( + final Row row, + final List valueSchema + ) { + final Stream keys = row.getKey().values().stream(); + + final Stream window = row.getWindow() + .map(w -> w.getEnd().isPresent() + ? Stream.of(w.getStart(), w.getEnd().getAsLong()) + : Stream.of(w.getStart())) + .orElse(Stream.of()); + + final Stream values = row.getValue() == null + ? valueSchema.stream().map(f -> null) + : row.getValue().values().stream(); + + return Stream.concat(keys, Stream.concat(window, values)) + .map(Objects::toString) + .collect(Collectors.toList()); + } +} diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java index 8380618cbab2..8946999ed2a4 100644 --- a/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/CliTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.any; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; @@ -54,7 +55,6 @@ import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.KsqlErrorMessage; import io.confluent.ksql.rest.entity.ServerInfo; -import io.confluent.ksql.rest.server.KsqlRestApplication; import io.confluent.ksql.rest.server.TestKsqlRestApp; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; @@ -143,7 +143,6 @@ public class CliTest { .withLookingForStuckThread(true) .build(); - private static final String COMMANDS_KSQL_TOPIC_NAME = KsqlRestApplication.COMMANDS_STREAM_NAME; private static final OutputFormat CLI_OUTPUT_FORMAT = OutputFormat.TABULAR; private static final long STREAMED_QUERY_ROW_LIMIT = 10000; @@ -184,6 +183,9 @@ public static void classSetUp() throws Exception { @Before public void setUp() { + REST_APP.closePersistentQueries(); + REST_APP.dropSourcesExcept(orderDataProvider.kstreamName()); + streamName = KsqlIdentifierTestUtil.uniqueIdentifierName(); tableName = KsqlIdentifierTestUtil.uniqueIdentifierName(); terminal = new TestTerminal(lineSupplier); @@ -196,8 +198,6 @@ public void setUp() { restClient, console ); - - maybeDropStream("SHOULDRUNSCRIPT"); } private void assertRunListCommand( @@ -349,18 +349,6 @@ private static void dropStreamOrTable(final boolean stream, final String name) { runStatement(dropStatement, restClient); } - private static void maybeDropStream(final String name) { - final String dropStatement = String.format("drop stream %s;", name); - - final RestResponse response = restClient.makeKsqlRequest(dropStatement, null); - if (response.isSuccessful() - || response.getErrorMessage().toString().contains("does not exist")) { - return; - } - - dropStream(name); - } - private void selectWithLimit( final String selectQuery, final int limit, @@ -604,7 +592,38 @@ public void testTransientSelect() { } @Test - public void testTransientSelectStar() { + public void testTransientStaticSelectStar() { + // Given: + run("CREATE TABLE X AS SELECT COUNT(1) AS COUNT " + + "FROM " + orderDataProvider.kstreamName() + + " GROUP BY ITEMID;", + localCli + ); + + assertRunCommand( + "SELECT * FROM X WHERE ROWKEY='unknowwn';", + is(emptyIterable()) + ); + } + + @Test + public void testTransientStaticHeader() { + // Given: + run("CREATE TABLE Y AS SELECT COUNT(1) AS COUNT " + + "FROM " + orderDataProvider.kstreamName() + + " GROUP BY ITEMID;", + localCli + ); + + // When: + run("SELECT * FROM Y WHERE ROWKEY='ITEM_1';", localCli); + + assertThat(terminal.getOutputString(), containsString("ROWKEY STRING KEY")); + assertThat(terminal.getOutputString(), containsString("COUNT BIGINT")); + } + + @Test + public void testTransientContinuousSelectStar() { final Map streamData = orderDataProvider.data(); final List row1 = streamData.get("1").getColumns(); final List row2 = streamData.get("2").getColumns(); @@ -621,9 +640,8 @@ public void testTransientSelectStar() { } @Test - public void testTransientHeader() { + public void testTransientContinuousHeader() { // When: - rowCaptor.resetTestResult(); run("SELECT * FROM " + orderDataProvider.kstreamName() + " EMIT CHANGES LIMIT 1", localCli); // Then: (note that some of these are truncated because of header wrapping) @@ -683,10 +701,6 @@ public void testCreateTable() { dropTable(tableName); } - // =================================================================== - // Below Tests are only used for coverage, not for results validation. - // =================================================================== - @Test public void testRunInteractively() { // Given: diff --git a/ksql-cli/src/test/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilderTest.java b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilderTest.java new file mode 100644 index 000000000000..7b8663ac96da --- /dev/null +++ b/ksql-cli/src/test/java/io/confluent/ksql/cli/console/table/builder/QueryResultTableBuilderTest.java @@ -0,0 +1,247 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.cli.console.table.builder; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.cli.console.table.Table; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.rest.entity.QueryResultEntity; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.rest.entity.QueryResultEntity.Window; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import java.util.LinkedHashMap; +import java.util.Optional; +import java.util.OptionalLong; +import org.junit.Before; +import org.junit.Test; + + +public class QueryResultTableBuilderTest { + + private static final String SOME_SQL = "some sql"; + + private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() + .keyColumn("k0", SqlTypes.BIGINT) + .keyColumn("k1", SqlTypes.DOUBLE) + .valueColumn("v0", SqlTypes.STRING) + .valueColumn("v1", SqlTypes.INTEGER) + .build(); + + private static final Optional SESSION_WINDOW = Optional + .of(new Window(12_234, OptionalLong.of(43_234))); + + private static final Optional TIME_WINDOW = Optional + .of(new Window(12_234, OptionalLong.empty())); + + private static final LinkedHashMap A_KEY = + orderedMap("k0", 10L, "k1", 5.1D); + + private static final LinkedHashMap A_VALUE = + orderedMap("v0", "x", "v1", 5); + + private QueryResultTableBuilder builder; + + @Before + public void setUp() { + builder = new QueryResultTableBuilder(); + } + + @Test + public void shouldBuildTableHeadings() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(Optional.empty(), A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.headers(), contains( + "k0 BIGINT KEY", + "k1 DOUBLE KEY", + "v0 STRING", + "v1 INTEGER" + )); + } + + @Test + public void shouldBuildTimeWindowedTableHeadings() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.TUMBLING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(TIME_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.headers(), contains( + "k0 BIGINT KEY", + "k1 DOUBLE KEY", + "WINDOWSTART BIGINT", + "v0 STRING", + "v1 INTEGER" + )); + } + + @Test + public void shouldBuildSessionWindowedTableHeadings() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.headers(), contains( + "k0 BIGINT KEY", + "k1 DOUBLE KEY", + "WINDOWSTART BIGINT", + "WINDOWEND BIGINT", + "v0 STRING", + "v1 INTEGER" + )); + } + + @Test + public void shouldBuildTableRows() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(Optional.empty(), A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.rows(), hasSize(1)); + assertThat(table.rows().get(0), contains("10", "5.1", "x", "5")); + } + + @Test + public void shouldBuildTimeWindowedTableRows() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.HOPPING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(TIME_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.rows(), hasSize(1)); + assertThat(table.rows().get(0), contains("10", "5.1", "12234", "x", "5")); + } + + @Test + public void shouldBuildSessionWindowedTableRows() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.rows(), hasSize(1)); + assertThat(table.rows().get(0), contains("10", "5.1", "12234", "43234", "x", "5")); + } + + @Test + public void shouldHandleNullValue() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(Optional.empty(), A_KEY, null)) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.rows(), hasSize(1)); + assertThat(table.rows().get(0), contains("10", "5.1", "null", "null")); + } + + @Test + public void shouldHandleNullFields() { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row( + Optional.empty(), + orderedMap("k0", 10L, "k1", null), + orderedMap("v0", "x", "v1", null) + )) + ); + + // When: + final Table table = builder.buildTable(entity); + + // Then: + assertThat(table.rows(), hasSize(1)); + assertThat(table.rows().get(0), contains("10", "null", "x", "null")); + } + + private static LinkedHashMap orderedMap(final Object... keysAndValues) { + assertThat("invalid test", keysAndValues.length % 2, is(0)); + + final LinkedHashMap orderedMap = new LinkedHashMap<>(); + + for (int idx = 0; idx < keysAndValues.length; idx = idx + 2) { + final Object key = keysAndValues[idx]; + final Object value = keysAndValues[idx + 1]; + + assertThat("invalid test", key, instanceOf(String.class)); + orderedMap.put((String) key, value); + } + + return orderedMap; + } +} \ No newline at end of file diff --git a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConstants.java b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConstants.java index 155740abcbb7..0bcd44f00429 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConstants.java +++ b/ksql-common/src/main/java/io/confluent/ksql/util/KsqlConstants.java @@ -32,7 +32,6 @@ private KsqlConstants() { public static final int legacyDefaultSinkPartitionCount = 4; public static final short legacyDefaultSinkReplicaCount = 1; - // TODO: Find out the best default value. public static final long defaultSinkWindowChangeLogAdditionalRetention = 1000000; public static final String defaultAutoOffsetRestConfig = "latest"; @@ -50,4 +49,10 @@ private KsqlConstants() { public static final String AVRO_SCHEMA_NAME = "KsqlDataSourceSchema"; public static final String DEFAULT_AVRO_SCHEMA_FULL_NAME = AVRO_SCHEMA_NAMESPACE + "." + AVRO_SCHEMA_NAME; + + /** + * Default time and date patterns + */ + public static final String TIME_PATTERN = "HH:mm:ss.SSS"; + public static final String DATE_TIME_PATTERN = "yyyy-MM-dd'T'" + TIME_PATTERN; } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterialization.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterialization.java new file mode 100644 index 000000000000..f590b8c47d0e --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterialization.java @@ -0,0 +1,135 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import static java.util.Objects.requireNonNull; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMap.Builder; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.time.Instant; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.function.Function; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.kstream.Predicate; + +/** + * {@link Materialization} implementation responsible for handling HAVING and SELECT clauses. + * + *

Underlying {@link Materialization} store data in a different schema and have not had any + * HAVING predicate applied. Mapping from the aggregate store schema to the table's schema and + * applying any HAVING predicate is handled by this class. + */ +class KsqlMaterialization implements Materialization { + + private final Materialization inner; + private final Predicate havingPredicate; + private final Function storeToTableTransform; + private final LogicalSchema schema; + + KsqlMaterialization( + final Materialization inner, + final Predicate havingPredicate, + final Function storeToTableTransform, + final LogicalSchema schema + ) { + this.inner = requireNonNull(inner, "table"); + this.havingPredicate = requireNonNull(havingPredicate, "havingPredicate"); + this.storeToTableTransform = requireNonNull(storeToTableTransform, "storeToTableTransform"); + this.schema = requireNonNull(schema, "schema"); + } + + @Override + public LogicalSchema schema() { + return schema; + } + + @Override + public Locator locator() { + return inner.locator(); + } + + @Override + public Optional windowType() { + return inner.windowType(); + } + + @Override + public MaterializedTable nonWindowed() { + return new KsqlMaterializedTable(inner.nonWindowed()); + } + + @Override + public MaterializedWindowedTable windowed() { + return new KsqlMaterializedWindowedTable(inner.windowed()); + } + + private Optional filterAndTransform( + final Struct key, + final GenericRow row + ) { + return Optional.of(row) + // HAVING predicate from source table query that has not already been applied to the + // store, so must be applied to any result from the store. + .filter(value -> havingPredicate.test(key, value)) + // SELECTS that map from the stores internal schema to the tables true schema + // i.e. maps from internal schema of the store to external schema of the table: + .map(storeToTableTransform); + } + + final class KsqlMaterializedTable implements MaterializedTable { + + private final MaterializedTable table; + + KsqlMaterializedTable(final MaterializedTable table) { + this.table = requireNonNull(table, "table'"); + } + + @Override + public Optional get(final Struct key) { + return table.get(key) + .flatMap(v -> filterAndTransform(key, v)); + } + } + + final class KsqlMaterializedWindowedTable implements MaterializedWindowedTable { + + private final MaterializedWindowedTable table; + + KsqlMaterializedWindowedTable(final MaterializedWindowedTable table) { + this.table = requireNonNull(table, "table'"); + } + + @Override + public Map get(final Struct key, final Instant lower, final Instant upper) { + final Map result = table.get(key, lower, upper); + + final Builder builder = ImmutableMap.builder(); + + for (final Entry e : result.entrySet()) { + filterAndTransform(key, e.getValue()) + .ifPresent(v -> builder.put(e.getKey(), v)); + } + + return builder.build(); + } + } +} + diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterializationFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterializationFactory.java new file mode 100644 index 000000000000..194e7acaeb31 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/KsqlMaterializationFactory.java @@ -0,0 +1,189 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import static java.util.Objects.requireNonNull; + +import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.execution.context.QueryContext.Stacker; +import io.confluent.ksql.execution.context.QueryLoggerUtil; +import io.confluent.ksql.execution.expression.tree.Expression; +import io.confluent.ksql.execution.plan.SelectExpression; +import io.confluent.ksql.execution.streams.SelectValueMapperFactory; +import io.confluent.ksql.function.FunctionRegistry; +import io.confluent.ksql.logging.processing.ProcessingLogContext; +import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.structured.SqlPredicate; +import io.confluent.ksql.util.KsqlConfig; +import java.util.List; +import java.util.function.Function; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.kstream.Predicate; + +/** + * Factor class for {@link KsqlMaterialization}. + */ +public final class KsqlMaterializationFactory { + + private static final String FILTER_OP_NAME = "filter"; + private static final String PROJECT_OP_NAME = "project"; + + private final KsqlConfig ksqlConfig; + private final FunctionRegistry functionRegistry; + private final ProcessingLogContext processingLogContext; + private final SqlPredicateFactory sqlPredicateFactory; + private final ValueMapperFactory valueMapperFactory; + private final MaterializationFactory materializationFactory; + + public KsqlMaterializationFactory( + final KsqlConfig ksqlConfig, + final FunctionRegistry functionRegistry, + final ProcessingLogContext processingLogContext + ) { + this( + ksqlConfig, + functionRegistry, + processingLogContext, + SqlPredicate::new, + defaultValueMapperFactory(), + KsqlMaterialization::new + ); + } + + @VisibleForTesting + KsqlMaterializationFactory( + final KsqlConfig ksqlConfig, + final FunctionRegistry functionRegistry, + final ProcessingLogContext processingLogContext, + final SqlPredicateFactory sqlPredicateFactory, + final ValueMapperFactory valueMapperFactory, + final MaterializationFactory materializationFactory + ) { + this.ksqlConfig = requireNonNull(ksqlConfig, "ksqlConfig"); + this.functionRegistry = requireNonNull(functionRegistry, "functionRegistry"); + this.processingLogContext = requireNonNull(processingLogContext, "processingLogContext"); + this.sqlPredicateFactory = requireNonNull(sqlPredicateFactory, "sqlPredicateFactory"); + this.valueMapperFactory = requireNonNull(valueMapperFactory, "valueMapperFactory"); + this.materializationFactory = requireNonNull(materializationFactory, "materializationFactory"); + } + + public Materialization create( + final Materialization delegate, + final MaterializationInfo info, + final QueryContext.Stacker contextStacker + ) { + final Predicate havingPredicate = + bakeHavingExpression(info, contextStacker); + + final Function valueMapper = + bakeStoreSelects(info, contextStacker); + + return materializationFactory.create( + delegate, + havingPredicate, + valueMapper, + info.tableSchema() + ); + } + + private Predicate bakeHavingExpression( + final MaterializationInfo info, + final QueryContext.Stacker contextStacker + ) { + if (!info.havingExpression().isPresent()) { + return (k, v) -> true; + } + + final Expression having = info.havingExpression().get(); + + final ProcessingLogger logger = processingLogContext.getLoggerFactory().getLogger( + QueryLoggerUtil.queryLoggerName(contextStacker.push(FILTER_OP_NAME).getQueryContext()) + ); + + final SqlPredicate predicate = sqlPredicateFactory.create( + having, + info.aggregationSchema(), + ksqlConfig, + functionRegistry, + logger + ); + + return predicate.getPredicate(); + } + + private Function bakeStoreSelects( + final MaterializationInfo info, + final Stacker contextStacker + ) { + final ProcessingLogger logger = processingLogContext.getLoggerFactory().getLogger( + QueryLoggerUtil.queryLoggerName(contextStacker.push(PROJECT_OP_NAME).getQueryContext()) + ); + + return valueMapperFactory.create( + info.tableSelects(), + info.aggregationSchema(), + ksqlConfig, + functionRegistry, + logger + ); + } + + private static ValueMapperFactory defaultValueMapperFactory() { + return (selectExpressions, sourceSchema, ksqlConfig, functionRegistry, processingLogger) -> + SelectValueMapperFactory.create( + selectExpressions, + sourceSchema, + ksqlConfig, + functionRegistry, + processingLogger + )::apply; + } + + interface SqlPredicateFactory { + + SqlPredicate create( + Expression filterExpression, + LogicalSchema schema, + KsqlConfig ksqlConfig, + FunctionRegistry functionRegistry, + ProcessingLogger processingLogger + ); + } + + interface ValueMapperFactory { + + Function create( + List selectExpressions, + LogicalSchema sourceSchema, + KsqlConfig ksqlConfig, + FunctionRegistry functionRegistry, + ProcessingLogger processingLogger + ); + } + + interface MaterializationFactory { + + KsqlMaterialization create( + Materialization inner, + Predicate havingPredicate, + Function storeToTableTransform, + LogicalSchema schema + ); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/Locator.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Locator.java new file mode 100644 index 000000000000..d1ea04d1a697 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Locator.java @@ -0,0 +1,54 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import java.net.URL; +import java.util.Optional; +import org.apache.kafka.connect.data.Struct; + +/** + * Type used to locate on which KSQL node materialized data is stored. + * + *

Data stored in materialized stores can be spread across KSQL nodes. This type can be used to + * determine which KSQL server stores a specific key. + */ +public interface Locator { + + /** + * Locate which KSQL node stores the supplied {@code key}. + * + *

Implementations are free to return {@link Optional#empty()} if the location is not known at + * this time. + * + * @param key the required key. + * @return the owning node, if known. + */ + Optional locate(Struct key); + + + interface KsqlNode { + + /** + * @return {@code true} if this is the local node, i.e. the KSQL instance handling the call. + */ + boolean isLocal(); + + /** + * @return The base URL of the node, including protocol, host and port. + */ + URL location(); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/Materialization.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Materialization.java new file mode 100644 index 000000000000..1b160ee8581b --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Materialization.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.util.Optional; + +public interface Materialization { + + /** + * @return The schema of the materialization + */ + LogicalSchema schema(); + + /** + * @return service for locating which node holds specific data. + */ + Locator locator(); + + /** + * @return the window type of the materialization. + */ + Optional windowType(); + + /** + * @return access to the materialized non-windowed table. + */ + MaterializedTable nonWindowed(); + + /** + * @return access to the materialized windowed table. + */ + MaterializedWindowedTable windowed(); +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationException.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationException.java new file mode 100644 index 000000000000..8590baa38cf6 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationException.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +/** + * Base class for all table materialization exceptions. + */ +public class MaterializationException extends RuntimeException { + + public MaterializationException(final String message) { + super(message); + } + + public MaterializationException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationInfo.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationInfo.java new file mode 100644 index 000000000000..f9eee18e79d7 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationInfo.java @@ -0,0 +1,100 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import static java.util.Objects.requireNonNull; + +import com.google.common.collect.ImmutableList; +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.expression.tree.Expression; +import io.confluent.ksql.execution.plan.SelectExpression; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.util.List; +import java.util.Optional; + +/** + * Pojo for passing around information about materialization of a query's state store + */ +@Immutable +public final class MaterializationInfo { + + private final String stateStoreName; + private final LogicalSchema aggregationSchema; + private final Optional havingExpression; + private final LogicalSchema tableSchema; + private final List tableSelects; + + /** + * Create instance. + * + * @param stateStoreName the name of the state store + * @param stateStoreSchema the schema of the state store + * @param havingExpression optional HAVING expression that should be apply to any store result. + * @param tableSchema the schema of the table. + * @param tableSelects SELECT expressions to convert state store schema to table schema. + * @return instance. + */ + public static MaterializationInfo of( + final String stateStoreName, + final LogicalSchema stateStoreSchema, + final Optional havingExpression, + final LogicalSchema tableSchema, + final List tableSelects + ) { + return new MaterializationInfo( + stateStoreName, + stateStoreSchema, + havingExpression, + tableSchema, + tableSelects + ); + } + + public String stateStoreName() { + return stateStoreName; + } + + public LogicalSchema aggregationSchema() { + return aggregationSchema; + } + + public Optional havingExpression() { + return havingExpression; + } + + public LogicalSchema tableSchema() { + return tableSchema; + } + + public List tableSelects() { + return tableSelects; + } + + private MaterializationInfo( + final String stateStoreName, + final LogicalSchema aggregationSchema, + final Optional havingExpression, + final LogicalSchema tableSchema, + final List tableSelects + ) { + this.stateStoreName = requireNonNull(stateStoreName, "stateStoreName"); + this.aggregationSchema = requireNonNull(aggregationSchema, "aggregationSchema"); + this.havingExpression = requireNonNull(havingExpression, "havingExpression"); + this.tableSchema = requireNonNull(tableSchema, "tableSchema"); + this.tableSelects = ImmutableList.copyOf(requireNonNull(tableSelects, "tableSelects")); + } +} + diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationProvider.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationProvider.java new file mode 100644 index 000000000000..2295ecdab9db --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationProvider.java @@ -0,0 +1,29 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import io.confluent.ksql.execution.context.QueryContext; + +public interface MaterializationProvider { + + /** + * Build a materialization in the context of a given query. + * + * @param contextStacker the query context stacker. + * @return the materialization. + */ + Materialization build(QueryContext.Stacker contextStacker); +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationTimeOutException.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationTimeOutException.java new file mode 100644 index 000000000000..0ea376a97f56 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializationTimeOutException.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +public class MaterializationTimeOutException extends MaterializationException { + + public MaterializationTimeOutException(final String message) { + super(message); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedTable.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedTable.java new file mode 100644 index 000000000000..e7ef1fd56e5a --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedTable.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import io.confluent.ksql.GenericRow; +import java.util.Optional; +import org.apache.kafka.connect.data.Struct; + +/** + * Materialization of a table with a non-windowed key + */ +public interface MaterializedTable { + + /** + * Get the value, if one exists, of the supplied {@code key}. + * + * @param key the key to look up. + * @return the value, if one is exists. + */ + Optional get(Struct key); +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedWindowedTable.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedWindowedTable.java new file mode 100644 index 000000000000..22786549deef --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/MaterializedWindowedTable.java @@ -0,0 +1,38 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import io.confluent.ksql.GenericRow; +import java.time.Instant; +import java.util.Map; +import org.apache.kafka.connect.data.Struct; + +/** + * Materialization of a table with a windowed key + */ +public interface MaterializedWindowedTable { + + /** + * Get the values in table of the supplied {@code key}, where the window start time is within + * the supplied {@code lower} and {@code upper} bounds. + * + * @param key the key to look up. + * @param lower the lower bound on the window's start time, (inclusive). + * @param upper the upper bound on the window's start time, (inclusive). + * @return the value, if one is exists. + */ + Map get(Struct key, Instant lower, Instant upper); +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/NotRunningException.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/NotRunningException.java new file mode 100644 index 000000000000..686709d73b63 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/NotRunningException.java @@ -0,0 +1,23 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +public class NotRunningException extends MaterializationException { + + public NotRunningException(final String message) { + super(message); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/Window.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Window.java new file mode 100644 index 000000000000..9be08d255f75 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/Window.java @@ -0,0 +1,86 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import java.time.Instant; +import java.util.Objects; +import java.util.Optional; + +/** + * Pojo for storing window bounds + */ +public final class Window { + + private final Instant start; + private final Optional end; + + /** + * Create instance. + * + * @param start the start of the window. + * @param end the end of a session window, otherwise {@link Optional#empty()}. + * @return the instance. + */ + public static Window of(final Instant start, final Optional end) { + return new Window(start, end); + } + + private Window(final Instant start, final Optional end) { + this.start = Objects.requireNonNull(start, "start"); + this.end = Objects.requireNonNull(end, "end"); + + if (end.map(e -> e.isBefore(start)).orElse(false)) { + throw new IllegalArgumentException("end before start." + + " start: " + start + + ", end: " + end.get() + ); + } + } + + public Instant start() { + return start; + } + + public Optional end() { + return end; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Window window = (Window) o; + return Objects.equals(start, window.start) + && Objects.equals(end, window.end); + } + + @Override + public int hashCode() { + return Objects.hash(start, end); + } + + @Override + public String toString() { + return "Window{" + + "start=" + start + + ", end=" + end + + '}'; + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsLocator.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsLocator.java new file mode 100644 index 000000000000..bf9f469f94d9 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsLocator.java @@ -0,0 +1,117 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static java.util.Objects.requireNonNull; + +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.materialization.Locator; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Optional; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.state.HostInfo; +import org.apache.kafka.streams.state.StreamsMetadata; + +/** + * Kafka Streams implementation of {@link Locator}. + */ +final class KsLocator implements Locator { + + private final String stateStoreName; + private final KafkaStreams kafkaStreams; + private final Serializer keySerializer; + private final URL localHost; + + KsLocator( + final String stateStoreName, + final KafkaStreams kafkaStreams, + final Serializer keySerializer, + final URL localHost + ) { + this.kafkaStreams = requireNonNull(kafkaStreams, "kafkaStreams"); + this.keySerializer = requireNonNull(keySerializer, "keySerializer"); + this.stateStoreName = requireNonNull(stateStoreName, "stateStoreName"); + this.localHost = requireNonNull(localHost, "localHost"); + } + + @Override + public Optional locate(final Struct key) { + final StreamsMetadata metadata = kafkaStreams + .metadataForKey(stateStoreName, key, keySerializer); + + if (metadata == StreamsMetadata.NOT_AVAILABLE) { + return Optional.empty(); + } + + final HostInfo hostInfo = metadata.hostInfo(); + return Optional.of(asNode(hostInfo)); + } + + private KsqlNode asNode(final HostInfo hostInfo) { + return new Node( + isLocalHost(hostInfo), + buildLocation(hostInfo) + ); + } + + private boolean isLocalHost(final HostInfo hostInfo) { + if (hostInfo.port() != localHost.getPort()) { + return false; + } + + return hostInfo.host().equalsIgnoreCase(localHost.getHost()) + || hostInfo.host().equalsIgnoreCase("localhost"); + } + + private URL buildLocation(final HostInfo remoteInfo) { + try { + return new URL( + localHost.getProtocol(), + remoteInfo.host(), + remoteInfo.port(), + "/" + ); + } catch (MalformedURLException e) { + throw new IllegalStateException("Failed to convert remote host info to URL." + + " remoteInfo: " + remoteInfo); + } + } + + @Immutable + private static final class Node implements KsqlNode { + + private final boolean local; + private final URL location; + + private Node(final boolean local, final URL location) { + this.local = local; + this.location = location; + } + + @Override + public boolean isLocal() { + return local; + } + + @Override + public URL location() { + return location; + } + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterialization.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterialization.java new file mode 100644 index 000000000000..2a8251b96a1b --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterialization.java @@ -0,0 +1,89 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static java.util.Objects.requireNonNull; + +import io.confluent.ksql.materialization.Locator; +import io.confluent.ksql.materialization.Materialization; +import io.confluent.ksql.materialization.MaterializedTable; +import io.confluent.ksql.materialization.MaterializedWindowedTable; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.util.Optional; + +/** + * Kafka Streams impl of {@link Materialization}. + */ +public final class KsMaterialization implements Materialization { + + private final Optional windowType; + private final KsStateStore stateStore; + private final Locator locator; + + KsMaterialization( + final Optional windowType, + final Locator locator, + final KsStateStore stateStore + ) { + this.windowType = requireNonNull(windowType, "windowType"); + this.stateStore = requireNonNull(stateStore, "stateStore"); + this.locator = requireNonNull(locator, "locator"); + } + + @Override + public LogicalSchema schema() { + // Not needed + throw new UnsupportedOperationException(); + } + + @Override + public Locator locator() { + return locator; + } + + @Override + public Optional windowType() { + return windowType; + } + + @Override + public MaterializedTable nonWindowed() { + if (windowType.isPresent()) { + throw new UnsupportedOperationException("Table has windowed key"); + } + return new KsMaterializedTable(stateStore); + } + + @Override + public MaterializedWindowedTable windowed() { + if (!windowType.isPresent()) { + throw new UnsupportedOperationException("Table has non-windowed key"); + } + + switch (windowType.get()) { + case SESSION: + return new KsMaterializedSessionTable(stateStore); + + case HOPPING: + case TUMBLING: + return new KsMaterializedWindowTable(stateStore); + + default: + throw new UnsupportedOperationException("Unknown window type: " + windowType.get()); + } + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializationFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializationFactory.java new file mode 100644 index 000000000000..7a91268a0404 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializationFactory.java @@ -0,0 +1,144 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static java.util.Objects.requireNonNull; + +import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.materialization.Locator; +import io.confluent.ksql.model.WindowType; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; + +/** + * Factory class for {@link KsMaterialization}. + */ +public final class KsMaterializationFactory { + + private final LocatorFactory locatorFactory; + private final StateStoreFactory storeFactory; + private final MaterializationFactory materializationFactory; + + public KsMaterializationFactory() { + this( + KsLocator::new, + KsStateStore::new, + KsMaterialization::new + ); + } + + @VisibleForTesting + KsMaterializationFactory( + final LocatorFactory locatorFactory, + final StateStoreFactory storeFactory, + final MaterializationFactory materializationFactory + ) { + this.locatorFactory = requireNonNull(locatorFactory, "locatorFactory"); + this.storeFactory = requireNonNull(storeFactory, "storeFactory"); + this.materializationFactory = requireNonNull(materializationFactory, "materializationFactory"); + } + + /** + * Create {@link KsMaterialization} instance. + * + * @param stateStoreName the name of the state store in the Kafka Streams instance. + * @param kafkaStreams the Kafka Streams instance. + * @param keySerializer the key serializer - used purely for location lookups. + * @param windowType the window type of the key. + * @param streamsProperties the Kafka Streams properties. + * @return the new instance if the streams props support IQ. + */ + public Optional create( + final String stateStoreName, + final KafkaStreams kafkaStreams, + final Serializer keySerializer, + final Optional windowType, + final Map streamsProperties + ) { + final Object appServer = streamsProperties.get(StreamsConfig.APPLICATION_SERVER_CONFIG); + if (appServer == null) { + return Optional.empty(); + } + + final URL localHost = buildLocalHost(appServer); + + final KsLocator locator = locatorFactory.create( + stateStoreName, + kafkaStreams, + keySerializer, + localHost + ); + + final KsStateStore stateStore = storeFactory.create( + stateStoreName, + kafkaStreams + ); + + final KsMaterialization materialization = materializationFactory.create( + windowType, + locator, + stateStore + ); + + return Optional.of(materialization); + } + + private static URL buildLocalHost(final Object appServer) { + if (!(appServer instanceof String)) { + throw new IllegalArgumentException(StreamsConfig.APPLICATION_SERVER_CONFIG + " not String"); + } + + try { + return new URL((String) appServer); + } catch (MalformedURLException e) { + throw new IllegalArgumentException(StreamsConfig.APPLICATION_SERVER_CONFIG + " malformed: " + + "'" + appServer + "'"); + } + } + + interface LocatorFactory { + + KsLocator create( + String stateStoreName, + KafkaStreams kafkaStreams, + Serializer keySerializer, + URL localHost + ); + } + + interface StateStoreFactory { + + KsStateStore create( + String stateStoreName, + KafkaStreams kafkaStreams + ); + } + + interface MaterializationFactory { + + KsMaterialization create( + Optional windowType, + Locator locator, + KsStateStore stateStore + ); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTable.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTable.java new file mode 100644 index 000000000000..bc3da2a0ad4d --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTable.java @@ -0,0 +1,96 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializedWindowedTable; +import io.confluent.ksql.materialization.Window; +import java.time.Instant; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlySessionStore; + +/** + * Kafka Streams impl of {@link MaterializedWindowedTable}. + */ +class KsMaterializedSessionTable implements MaterializedWindowedTable { + + private final KsStateStore stateStore; + + KsMaterializedSessionTable(final KsStateStore store) { + this.stateStore = Objects.requireNonNull(store, "store"); + } + + @Override + public Map get( + final Struct key, + final Instant lower, + final Instant upper + ) { + try { + final ReadOnlySessionStore store = stateStore + .store(QueryableStoreTypes.sessionStore()); + + return findSession(store, key, lower, upper); + } catch (final Exception e) { + throw new MaterializationException("Failed to get value from materialized table", e); + } + } + + private static Map findSession( + final ReadOnlySessionStore store, + final Struct key, + final Instant lower, + final Instant upper + ) { + try (KeyValueIterator, GenericRow> it = store.fetch(key)) { + + while (it.hasNext()) { + final KeyValue, GenericRow> next = it.next(); + + if (intersects(next.key.window().startTime(), lower, upper)) { + + final Window window = Window.of( + next.key.window().startTime(), + Optional.of(next.key.window().endTime()) + ); + + return ImmutableMap.of(window, next.value); + } + } + } + + return ImmutableMap.of(); + } + + private static boolean intersects( + final Instant wndStart, + final Instant lower, + final Instant upper + ) { + return wndStart.equals(lower) // lower inclusive + || wndStart.equals(upper) // upper inclusive + || (lower.isBefore(wndStart) && wndStart.isBefore(upper)); + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedTable.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedTable.java new file mode 100644 index 000000000000..ab7691b1b472 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedTable.java @@ -0,0 +1,51 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializedTable; +import java.util.Objects; +import java.util.Optional; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; + +/** + * Kafka Streams impl of {@link MaterializedTable}. + */ +class KsMaterializedTable implements MaterializedTable { + + private final KsStateStore stateStore; + + KsMaterializedTable(final KsStateStore store) { + this.stateStore = Objects.requireNonNull(store, "store"); + } + + @Override + public Optional get( + final Struct key + ) { + try { + final ReadOnlyKeyValueStore store = stateStore + .store(QueryableStoreTypes.keyValueStore()); + + return Optional.ofNullable(store.get(key)); + } catch (final Exception e) { + throw new MaterializationException("Failed to get value from materialized table", e); + } + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTable.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTable.java new file mode 100644 index 000000000000..5af214384d1d --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTable.java @@ -0,0 +1,71 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMap.Builder; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializedWindowedTable; +import io.confluent.ksql.materialization.Window; +import java.time.Instant; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; + +/** + * Kafka Streams impl of {@link MaterializedWindowedTable}. + */ +class KsMaterializedWindowTable implements MaterializedWindowedTable { + + private final KsStateStore stateStore; + + KsMaterializedWindowTable(final KsStateStore store) { + this.stateStore = Objects.requireNonNull(store, "store"); + } + + @Override + public Map get( + final Struct key, + final Instant lower, + final Instant upper + ) { + try { + final ReadOnlyWindowStore store = stateStore + .store(QueryableStoreTypes.windowStore()); + + try (WindowStoreIterator it = store.fetch(key, lower, upper)) { + + final Builder builder = ImmutableMap.builder(); + + while (it.hasNext()) { + final KeyValue next = it.next(); + final Window window = Window.of(Instant.ofEpochMilli(next.key), Optional.empty()); + builder.put(window, next.value); + } + + return builder.build(); + } + } catch (final Exception e) { + throw new MaterializationException("Failed to get value from materialized table", e); + } + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsStateStore.java b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsStateStore.java new file mode 100644 index 000000000000..96f84cef5d85 --- /dev/null +++ b/ksql-engine/src/main/java/io/confluent/ksql/materialization/ks/KsStateStore.java @@ -0,0 +1,87 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import io.confluent.ksql.materialization.NotRunningException; +import io.confluent.support.metrics.common.time.Clock; +import java.time.Duration; +import java.util.Objects; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.state.QueryableStoreType; + +/** + * Wrapper around Kafka Streams state store. + */ +class KsStateStore { + + private static final Duration DEFAULT_TIMEOUT = Duration.ofSeconds(10); + + private final String stateStoreName; + private final KafkaStreams kafkaStreams; + private final Duration timeout; + private final Clock clock; + + KsStateStore( + final String stateStoreName, + final KafkaStreams kafkaStreams + ) { + this(stateStoreName, kafkaStreams, DEFAULT_TIMEOUT, System::currentTimeMillis); + } + + @VisibleForTesting + KsStateStore( + final String stateStoreName, + final KafkaStreams kafkaStreams, + final Duration timeout, + final Clock clock + ) { + this.kafkaStreams = Objects.requireNonNull(kafkaStreams, "kafkaStreams"); + this.stateStoreName = Objects.requireNonNull(stateStoreName, "stateStoreName"); + this.timeout = Objects.requireNonNull(timeout, "timeout"); + this.clock = Objects.requireNonNull(clock, "clock"); + } + + T store(final QueryableStoreType queryableStoreType) { + awaitRunning(); + + try { + return kafkaStreams.store(stateStoreName, queryableStoreType); + } catch (final Exception e) { + throw new MaterializationException("State store currently unavailable: " + stateStoreName, e); + } + } + + private void awaitRunning() { + final long threshold = clock.currentTimeMs() + timeout.toMillis(); + while (kafkaStreams.state() == State.REBALANCING) { + if (clock.currentTimeMs() > threshold) { + throw new MaterializationTimeOutException("Store failed to rebalance within the configured " + + "timeout. timeout: " + timeout.toMillis() + "ms"); + } + + Thread.yield(); + } + + final State state = kafkaStreams.state(); + if (state != State.RUNNING) { + throw new NotRunningException("The query was not in a running state. state: " + state); + } + } +} diff --git a/ksql-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java b/ksql-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java index 033cb7f115eb..f983ad6d35ac 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/physical/PhysicalPlanBuilder.java @@ -20,8 +20,14 @@ import io.confluent.ksql.errors.ProductionExceptionHandlerUtil; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.function.FunctionRegistry; +import io.confluent.ksql.logging.processing.NoopProcessingLogContext; import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.materialization.KsqlMaterializationFactory; +import io.confluent.ksql.materialization.MaterializationInfo; +import io.confluent.ksql.materialization.MaterializationProvider; +import io.confluent.ksql.materialization.ks.KsMaterialization; +import io.confluent.ksql.materialization.ks.KsMaterializationFactory; import io.confluent.ksql.metastore.MutableMetaStore; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.metastore.model.KsqlStream; @@ -30,6 +36,7 @@ import io.confluent.ksql.metrics.ProducerCollector; import io.confluent.ksql.planner.LogicalPlanNode; import io.confluent.ksql.planner.PlanSourceExtractorVisitor; +import io.confluent.ksql.planner.plan.AggregateNode; import io.confluent.ksql.planner.plan.KsqlBareOutputNode; import io.confluent.ksql.planner.plan.KsqlStructuredDataOutputNode; import io.confluent.ksql.planner.plan.OutputNode; @@ -38,6 +45,8 @@ import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.serde.GenericKeySerDe; +import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.structured.SchemaKStream; import io.confluent.ksql.structured.SchemaKTable; @@ -60,12 +69,16 @@ import java.util.function.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.Topology; +// CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling public class PhysicalPlanBuilder { + // CHECKSTYLE_RULES.ON: ClassDataAbstractionCoupling private final StreamsBuilder builder; private final KsqlConfig ksqlConfig; @@ -77,6 +90,9 @@ public class PhysicalPlanBuilder { private final QueryIdGenerator queryIdGenerator; private final KafkaStreamsBuilder kafkaStreamsBuilder; private final Consumer queryCloseCallback; + private final KsMaterializationFactory ksMaterializationFactory; + private final KsqlMaterializationFactory ksqlMaterializationFactory; + public PhysicalPlanBuilder( final StreamsBuilder builder, @@ -103,6 +119,12 @@ public PhysicalPlanBuilder( this.queryIdGenerator = Objects.requireNonNull(queryIdGenerator, "queryIdGenerator"); this.kafkaStreamsBuilder = Objects.requireNonNull(kafkaStreamsBuilder, "kafkaStreamsBuilder"); this.queryCloseCallback = Objects.requireNonNull(queryCloseCallback, "queryCloseCallback"); + this.ksMaterializationFactory = new KsMaterializationFactory(); + this.ksqlMaterializationFactory = new KsqlMaterializationFactory( + ksqlConfig, + functionRegistry, + processingLogContext + ); } public QueryMetadata buildPhysicalPlan(final LogicalPlanNode logicalPlanNode) { @@ -214,30 +236,31 @@ private QueryMetadata buildPlanForStructuredOutputNode( final QueryId queryId, final QuerySchemas schemas ) { + final DataSourceType sourceType = (schemaKStream instanceof SchemaKTable) + ? DataSourceType.KTABLE + : DataSourceType.KSTREAM; + final DataSource sinkDataSource; - if (schemaKStream instanceof SchemaKTable) { - final SchemaKTable schemaKTable = (SchemaKTable) schemaKStream; - sinkDataSource = - new KsqlTable<>( - sqlExpression, - outputNode.getId().toString(), - outputNode.getSchema(), - outputNode.getSerdeOptions(), - schemaKTable.getKeyField(), - outputNode.getTimestampExtractionPolicy(), - outputNode.getKsqlTopic() - ); + if (sourceType == DataSourceType.KTABLE) { + sinkDataSource = new KsqlTable<>( + sqlExpression, + outputNode.getId().toString(), + outputNode.getSchema(), + outputNode.getSerdeOptions(), + schemaKStream.getKeyField(), + outputNode.getTimestampExtractionPolicy(), + outputNode.getKsqlTopic() + ); } else { - sinkDataSource = - new KsqlStream<>( - sqlExpression, - outputNode.getId().toString(), - outputNode.getSchema(), - outputNode.getSerdeOptions(), - schemaKStream.getKeyField(), - outputNode.getTimestampExtractionPolicy(), - outputNode.getKsqlTopic() - ); + sinkDataSource = new KsqlStream<>( + sqlExpression, + outputNode.getId().toString(), + outputNode.getSchema(), + outputNode.getSerdeOptions(), + schemaKStream.getKeyField(), + outputNode.getTimestampExtractionPolicy(), + outputNode.getKsqlTopic() + ); } sinkSetUp(outputNode, sinkDataSource); @@ -262,6 +285,19 @@ private QueryMetadata buildPlanForStructuredOutputNode( final PhysicalSchema querySchema = PhysicalSchema .from(outputNode.getSchema(), outputNode.getSerdeOptions()); + final Optional materializationInfo = sourceType == DataSourceType.KTABLE + ? findMaterializationInfo(outputNode) + : Optional.empty(); + + final Optional materializationBuilder = materializationInfo + .flatMap(info -> buildMaterializationProvider( + info, + streams, + querySchema, + sinkDataSource.getKsqlTopic().getKeyFormat(), + streamsProperties + )); + return new PersistentQueryMetadata( sqlExpression, streams, @@ -270,9 +306,8 @@ private QueryMetadata buildPlanForStructuredOutputNode( sinkDataSource.getName(), schemaKStream.getExecutionPlan(""), queryId, - (schemaKStream instanceof SchemaKTable) - ? DataSourceType.KTABLE - : DataSourceType.KSTREAM, + sourceType, + materializationBuilder, applicationId, sinkDataSource.getKsqlTopic(), topology, @@ -283,6 +318,53 @@ private QueryMetadata buildPlanForStructuredOutputNode( ); } + private static Optional findMaterializationInfo( + final PlanNode node + ) { + if (node instanceof AggregateNode) { + return ((AggregateNode) node).getMaterializationInfo(); + } + + return node.getSources().stream() + .map(PhysicalPlanBuilder::findMaterializationInfo) + .filter(Optional::isPresent) + .map(Optional::get) + .findFirst(); + } + + private Optional buildMaterializationProvider( + final MaterializationInfo info, + final KafkaStreams kafkaStreams, + final PhysicalSchema schema, + final KeyFormat keyFormat, + final Map streamsProperties + ) { + final Serializer keySerializer = new GenericKeySerDe().create( + keyFormat.getFormatInfo(), + schema.keySchema(), + ksqlConfig, + serviceContext.getSchemaRegistryClientFactory(), + "", + NoopProcessingLogContext.INSTANCE + ).serializer(); + + final Optional ksMaterialization = ksMaterializationFactory + .create( + info.stateStoreName(), + kafkaStreams, + keySerializer, + keyFormat.getWindowType(), + streamsProperties + ); + + return ksMaterialization.map(ksMat -> contextStacker -> ksqlMaterializationFactory + .create( + ksMat, + info, + contextStacker + )); + } + private void sinkSetUp( final KsqlStructuredDataOutputNode outputNode, final DataSource sinkDataSource diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java index 29b70aebeb00..becfc912eedf 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/AggregateNode.java @@ -35,6 +35,7 @@ import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.function.KsqlAggregateFunction; import io.confluent.ksql.function.udaf.KudafInitializer; +import io.confluent.ksql.materialization.MaterializationInfo; import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.parser.rewrite.ExpressionTreeRewriter; import io.confluent.ksql.parser.rewrite.ExpressionTreeRewriter.Context; @@ -69,6 +70,7 @@ public class AggregateNode extends PlanNode { + private static final String AGGREGATE_STATE_STORE_NAME = "Aggregate-aggregate"; private static final String INTERNAL_COLUMN_NAME_PREFIX = "KSQL_INTERNAL_COL_"; private static final String PREPARE_OP_NAME = "prepare"; @@ -87,6 +89,7 @@ public class AggregateNode extends PlanNode { private final List requiredColumns; private final List finalSelectExpressions; private final Expression havingExpressions; + private Optional materializationInfo = Optional.empty(); // CHECKSTYLE_RULES.OFF: ParameterNumberCheck public AggregateNode( @@ -160,6 +163,10 @@ public List getRequiredColumns() { return requiredColumns; } + public Optional getMaterializationInfo() { + return materializationInfo; + } + private List getFinalSelectExpressions() { final List finalSelectExpressionList = new ArrayList<>(); if (finalSelectExpressions.size() != schema.value().size()) { @@ -200,6 +207,11 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { contextStacker.push(PREPARE_OP_NAME), builder); + // This is the schema used in any repartition topic + // It contains only the fields from the source that are needed by the aggregation + // It uses internal column names, e.g. KSQL_INTERNAL_COL_0 + final LogicalSchema prepareSchema = aggregateArgExpanded.getSchema(); + final QueryContext.Stacker groupByContext = contextStacker.push(GROUP_BY_OP_NAME); final ValueFormat valueFormat = streamSourceNode @@ -209,7 +221,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { final Serde genericRowSerde = builder.buildValueSerde( valueFormat.getFormatInfo(), - PhysicalSchema.from(aggregateArgExpanded.getSchema(), SerdeOption.none()), + PhysicalSchema.from(prepareSchema, SerdeOption.none()), groupByContext.getQueryContext() ); @@ -234,8 +246,11 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { internalSchema ); - final LogicalSchema aggStageSchema = buildAggregateSchema( - aggregateArgExpanded.getSchema(), + // This is the schema of the aggregation change log topic and associated state store. + // It contains all columns from prepareSchema and columns for any aggregating functions + // It uses internal column names, e.g. KSQL_INTERNAL_COL_0 and KSQL_AGG_VARIABLE_0 + final LogicalSchema aggregationSchema = buildAggregateSchema( + prepareSchema, aggValToFunctionMap ); @@ -243,7 +258,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { final Serde aggValueGenericRowSerde = builder.buildValueSerde( valueFormat.getFormatInfo(), - PhysicalSchema.from(aggStageSchema, SerdeOption.none()), + PhysicalSchema.from(aggregationSchema, SerdeOption.none()), aggregationContext.getQueryContext() ); @@ -252,7 +267,7 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { .map(FunctionCall.class::cast) .collect(Collectors.toList()); SchemaKTable aggregated = schemaKGroupedStream.aggregate( - aggStageSchema, + aggregationSchema, initializer, requiredColumns.size(), functionsWithInternalIdentifiers, @@ -263,15 +278,29 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { aggregationContext ); - if (havingExpressions != null) { + final Optional havingExpression = Optional.ofNullable(havingExpressions) + .map(internalSchema::resolveToInternal); + + if (havingExpression.isPresent()) { aggregated = aggregated.filter( - internalSchema.resolveToInternal(havingExpressions), + havingExpression.get(), contextStacker.push(FILTER_OP_NAME), builder.getProcessingLogContext()); } + final List finalSelects = internalSchema + .updateFinalSelectExpressions(getFinalSelectExpressions()); + + materializationInfo = Optional.of(MaterializationInfo.of( + AGGREGATE_STATE_STORE_NAME, + aggregationSchema, + havingExpression, + schema, + finalSelects + )); + return aggregated.select( - internalSchema.updateFinalSelectExpressions(getFinalSelectExpressions()), + finalSelects, contextStacker.push(PROJECT_OP_NAME), builder); } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java index 74a87e12c850..6c92d4cf3e96 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/DataSourceNode.java @@ -17,6 +17,7 @@ import static java.util.Objects.requireNonNull; +import com.google.common.collect.ImmutableList; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.context.QueryContext.Stacker; @@ -117,7 +118,7 @@ public int getPartitions(final KafkaTopicClient kafkaTopicClient) { @Override public List getSources() { - return null; + return ImmutableList.of(); } @Override diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SqlPredicate.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SqlPredicate.java index 03745f56541e..234b4709508c 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SqlPredicate.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SqlPredicate.java @@ -37,7 +37,7 @@ import org.codehaus.commons.compiler.CompilerFactoryFactory; import org.codehaus.commons.compiler.IExpressionEvaluator; -class SqlPredicate { +public class SqlPredicate { private final Expression filterExpression; private final LogicalSchema schema; @@ -48,7 +48,7 @@ class SqlPredicate { private final GenericRowValueTypeEnforcer genericRowValueTypeEnforcer; private final ProcessingLogger processingLogger; - SqlPredicate( + public SqlPredicate( final Expression filterExpression, final LogicalSchema schema, final KsqlConfig ksqlConfig, @@ -108,7 +108,7 @@ private Expression rewriteFilter(final Expression expression) { } - Predicate getPredicate() { + public Predicate getPredicate() { final ExpressionMetadata expressionEvaluator = createExpressionMetadata(); return (key, row) -> { diff --git a/ksql-engine/src/main/java/io/confluent/ksql/util/PersistentQueryMetadata.java b/ksql-engine/src/main/java/io/confluent/ksql/util/PersistentQueryMetadata.java index c02a3510f540..757ce876bec0 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/util/PersistentQueryMetadata.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/util/PersistentQueryMetadata.java @@ -17,13 +17,17 @@ import static java.util.Objects.requireNonNull; +import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; +import io.confluent.ksql.materialization.Materialization; +import io.confluent.ksql.materialization.MaterializationProvider; import io.confluent.ksql.metastore.model.DataSource.DataSourceType; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.serde.Format; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import org.apache.kafka.streams.KafkaStreams; @@ -39,6 +43,7 @@ public class PersistentQueryMetadata extends QueryMetadata { private final String sinkName; private final QuerySchemas schemas; private final PhysicalSchema resultSchema; + private final Optional materializationProvider; // CHECKSTYLE_RULES.OFF: ParameterNumberCheck public PersistentQueryMetadata( @@ -50,6 +55,7 @@ public PersistentQueryMetadata( final String executionPlan, final QueryId id, final DataSourceType dataSourceType, + final Optional materializationProvider, final String queryApplicationId, final KsqlTopic resultTopic, final Topology topology, @@ -77,6 +83,8 @@ public PersistentQueryMetadata( this.sinkName = Objects.requireNonNull(sinkName, "sinkName"); this.schemas = requireNonNull(schemas, "schemas"); this.resultSchema = requireNonNull(schema, "schema"); + this.materializationProvider = + requireNonNull(materializationProvider, "materializationProvider"); } private PersistentQueryMetadata( @@ -89,6 +97,7 @@ private PersistentQueryMetadata( this.sinkName = other.sinkName; this.schemas = other.schemas; this.resultSchema = other.resultSchema; + this.materializationProvider = other.materializationProvider; } public PersistentQueryMetadata copyWith(final Consumer closeCallback) { @@ -118,4 +127,10 @@ public String getSchemasDescription() { public PhysicalSchema getPhysicalSchema() { return resultSchema; } + + public Optional getMaterialization( + final QueryContext.Stacker contextStacker + ) { + return materializationProvider.map(builder -> builder.build(contextStacker)); + } } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java b/ksql-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java index 6370b2f01ef7..bc162f368f40 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java @@ -15,6 +15,7 @@ package io.confluent.ksql.engine; +import static io.confluent.ksql.metastore.model.MetaStoreMatchers.FieldMatchers.hasFullName; import static io.confluent.ksql.util.KsqlExceptionMatcher.rawMessage; import static io.confluent.ksql.util.KsqlExceptionMatcher.statementText; import static org.hamcrest.MatcherAssert.assertThat; @@ -61,6 +62,7 @@ import io.confluent.ksql.util.MetaStoreFixture; import io.confluent.ksql.util.PersistentQueryMetadata; import io.confluent.ksql.util.QueryMetadata; +import io.confluent.ksql.util.SchemaUtil; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -142,6 +144,24 @@ public void shouldCreatePersistentQueries() { assertThat(((PersistentQueryMetadata) queries.get(1)).getSinkName(), is("FOO")); } + @Test + public void shouldNotHaveRowTimeAndRowKeyColumnsInPersistentQueryValueSchema() { + // When: + final PersistentQueryMetadata query = (PersistentQueryMetadata) KsqlEngineTestUtil.execute( + ksqlEngine, + "create table bar as select * from test2;", + KSQL_CONFIG, + Collections.emptyMap() + ).get(0); + + // Then: + assertThat(query.getLogicalSchema().value(), + not(hasItem(hasFullName(SchemaUtil.ROWTIME_NAME)))); + + assertThat(query.getLogicalSchema().value(), + not(hasItem(hasFullName(SchemaUtil.ROWKEY_NAME)))); + } + @Test public void shouldThrowOnTerminateAsNotExecutable() { // Given: diff --git a/ksql-engine/src/test/java/io/confluent/ksql/integration/TestKsqlContext.java b/ksql-engine/src/test/java/io/confluent/ksql/integration/TestKsqlContext.java index 653ebe02caba..918e1e3c77d9 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/integration/TestKsqlContext.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/integration/TestKsqlContext.java @@ -34,7 +34,7 @@ /** * Junit external resource for managing an instance of {@link KsqlContext}. */ -public final class TestKsqlContext extends ExternalResource { +public final class TestKsqlContext extends ExternalResource implements AutoCloseable { private final IntegrationTestHarness testHarness; private final Map additionalConfig; @@ -68,6 +68,19 @@ void terminateQuery(final QueryId queryId) { delegate.terminateQuery(queryId); } + public void ensureStarted() { + if (delegate != null) { + return; + } + + before(); + } + + @Override + public void close() { + after(); + } + @Override protected void before() { final KsqlConfig ksqlConfig = KsqlConfigTestUtil.create( @@ -85,6 +98,9 @@ protected void before() { @Override protected void after() { - delegate.close(); + if (delegate != null) { + delegate.close(); + delegate = null; + } } } \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationFactoryTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationFactoryTest.java new file mode 100644 index 000000000000..8353422377c8 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationFactoryTest.java @@ -0,0 +1,235 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableList; +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.execution.context.QueryContext.Stacker; +import io.confluent.ksql.execution.expression.tree.Expression; +import io.confluent.ksql.execution.plan.SelectExpression; +import io.confluent.ksql.function.FunctionRegistry; +import io.confluent.ksql.logging.processing.ProcessingLogContext; +import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.logging.processing.ProcessingLoggerFactory; +import io.confluent.ksql.materialization.KsqlMaterializationFactory.MaterializationFactory; +import io.confluent.ksql.materialization.KsqlMaterializationFactory.SqlPredicateFactory; +import io.confluent.ksql.materialization.KsqlMaterializationFactory.ValueMapperFactory; +import io.confluent.ksql.query.QueryId; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.structured.SqlPredicate; +import io.confluent.ksql.util.KsqlConfig; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.kstream.Predicate; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsqlMaterializationFactoryTest { + + private static final Expression HAVING_EXP = mock(Expression.class); + + private static final LogicalSchema AGGREGATE_SCHEMA = LogicalSchema.builder() + .keyColumn("ROWKEY", SqlTypes.STRING) + .valueColumn("KSQL_INTERNAL_COL_0", SqlTypes.DOUBLE) + .build(); + + private static final LogicalSchema TABLE_SCHEMA = LogicalSchema.builder() + .keyColumn("ROWKEY", SqlTypes.STRING) + .valueColumn("SUM", SqlTypes.DOUBLE) + .build(); + + private static final List SELECTS = ImmutableList.of( + mock(SelectExpression.class) + ); + + @Mock + private KsqlConfig ksqlConfig; + @Mock + private FunctionRegistry functionRegistry; + @Mock + private ProcessingLogContext processingLogContext; + @Mock + private Materialization materialization; + @Mock + private MaterializationInfo info; + @Mock + private SqlPredicateFactory sqlPredicateFactory; + @Mock + private ProcessingLogger filterProcessingLogger; + @Mock + private ProcessingLogger projectProcessingLogger; + @Mock + private ProcessingLoggerFactory processingLoggerFactory; + @Mock + private ValueMapperFactory valueMapperFactory; + @Mock + private SqlPredicate havingSqlPredicate; + @Mock + private Predicate havingPredicate; + @Mock + private Function valueMapper; + @Mock + private MaterializationFactory materializationFactory; + + private final Stacker contextStacker = new Stacker(new QueryId("start")); + + private KsqlMaterializationFactory factory; + + @SuppressWarnings("unchecked") + @Before + public void setUp() { + factory = new KsqlMaterializationFactory( + ksqlConfig, + functionRegistry, + processingLogContext, + sqlPredicateFactory, + valueMapperFactory, + materializationFactory + ); + + when(processingLogContext.getLoggerFactory()).thenReturn(processingLoggerFactory); + when(processingLoggerFactory.getLogger("start.filter")).thenReturn(filterProcessingLogger); + when(processingLoggerFactory.getLogger("start.project")).thenReturn(projectProcessingLogger); + + when(info.aggregationSchema()).thenReturn(AGGREGATE_SCHEMA); + when(info.tableSchema()).thenReturn(TABLE_SCHEMA); + + when(havingSqlPredicate.getPredicate()).thenReturn((Predicate) havingPredicate); + when(sqlPredicateFactory.create(any(), any(), any(), any(), any())) + .thenReturn(havingSqlPredicate); + when(valueMapperFactory.create(any(), any(), any(), any(), any())).thenReturn(valueMapper); + + when(info.havingExpression()).thenReturn(Optional.of(HAVING_EXP)); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KsqlConfig.class, ksqlConfig) + .setDefault(FunctionRegistry.class, functionRegistry) + .setDefault(ProcessingLogContext.class, processingLogContext) + .testConstructors(KsqlMaterializationFactory.class, Visibility.PACKAGE); + } + + @Test + public void shouldNotCreateSqlPredicateIfNoHavingClause() { + // Given: + when(info.havingExpression()).thenReturn(Optional.empty()); + + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(sqlPredicateFactory, never()).create(any(), any(), any(), any(), any()); + } + + @Test + public void shouldGetFilterProcessingLoggerWithCorrectParams() { + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(processingLoggerFactory).getLogger("start.filter"); + } + + @Test + public void shouldBuildHavingPredicateWithCorrectParams() { + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(sqlPredicateFactory).create( + HAVING_EXP, + AGGREGATE_SCHEMA, + ksqlConfig, + functionRegistry, + filterProcessingLogger + ); + } + + @Test + public void shouldGetProjectProcessingLoggerWithCorrectParams() { + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(processingLoggerFactory).getLogger("start.project"); + } + + @Test + public void shouldBuildSelectValueMapperWithCorrectParameters() { + // Given: + when(info.tableSelects()).thenReturn(SELECTS); + + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(valueMapperFactory).create( + SELECTS, + AGGREGATE_SCHEMA, + ksqlConfig, + functionRegistry, + projectProcessingLogger + ); + } + + @Test + public void shouldBuildMaterializationWithCorrectParams() { + // When: + factory.create(materialization, info, contextStacker); + + // Then: + verify(materializationFactory).create( + eq(materialization), + eq(havingPredicate), + any(), + eq(TABLE_SCHEMA) + ); + } + + @Test + public void shouldReturnMaterialization() { + // Given: + final KsqlMaterialization ksqlMaterialization = mock(KsqlMaterialization.class); + when(materializationFactory.create(any(), any(), any(), any())).thenReturn(ksqlMaterialization); + + // When: + final Materialization result = factory + .create(materialization, info, contextStacker); + + // Then: + assertThat(result, is(ksqlMaterialization)); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationTest.java new file mode 100644 index 000000000000..5b4795658bc7 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/KsqlMaterializationTest.java @@ -0,0 +1,346 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.KsqlMaterialization.KsqlMaterializedTable; +import io.confluent.ksql.materialization.KsqlMaterialization.KsqlMaterializedWindowedTable; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.time.Instant; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.kstream.Predicate; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsqlMaterializationTest { + + private static final Struct A_KEY = new Struct(SchemaBuilder.struct().build()); + private static final Instant AN_INSTANT = Instant.now(); + private static final Instant LATER_INSTANT = AN_INSTANT.plusSeconds(10); + private static final GenericRow A_VALUE = new GenericRow("a", "b"); + private static final GenericRow TRANSFORMED = new GenericRow("x", "y"); + private static final Window A_WINDOW = Window.of(Instant.now(), Optional.empty()); + + @Mock + private Materialization inner; + @Mock + private Predicate havingPredicate; + @Mock + private Function storeToTableTransform; + @Mock + private MaterializedTable innerNonWindowed; + @Mock + private MaterializedWindowedTable innerWindowed; + @Mock + private LogicalSchema schema; + private KsqlMaterialization materialization; + + + @Before + public void setUp() { + materialization = new KsqlMaterialization( + inner, + havingPredicate, + storeToTableTransform, + schema + ); + + when(inner.nonWindowed()).thenReturn(innerNonWindowed); + when(inner.windowed()).thenReturn(innerWindowed); + + when(innerNonWindowed.get(any())).thenReturn(Optional.of(A_VALUE)); + when(innerWindowed.get(any(), any(), any())).thenReturn(ImmutableMap.of(A_WINDOW, A_VALUE)); + + when(havingPredicate.test(any(), any())).thenReturn(true); + + when(storeToTableTransform.apply(any())).thenAnswer(inv -> inv.getArgument(0)); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(Materialization.class, inner) + .setDefault(LogicalSchema.class, schema) + .testConstructors(KsqlMaterialization.class, Visibility.PACKAGE); + } + + @Test + public void shouldReturnInnerLocator() { + // Given: + final Locator expected = mock(Locator.class); + when(inner.locator()).thenReturn(expected); + + // When: + final Locator locator = materialization.locator(); + + // Then: + assertThat(locator, is(sameInstance(expected))); + } + + @Test + public void shouldReturnInnerWindowType() { + // Given: + when(inner.windowType()).thenReturn(Optional.of(WindowType.SESSION)); + + // When: + final Optional windowType = materialization.windowType(); + + // Then: + assertThat(windowType, is(Optional.of(WindowType.SESSION))); + } + + @Test + public void shouldWrappedNonWindowed() { + // When: + final MaterializedTable table = materialization.nonWindowed(); + + // Then: + assertThat(table, is(instanceOf(KsqlMaterializedTable.class))); + } + + @Test + public void shouldWrappedWindowed() { + // When: + final MaterializedWindowedTable table = materialization.windowed(); + + // Then: + assertThat(table, is(instanceOf(KsqlMaterializedWindowedTable.class))); + } + + @Test + public void shouldCallInnerNonWindowedWithCorrectParamsOnGet() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + + // When: + table.get(A_KEY); + + // Then: + verify(innerNonWindowed).get(A_KEY); + } + + @Test + public void shouldCallInnerWindowedWithCorrectParamsOnGet() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(innerWindowed).get(A_KEY, AN_INSTANT, LATER_INSTANT); + } + + @Test + public void shouldCallHavingPredicateWithCorrectValuesOnNonWindowedGet() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + + // When: + table.get(A_KEY); + + // Then: + verify(havingPredicate).test(A_KEY, A_VALUE); + } + + @Test + public void shouldCallHavingPredicateWithCorrectValuesOnWindowedGet() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(havingPredicate).test(A_KEY, A_VALUE); + } + + @Test + public void shouldReturnEmptyIfInnerNonWindowedReturnsEmpty() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + when(innerNonWindowed.get(any())).thenReturn(Optional.empty()); + + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldReturnEmptyIfInnerWindowedReturnsEmpty() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + when(innerWindowed.get(any(), any(), any())).thenReturn(ImmutableMap.of()); + + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result.entrySet(), is(empty())); + } + + @Test + public void shouldFilterNonWindowed() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + when(havingPredicate.test(any(), any())).thenReturn(false); + + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldFilterWindowed() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + when(havingPredicate.test(any(), any())).thenReturn(false); + + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result.entrySet(), is(empty())); + } + + @Test + public void shouldTransformRowAfterFilterNonWindowed() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + + // When: + table.get(A_KEY); + + // Then: + final InOrder inOrder = inOrder(havingPredicate, storeToTableTransform); + inOrder.verify(havingPredicate).test(any(), any()); + inOrder.verify(storeToTableTransform).apply(any()); + } + + @Test + public void shouldTransformRowAfterFilterWindowed() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + final InOrder inOrder = inOrder(havingPredicate, storeToTableTransform); + inOrder.verify(havingPredicate).test(any(), any()); + inOrder.verify(storeToTableTransform).apply(any()); + } + + @Test + public void shouldCallTransformWithCorrectParamsNonWindowed() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + + // When: + table.get(A_KEY); + + // Then: + verify(storeToTableTransform).apply(A_VALUE); + } + + @Test + public void shouldCallTransformWithCorrectParamsWindowed() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(storeToTableTransform).apply(A_VALUE); + } + + @Test + public void shouldReturnTransformedFromNonWindowed() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + when(storeToTableTransform.apply(any())).thenReturn(TRANSFORMED); + + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.of(TRANSFORMED))); + } + + @Test + public void shouldReturnTransformedFromWindowed() { + // Given: + final MaterializedTable table = materialization.nonWindowed(); + when(storeToTableTransform.apply(any())).thenReturn(TRANSFORMED); + + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.of(TRANSFORMED))); + } + + @Test + public void shouldMaintainResultOrdering() { + // Given: + final MaterializedWindowedTable table = materialization.windowed(); + + final Window window1 = mock(Window.class); + final Window window2 = mock(Window.class); + final Window window3 = mock(Window.class); + when(innerWindowed.get(any(), any(), any())).thenReturn(ImmutableMap.of( + window1, new GenericRow(), + window2, new GenericRow(), + window3, new GenericRow() + )); + + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result.keySet(), contains(window1, window2, window3)); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/WindowTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/WindowTest.java new file mode 100644 index 000000000000..5324befdc093 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/WindowTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization; + +import com.google.common.testing.EqualsTester; +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import java.time.Instant; +import java.util.Optional; +import org.junit.Test; + +public class WindowTest { + + private static final Instant INSTANT = Instant.now(); + private static final Instant LATER_INSTANCE = INSTANT.plusMillis(1); + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .testStaticMethods(Window.class, Visibility.PACKAGE); + } + + @Test + public void shouldImplementEquals() { + new EqualsTester() + .addEqualityGroup( + Window.of(INSTANT, Optional.of(LATER_INSTANCE)), + Window.of(INSTANT, Optional.of(LATER_INSTANCE)) + ) + .addEqualityGroup( + Window.of(INSTANT.minusMillis(1), Optional.of(LATER_INSTANCE)) + ) + .addEqualityGroup( + Window.of(INSTANT, Optional.empty()) + ) + .testEquals(); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfEndBeforeStart() { + Window.of(LATER_INSTANCE, Optional.of(INSTANT)); + } +} diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsLocatorTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsLocatorTest.java new file mode 100644 index 000000000000..26b5f3cc4d17 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsLocatorTest.java @@ -0,0 +1,206 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.materialization.Locator.KsqlNode; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Optional; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.state.HostInfo; +import org.apache.kafka.streams.state.StreamsMetadata; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsLocatorTest { + + private static final String STORE_NAME = "someStoreName"; + private static final URL LOCAL_HOST_URL = localHost(); + private static final Struct SOME_KEY = new Struct(SchemaBuilder.struct().build()); + + @Mock + private KafkaStreams kafkaStreams; + @Mock + private Serializer keySerializer; + @Mock + private HostInfo hostInfo; + + private KsLocator locator; + + @Before + public void setUp() { + locator = new KsLocator(STORE_NAME, kafkaStreams, keySerializer, LOCAL_HOST_URL); + + givenOwnerMetadata(Optional.empty()); + + when(hostInfo.host()).thenReturn("remoteHost"); + when(hostInfo.port()).thenReturn(2345); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(URL.class, LOCAL_HOST_URL) + .setDefault(KafkaStreams.class, kafkaStreams) + .setDefault(Serializer.class, keySerializer) + .testConstructors(KsLocator.class, Visibility.PACKAGE); + } + + @Test + public void shouldRequestMetadata() { + // When: + locator.locate(SOME_KEY); + + // Then: + verify(kafkaStreams).metadataForKey(STORE_NAME, SOME_KEY, keySerializer); + } + + @Test + public void shouldReturnEmptyIfOwnerNotKnown() { + // Given: + givenOwnerMetadata(Optional.empty()); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldReturnOwnerIfKnown() { + // Given: + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + final Optional url = result.map(KsqlNode::location); + assertThat(url.map(URL::getProtocol), is(Optional.of(LOCAL_HOST_URL.getProtocol()))); + assertThat(url.map(URL::getHost), is(Optional.of(hostInfo.host()))); + assertThat(url.map(URL::getPort), is(Optional.of(hostInfo.port()))); + assertThat(url.map(URL::getPath), is(Optional.of("/"))); + } + + @Test + public void shouldReturnLocalOwnerIfSameAsSuppliedLocalHost() { + // Given: + when(hostInfo.host()).thenReturn(LOCAL_HOST_URL.getHost()); + when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result.map(KsqlNode::isLocal), is(Optional.of(true))); + } + + @Test + public void shouldReturnLocalOwnerIfExplicitlyLocalHostOnSamePortAsSuppliedLocalHost() { + // Given: + when(hostInfo.host()).thenReturn("LocalHOST"); + when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result.map(KsqlNode::isLocal), is(Optional.of(true))); + } + + @Test + public void shouldReturnRemoteOwnerForDifferentHost() { + // Given: + when(hostInfo.host()).thenReturn("different"); + when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort()); + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + } + + @Test + public void shouldReturnRemoteOwnerForDifferentPort() { + // Given: + when(hostInfo.host()).thenReturn(LOCAL_HOST_URL.getHost()); + when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort() + 1); + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + } + + @Test + public void shouldReturnRemoteOwnerForDifferentPortOnLocalHost() { + // Given: + when(hostInfo.host()).thenReturn("LOCALhost"); + when(hostInfo.port()).thenReturn(LOCAL_HOST_URL.getPort() + 1); + givenOwnerMetadata(Optional.of(hostInfo)); + + // When: + final Optional result = locator.locate(SOME_KEY); + + // Then: + assertThat(result.map(KsqlNode::isLocal), is(Optional.of(false))); + } + + @SuppressWarnings("unchecked") + private void givenOwnerMetadata(final Optional hostInfo) { + final StreamsMetadata metadata = hostInfo + .map(hi -> { + final StreamsMetadata md = mock(StreamsMetadata.class); + when(md.hostInfo()).thenReturn(hostInfo.get()); + return md; + }) + .orElse(StreamsMetadata.NOT_AVAILABLE); + + when(kafkaStreams.metadataForKey(any(), any(), any(Serializer.class))) + .thenReturn(metadata); + } + + private static URL localHost() { + try { + return new URL("http://somehost:1234"); + } catch (MalformedURLException e) { + throw new AssertionError("Failed to build URL", e); + } + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFactoryTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFactoryTest.java new file mode 100644 index 000000000000..edb12bfb1325 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFactoryTest.java @@ -0,0 +1,168 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.materialization.ks.KsMaterializationFactory.LocatorFactory; +import io.confluent.ksql.materialization.ks.KsMaterializationFactory.MaterializationFactory; +import io.confluent.ksql.materialization.ks.KsMaterializationFactory.StateStoreFactory; +import io.confluent.ksql.model.WindowType; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsMaterializationFactoryTest { + + private static final String STORE_NAME = "someStore"; + private static final URL DEFAULT_APP_SERVER = buildDefaultAppServer(); + + @Mock + private KafkaStreams kafkaStreams; + @Mock + private Serializer keySerializer; + + @Mock + private LocatorFactory locatorFactory; + @Mock + private KsLocator locator; + @Mock + private StateStoreFactory storeFactory; + @Mock + private KsStateStore stateStore; + @Mock + private MaterializationFactory materializationFactory; + @Mock + private KsMaterialization materialization; + private KsMaterializationFactory factory; + private final Map streamsProperties = new HashMap<>(); + + @Before + public void setUp() { + factory = new KsMaterializationFactory( + locatorFactory, + storeFactory, + materializationFactory + ); + + when(locatorFactory.create(any(), any(), any(), any())).thenReturn(locator); + when(storeFactory.create(any(), any())).thenReturn(stateStore); + when(materializationFactory.create(any(), any(), any())).thenReturn(materialization); + + streamsProperties.clear(); + streamsProperties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, DEFAULT_APP_SERVER.toString()); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(LocatorFactory.class, locatorFactory) + .setDefault(StateStoreFactory.class, storeFactory) + .setDefault(MaterializationFactory.class, materializationFactory) + .testConstructors(KsMaterializationFactory.class, Visibility.PACKAGE); + } + + @Test + public void shouldReturnEmptyIfAppServerNotConfigured() { + // Given: + streamsProperties.remove(StreamsConfig.APPLICATION_SERVER_CONFIG); + + // When: + final Optional result = factory + .create(STORE_NAME, kafkaStreams, keySerializer, Optional.empty(), streamsProperties); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldBuildLocatorWithCorrectParams() { + // When: + factory.create(STORE_NAME, kafkaStreams, keySerializer, Optional.empty(), streamsProperties); + + // Then: + verify(locatorFactory).create( + STORE_NAME, + kafkaStreams, + keySerializer, + DEFAULT_APP_SERVER + ); + } + + @Test + public void shouldBuildStateStoreWithCorrectParams() { + // When: + factory.create(STORE_NAME, kafkaStreams, keySerializer, Optional.empty(), streamsProperties); + + // Then: + verify(storeFactory).create( + STORE_NAME, + kafkaStreams + ); + } + + @Test + public void shouldBuildMaterializationWithCorrectParams() { + // Given: + final Optional windowType = Optional.of(WindowType.SESSION); + + // When: + factory.create(STORE_NAME, kafkaStreams, keySerializer, windowType, streamsProperties); + + // Then: + verify(materializationFactory).create( + windowType, + locator, + stateStore + ); + } + + @Test + public void shouldReturnMaterialization() { + // When: + final Optional result = factory + .create(STORE_NAME, kafkaStreams, keySerializer, Optional.empty(), streamsProperties); + + // Then: + assertThat(result, is(Optional.of(materialization))); + } + + private static URL buildDefaultAppServer() { + try { + return new URL("https://someHost:9876"); + } catch (final MalformedURLException e) { + throw new AssertionError("Failed to build app server URL"); + } + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java new file mode 100644 index 000000000000..c604775a06fe --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationFunctionalTest.java @@ -0,0 +1,593 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static io.confluent.ksql.serde.Format.JSON; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.integration.IntegrationTestHarness; +import io.confluent.ksql.integration.Retry; +import io.confluent.ksql.integration.TestKsqlContext; +import io.confluent.ksql.materialization.Materialization; +import io.confluent.ksql.materialization.MaterializedTable; +import io.confluent.ksql.materialization.MaterializedWindowedTable; +import io.confluent.ksql.materialization.Window; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.query.QueryId; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlType; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.Format; +import io.confluent.ksql.serde.SerdeOption; +import io.confluent.ksql.test.util.KsqlIdentifierTestUtil; +import io.confluent.ksql.util.PersistentQueryMetadata; +import io.confluent.ksql.util.QueryMetadata; +import io.confluent.ksql.util.SchemaUtil; +import io.confluent.ksql.util.UserDataProvider; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import kafka.zookeeper.ZooKeeperClientException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.WindowedSerdes; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.RuleChain; +import org.junit.rules.Timeout; + +@SuppressWarnings("OptionalGetWithoutIsPresent") +@Category({IntegrationTest.class}) +public class KsMaterializationFunctionalTest { + + private static final String USERS_TOPIC = "users_topic"; + private static final String USER_TABLE = "users_table"; + private static final String USER_STREAM = "users_stream"; + + private static final Format VALUE_FORMAT = JSON; + private static final UserDataProvider USER_DATA_PROVIDER = new UserDataProvider(); + + private static final Duration WINDOW_SIZE = Duration.ofSeconds(5); + + private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); + + private static final Deserializer> TIME_WINDOWED_DESERIALIZER = + WindowedSerdes + .timeWindowedSerdeFrom(String.class, WINDOW_SIZE.toMillis()) + .deserializer(); + + private static final Deserializer> SESSION_WINDOWED_DESERIALIZER = + WindowedSerdes + .sessionWindowedSerdeFrom(String.class) + .deserializer(); + + private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + + @ClassRule + public static final RuleChain CLUSTER_WITH_RETRY = RuleChain + .outerRule(Retry.of(3, ZooKeeperClientException.class, 3, TimeUnit.SECONDS)) + .around(TEST_HARNESS); + + @Rule + public final TestKsqlContext ksqlContext = TEST_HARNESS.ksqlContextBuilder() + .withAdditionalConfig(StreamsConfig.APPLICATION_SERVER_CONFIG, "https://localhost:34") + .build(); + + @Rule + public final Timeout timeout = Timeout.seconds(120); + + private final List toClose = new ArrayList<>(); + + private String output; + private final QueryContext.Stacker contextStacker = + new QueryContext.Stacker(new QueryId("static")); + + @BeforeClass + public static void classSetUp() { + TEST_HARNESS.ensureTopics(USERS_TOPIC); + + TEST_HARNESS.produceRows( + USERS_TOPIC, + USER_DATA_PROVIDER, + VALUE_FORMAT + ); + } + + @Before + public void setUp() { + output = KsqlIdentifierTestUtil.uniqueIdentifierName(); + + toClose.clear(); + + initializeKsql(ksqlContext); + } + + @After + public void after() { + toClose.forEach(QueryMetadata::close); + } + + @Test + public void shouldReturnEmptyIfNotMaterializedTable() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT * FROM " + USER_TABLE + ";" + ); + + // When: + final Optional result = query.getMaterialization(contextStacker); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldReturnEmptyIfNotMaterializedStream() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE STREAM " + output + " AS" + + " SELECT * FROM " + USER_STREAM + ";" + ); + + // When: + final Optional result = query.getMaterialization(contextStacker); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldReturnEmptyIfAppServerNotConfigured() { + // Given: + try (TestKsqlContext ksqlNoAppServer = TEST_HARNESS.ksqlContextBuilder().build()) { + initializeKsql(ksqlNoAppServer); + + PersistentQueryMetadata query = executeQuery( + ksqlNoAppServer, + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_TABLE + + " GROUP BY USERID;" + ); + + // When: + final Optional result = query.getMaterialization(contextStacker); + + // Then: + assertThat(result, is(Optional.empty())); + } + } + + @Test + public void shouldQueryMaterializedTableForAggregatedTable() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) FROM " + USER_TABLE + + " GROUP BY USERID;" + ); + + final Map rows = waitForTableRows( + STRING_DESERIALIZER, + physicalSchema("KSQL_COL_0", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.empty())); + + final MaterializedTable table = materialization.nonWindowed(); + + rows.forEach((rowKey, value) -> { + final Struct key = asKeyStruct(rowKey, query.getPhysicalSchema()); + assertThat("expected key", table.get(key), is(Optional.of(value))); + }); + + final Struct key = asKeyStruct("Won't find me", query.getPhysicalSchema()); + assertThat("unknown key", table.get(key), is(Optional.empty())); + } + + @Test + public void shouldQueryMaterializedTableForAggregatedStream() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_STREAM + + " GROUP BY USERID;" + ); + + final Map rows = waitForTableRows( + STRING_DESERIALIZER, + physicalSchema("COUNT", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.empty())); + + final MaterializedTable table = materialization.nonWindowed(); + + rows.forEach((rowKey, value) -> { + final Struct key = asKeyStruct(rowKey, query.getPhysicalSchema()); + assertThat("expected key", table.get(key), is(Optional.of(value))); + }); + + final Struct key = asKeyStruct("Won't find me", query.getPhysicalSchema()); + assertThat("unknown key", table.get(key), is(Optional.empty())); + } + + @Test + public void shouldQueryMaterializedTableForTumblingWindowed() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_STREAM + + " WINDOW TUMBLING (SIZE " + WINDOW_SIZE.getSeconds() + " SECONDS)" + + " GROUP BY USERID;" + ); + + final Map, GenericRow> rows = waitForTableRows( + TIME_WINDOWED_DESERIALIZER, + physicalSchema("COUNT", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.of(WindowType.TUMBLING))); + + final MaterializedWindowedTable table = materialization.windowed(); + + rows.forEach((k, v) -> { + final Window w = Window.of(k.window().startTime(), Optional.empty()); + final Struct key = asKeyStruct(k.key(), query.getPhysicalSchema()); + + assertThat( + "at exact window start", + table.get(key, w.start(), w.start()), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "range including window start", + table.get(key, w.start().minusMillis(1), w.start().plusMillis(1)), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "past start", + table.get(key, w.start().plusMillis(1), w.start().plusMillis(1)).keySet(), + is(empty()) + ); + }); + } + + @Test + public void shouldQueryMaterializedTableForHoppingWindowed() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_STREAM + + " WINDOW HOPPING (SIZE " + WINDOW_SIZE.getSeconds() + " SECONDS," + + " ADVANCE BY " + WINDOW_SIZE.getSeconds() + " SECONDS)" + + " GROUP BY USERID;" + ); + + final Map, GenericRow> rows = waitForTableRows( + TIME_WINDOWED_DESERIALIZER, + physicalSchema("COUNT", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.of(WindowType.HOPPING))); + + final MaterializedWindowedTable table = materialization.windowed(); + + rows.forEach((k, v) -> { + final Window w = Window.of(k.window().startTime(), Optional.empty()); + final Struct key = asKeyStruct(k.key(), query.getPhysicalSchema()); + + assertThat( + "at exact window start", + table.get(key, w.start(), w.start()), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "range including window start", + table.get(key, w.start().minusMillis(1), w.start().plusMillis(1)), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "past start", + table.get(key, w.start().plusMillis(1), w.start().plusMillis(1)).keySet(), + is(empty()) + ); + }); + } + + @Test + public void shouldQueryMaterializedTableForSessionWindowed() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_STREAM + + " WINDOW SESSION (" + WINDOW_SIZE.getSeconds() + " SECONDS)" + + " GROUP BY USERID;" + ); + + final Map, GenericRow> rows = waitForTableRows( + SESSION_WINDOWED_DESERIALIZER, + physicalSchema("COUNT", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.of(WindowType.SESSION))); + + final MaterializedWindowedTable table = materialization.windowed(); + + rows.forEach((k, v) -> { + final Window w = Window.of(k.window().startTime(), Optional.of(k.window().endTime())); + final Struct key = asKeyStruct(k.key(), query.getPhysicalSchema()); + + assertThat( + "at exact window start", + table.get(key, w.start(), w.start()), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "range including window start", + table.get(key, w.start().minusMillis(1), w.start().plusMillis(1)), + is(ImmutableMap.of(w, v)) + ); + + assertThat( + "past start", + table.get(key, w.start().plusMillis(1), w.start().plusMillis(1)).keySet(), + is(empty()) + ); + }); + } + + @Test + public void shouldQueryMaterializedTableWithKeyFieldsInProjection() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT USERID, COUNT(*), USERID AS USERID_2 FROM " + USER_TABLE + + " GROUP BY USERID;" + ); + + final Map rows = waitForTableRows( + STRING_DESERIALIZER, + physicalSchema( + "USERID", SqlTypes.STRING, + "KSQL_COL_1", SqlTypes.BIGINT, + "USERID_2", SqlTypes.STRING + ) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.empty())); + + final MaterializedTable table = materialization.nonWindowed(); + + rows.forEach((rowKey, value) -> { + final Struct key = asKeyStruct(rowKey, query.getPhysicalSchema()); + assertThat(table.get(key), is(Optional.of(value))); + }); + } + + @Test + public void shouldQueryMaterializedTableWitMultipleAggregationColumns() { + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(1) AS COUNT, SUM(REGISTERTIME) AS SUM FROM " + USER_TABLE + + " GROUP BY USERID;" + ); + + final Map rows = waitForTableRows( + STRING_DESERIALIZER, + physicalSchema( + "COUNT", SqlTypes.BIGINT, + "SUM", SqlTypes.BIGINT + ) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + assertThat(materialization.windowType(), is(Optional.empty())); + + final MaterializedTable table = materialization.nonWindowed(); + + rows.forEach((rowKey, value) -> { + final Struct key = asKeyStruct(rowKey, query.getPhysicalSchema()); + assertThat(table.get(key), is(Optional.of(value))); + }); + } + + @Test + public void shouldIgnoreHavingClause() { + // Note: HAVING clause are handled centrally by KsqlMaterialization + + // Given: + PersistentQueryMetadata query = executeQuery( + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(*) AS COUNT FROM " + USER_TABLE + + " GROUP BY USERID" + + " HAVING SUM(REGISTERTIME) > 2;" + ); + + final Map rows = waitForTableRows( + STRING_DESERIALIZER, + physicalSchema("COUNT", SqlTypes.BIGINT) + ); + + // When: + final Materialization materialization = query.getMaterialization(contextStacker).get(); + + // Then: + final MaterializedTable table = materialization.nonWindowed(); + + rows.forEach((rowKey, value) -> { + final Struct key = asKeyStruct(rowKey, query.getPhysicalSchema()); + assertThat(table.get(key), is(Optional.ofNullable(value))); + }); + } + + private Map waitForTableRows( + final Deserializer keyDeserializer, + final PhysicalSchema aggregateSchema + ) { + return TEST_HARNESS.verifyAvailableUniqueRows( + output.toUpperCase(), + USER_DATA_PROVIDER.data().size(), + VALUE_FORMAT, + aggregateSchema, + keyDeserializer + ); + } + + private PersistentQueryMetadata executeQuery(final String statement) { + return executeQuery(ksqlContext, statement); + } + + private PersistentQueryMetadata executeQuery( + final TestKsqlContext ksqlContext, + final String statement + ) { + final List queries = ksqlContext.sql(statement); + + assertThat(queries, hasSize(1)); + assertThat(queries.get(0), instanceOf(PersistentQueryMetadata.class)); + + final PersistentQueryMetadata query = (PersistentQueryMetadata) queries.get(0); + + toClose.add(query); + + return query; + } + + private static Struct asKeyStruct(final String rowKey, final PhysicalSchema physicalSchema) { + final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); + key.put(SchemaUtil.ROWKEY_NAME, rowKey); + return key; + } + + @SuppressWarnings("SameParameterValue") + private static PhysicalSchema physicalSchema( + final String columnName0, final SqlType columnType0 + ) { + return PhysicalSchema.from( + LogicalSchema.builder() + .valueColumn(columnName0, columnType0) + .build(), + SerdeOption.none() + ); + } + + @SuppressWarnings("SameParameterValue") + private static PhysicalSchema physicalSchema( + final String columnName0, final SqlType columnType0, + final String columnName1, final SqlType columnType1 + ) { + return PhysicalSchema.from( + LogicalSchema.builder() + .valueColumn(columnName0, columnType0) + .valueColumn(columnName1, columnType1) + .build(), + SerdeOption.none() + ); + } + + @SuppressWarnings("SameParameterValue") + private static PhysicalSchema physicalSchema( + final String columnName0, final SqlType columnType0, + final String columnName1, final SqlType columnType1, + final String columnName2, final SqlType columnType2 + ) { + return PhysicalSchema.from( + LogicalSchema.builder() + .valueColumn(columnName0, columnType0) + .valueColumn(columnName1, columnType1) + .valueColumn(columnName2, columnType2) + .build(), + SerdeOption.none() + ); + } + + private static void initializeKsql(final TestKsqlContext ksqlContext) { + ksqlContext.ensureStarted(); + + ksqlContext.sql("CREATE TABLE " + USER_TABLE + " " + + USER_DATA_PROVIDER.ksqlSchemaString() + + " WITH (" + + " kafka_topic='" + USERS_TOPIC + "', " + + " value_format='" + VALUE_FORMAT + "', " + + " key = '" + USER_DATA_PROVIDER.key() + "'" + + ");" + ); + + ksqlContext.sql("CREATE STREAM " + USER_STREAM + " " + + USER_DATA_PROVIDER.ksqlSchemaString() + + " WITH (" + + " kafka_topic='" + USERS_TOPIC + "', " + + " value_format='" + VALUE_FORMAT + "', " + + " key = '" + USER_DATA_PROVIDER.key() + "'" + + ");" + ); + } +} + diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationTest.java new file mode 100644 index 000000000000..f66f5714a33b --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializationTest.java @@ -0,0 +1,140 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.sameInstance; + +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.materialization.Locator; +import io.confluent.ksql.materialization.MaterializedTable; +import io.confluent.ksql.materialization.MaterializedWindowedTable; +import io.confluent.ksql.model.WindowType; +import java.util.Optional; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsMaterializationTest { + + @Mock + private Locator locator; + @Mock + private KsStateStore stateStore; + private KsMaterialization materialization; + + @Before + public void setUp() { + givenWindowType(Optional.empty()); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KsStateStore.class, stateStore) + .testConstructors(KsMaterialization.class, Visibility.PACKAGE); + } + + @Test + public void shouldReturnLocator() { + assertThat(materialization.locator(), is(sameInstance(locator))); + } + + @Test + public void shouldReturnWindowType() { + // Given: + givenWindowType(Optional.of(WindowType.TUMBLING)); + + // Then: + assertThat(materialization.windowType(), is(Optional.of(WindowType.TUMBLING))); + } + + @Test(expected = UnsupportedOperationException.class) + public void shouldThrowOnWindowedIfNotWindowed() { + // Given: + givenWindowType(Optional.empty()); + + // When: + materialization.windowed(); + } + + @Test(expected = UnsupportedOperationException.class) + public void shouldThrowOnNonWindowedIfWindowed() { + // Given: + givenWindowType(Optional.of(WindowType.SESSION)); + + // When: + materialization.nonWindowed(); + } + + @Test + public void shouldReturnNonWindowed() { + // Given: + givenWindowType(Optional.empty()); + + // When: + final MaterializedTable table = materialization.nonWindowed(); + + // Then: + assertThat(table, is(instanceOf(KsMaterializedTable.class))); + } + + @Test + public void shouldReturnWindowedForSession() { + // Given: + givenWindowType(Optional.of(WindowType.SESSION)); + + // When: + final MaterializedWindowedTable table = materialization.windowed(); + + // Then: + assertThat(table, is(instanceOf(KsMaterializedSessionTable.class))); + } + + @Test + public void shouldReturnWindowedForTumbling() { + // Given: + givenWindowType(Optional.of(WindowType.TUMBLING)); + + // When: + final MaterializedWindowedTable table = materialization.windowed(); + + // Then: + assertThat(table, is(instanceOf(KsMaterializedWindowTable.class))); + } + + @Test + public void shouldReturnWindowedForHopping() { + // Given: + givenWindowType(Optional.of(WindowType.HOPPING)); + + // When: + final MaterializedWindowedTable table = materialization.windowed(); + + // Then: + assertThat(table, is(instanceOf(KsMaterializedWindowTable.class))); + } + + private void givenWindowType(final Optional windowType) { + materialization = new KsMaterialization(windowType, locator, stateStore); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTableTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTableTest.java new file mode 100644 index 000000000000..24811b7faaf7 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedSessionTableTest.java @@ -0,0 +1,241 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import io.confluent.ksql.materialization.Window; +import java.time.Instant; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.QueryableStoreTypes.SessionStoreType; +import org.apache.kafka.streams.state.ReadOnlySessionStore; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsMaterializedSessionTableTest { + + private static final Struct A_KEY = new Struct(SchemaBuilder.struct().build()); + private static final Instant LOWER_INSTANT = Instant.now(); + private static final Instant UPPER_INSTANT = LOWER_INSTANT.plusSeconds(10); + private static final GenericRow A_VALUE = new GenericRow("c0l"); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Mock + private KsStateStore stateStore; + @Mock + private ReadOnlySessionStore sessionStore; + @Mock + private KeyValueIterator, GenericRow> fetchIterator; + private KsMaterializedSessionTable table; + + @Before + public void setUp() { + table = new KsMaterializedSessionTable(stateStore); + + when(stateStore.store(any())).thenReturn(sessionStore); + + when(sessionStore.fetch(any())).thenReturn(fetchIterator); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KsStateStore.class, stateStore) + .testConstructors(KsMaterializedSessionTable.class, Visibility.PACKAGE); + } + + @Test + public void shouldThrowIfGettingStateStoreFails() { + // Given: + when(stateStore.store(any())).thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + } + + @Test + public void shouldThrowIfStoreFetchFails() { + // Given: + when(sessionStore.fetch(any())).thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldGetStoreWithCorrectParams() { + // When: + table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + verify(stateStore).store(any(SessionStoreType.class)); + } + + @Test + public void shouldFetchWithCorrectParams() { + // When: + table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + verify(sessionStore).fetch(A_KEY); + } + + @Test + public void shouldCloseIterator() { + // When: + table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + verify(fetchIterator).close(); + } + + @Test + public void shouldReturnEmptyIfKeyNotPresent() { + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result.keySet(), is(empty())); + } + + @Test + public void shouldIgnoreSessionsThatFinishBeforeLowerBound() { + // Given: + givenSingleSession(LOWER_INSTANT.minusMillis(1), LOWER_INSTANT.minusMillis(1)); + + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result.keySet(), is(empty())); + } + + @Test + public void shouldIgnoreSessionsThatStartAfterUpperBound() { + // Given: + givenSingleSession(UPPER_INSTANT.plusMillis(1), UPPER_INSTANT.plusMillis(1)); + + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result.keySet(), is(empty())); + } + + @Test + public void shouldReturnValueIfSessionStartsAtLowerBound() { + // Given: + givenSingleSession(LOWER_INSTANT, LOWER_INSTANT.plusMillis(1)); + + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result, is(ImmutableMap.of( + Window.of(LOWER_INSTANT, Optional.of(LOWER_INSTANT.plusMillis(1))), + A_VALUE + ))); + } + + @Test + public void shouldReturnValueIfSessionStartsAtUpperBound() { + // Given: + givenSingleSession(UPPER_INSTANT, UPPER_INSTANT.plusMillis(1)); + + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result, is(ImmutableMap.of( + Window.of(UPPER_INSTANT, Optional.of(UPPER_INSTANT.plusMillis(1))), + A_VALUE + ))); + } + + @Test + public void shouldReturnValueIfSessionStartsBetweenBounds() { + // Given: + givenSingleSession(LOWER_INSTANT.plusMillis(1), UPPER_INSTANT.plusMillis(5)); + + // When: + final Map result = table.get(A_KEY, LOWER_INSTANT, UPPER_INSTANT); + + // Then: + assertThat(result, is(ImmutableMap.of( + Window.of(LOWER_INSTANT.plusMillis(1), Optional.of(UPPER_INSTANT.plusMillis(5))), + A_VALUE + ))); + } + + private void givenSingleSession( + final Instant start, + final Instant end + ) { + final KeyValue, GenericRow> kv = new KeyValue<>( + new Windowed<>( + A_KEY, + new SessionWindow(start.toEpochMilli(), end.toEpochMilli()) + ), + A_VALUE + ); + + when(fetchIterator.hasNext()) + .thenReturn(true) + .thenReturn(false); + + when(fetchIterator.next()) + .thenReturn(kv) + .thenThrow(new AssertionError()); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedTableTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedTableTest.java new file mode 100644 index 000000000000..0684aafd165d --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedTableTest.java @@ -0,0 +1,140 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import java.util.Optional; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.state.QueryableStoreTypes.KeyValueStoreType; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsMaterializedTableTest { + + private static final Struct A_KEY = new Struct(SchemaBuilder.struct().build()); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Mock + private KsStateStore stateStore; + @Mock + private ReadOnlyKeyValueStore tableStore; + + private KsMaterializedTable table; + + @Before + public void setUp() { + table = new KsMaterializedTable(stateStore); + + when(stateStore.store(any())).thenReturn(tableStore); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KsStateStore.class, stateStore) + .testConstructors(KsMaterializedTable.class, Visibility.PACKAGE); + } + + @Test + public void shouldThrowIfGettingStateStoreFails() { + // Given: + when(stateStore.store(any())).thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY); + } + + @Test + public void shouldThrowIfStoreGetFails() { + // Given: + when(tableStore.get(any())).thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldGetStoreWithCorrectParams() { + // When: + table.get(A_KEY); + + // Then: + verify(stateStore).store(any(KeyValueStoreType.class)); + } + + @Test + public void shouldGetWithCorrectParams() { + // When: + table.get(A_KEY); + + // Then: + verify(tableStore).get(A_KEY); + } + + @Test + public void shouldReturnEmptyIfKeyNotPresent() { + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.empty())); + } + + @Test + public void shouldReturnValueIfKeyPresent() { + // Given: + final GenericRow value = new GenericRow("col0"); + when(tableStore.get(any())).thenReturn(value); + + // When: + final Optional result = table.get(A_KEY); + + // Then: + assertThat(result, is(Optional.of(value))); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTableTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTableTest.java new file mode 100644 index 000000000000..6753f4a84354 --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsMaterializedWindowTableTest.java @@ -0,0 +1,206 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import io.confluent.ksql.materialization.Window; +import java.time.Instant; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.QueryableStoreTypes.WindowStoreType; +import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsMaterializedWindowTableTest { + + private static final Struct A_KEY = new Struct(SchemaBuilder.struct().build()); + private static final Instant AN_INSTANT = Instant.now(); + private static final Instant LATER_INSTANT = AN_INSTANT.plusSeconds(10); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Mock + private KsStateStore stateStore; + @Mock + private ReadOnlyWindowStore tableStore; + @Mock + private WindowStoreIterator fetchIterator; + + private KsMaterializedWindowTable table; + + @Before + public void setUp() { + table = new KsMaterializedWindowTable(stateStore); + + when(stateStore.store(any())).thenReturn(tableStore); + when(tableStore.fetch(any(), any(), any())).thenReturn(fetchIterator); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KsStateStore.class, stateStore) + .testConstructors(KsMaterializedWindowTable.class, Visibility.PACKAGE); + } + + @Test + public void shouldThrowIfGettingStateStoreFails() { + // Given: + when(stateStore.store(any())).thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + } + + @Test + public void shouldThrowIfStoreFetchFails() { + // Given: + when(tableStore.fetch(any(), any(), any())) + .thenThrow(new MaterializationTimeOutException("Boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("Failed to get value from materialized table"); + expectedException.expectCause(instanceOf(MaterializationTimeOutException.class)); + + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldGetStoreWithCorrectParams() { + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(stateStore).store(any(WindowStoreType.class)); + } + + @Test + public void shouldFetchWithCorrectParams() { + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(tableStore).fetch(A_KEY, AN_INSTANT, LATER_INSTANT); + } + + @Test + public void shouldCloseIterator() { + // When: + table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + verify(fetchIterator).close(); + } + + @Test + public void shouldReturnEmptyIfKeyNotPresent() { + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result.keySet(), is(empty())); + } + + @Test + public void shouldReturnValueIfKeyPresent() { + // Given: + final GenericRow value1 = new GenericRow("col0"); + final GenericRow value2 = new GenericRow("col1"); + + when(fetchIterator.hasNext()) + .thenReturn(true) + .thenReturn(true) + .thenReturn(false); + + when(fetchIterator.next()) + .thenReturn(new KeyValue<>(1L, value1)) + .thenReturn(new KeyValue<>(2L, value2)) + .thenThrow(new AssertionError()); + + when(tableStore.fetch(any(), any(), any())).thenReturn(fetchIterator); + + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result, is(ImmutableMap.of( + Window.of(Instant.ofEpochMilli(1), Optional.empty()), value1, + Window.of(Instant.ofEpochMilli(2), Optional.empty()), value2 + ))); + } + + @Test + public void shouldMaintainResultOrder() { + // Given: + when(fetchIterator.hasNext()) + .thenReturn(true) + .thenReturn(true) + .thenReturn(true) + .thenReturn(false); + + when(fetchIterator.next()) + .thenReturn(new KeyValue<>(1L, new GenericRow())) + .thenReturn(new KeyValue<>(3L, new GenericRow())) + .thenReturn(new KeyValue<>(2L, new GenericRow())) + .thenThrow(new AssertionError()); + + when(tableStore.fetch(any(), any(), any())).thenReturn(fetchIterator); + + // When: + final Map result = table.get(A_KEY, AN_INSTANT, LATER_INSTANT); + + // Then: + assertThat(result.keySet(), contains( + Window.of(Instant.ofEpochMilli(1), Optional.empty()), + Window.of(Instant.ofEpochMilli(3), Optional.empty()), + Window.of(Instant.ofEpochMilli(2), Optional.empty()) + )); + } +} \ No newline at end of file diff --git a/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsStateStoreTest.java b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsStateStoreTest.java new file mode 100644 index 000000000000..1a1a3b60241b --- /dev/null +++ b/ksql-engine/src/test/java/io/confluent/ksql/materialization/ks/KsStateStoreTest.java @@ -0,0 +1,222 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.materialization.ks; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.testing.NullPointerTester; +import com.google.common.testing.NullPointerTester.Visibility; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.confluent.ksql.materialization.MaterializationException; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import io.confluent.ksql.materialization.NotRunningException; +import io.confluent.support.metrics.common.time.Clock; +import java.time.Duration; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.state.QueryableStoreType; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlySessionStore; +import org.apache.kafka.streams.state.ReadOnlyWindowStore; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +@SuppressFBWarnings("RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT") +@RunWith(MockitoJUnitRunner.class) +public class KsStateStoreTest { + + private static final String STORE_NAME = "someStore"; + private static final Duration TIMEOUT = Duration.ofMillis(10); + + @Rule + public final Timeout timeout = Timeout.seconds(1); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Mock + private KafkaStreams kafkaStreams; + @Mock + private Clock clock; + + private KsStateStore store; + + @Before + public void setUp() { + store = new KsStateStore(STORE_NAME, kafkaStreams, TIMEOUT, clock); + + when(kafkaStreams.state()).thenReturn(State.RUNNING); + } + + @Test + public void shouldThrowNPEs() { + new NullPointerTester() + .setDefault(KafkaStreams.class, kafkaStreams) + .setDefault(Clock.class, clock) + .testConstructors(KsStateStore.class, Visibility.PACKAGE); + } + + @Test + public void shouldAwaitRunning() { + // Given: + when(kafkaStreams.state()) + .thenReturn(State.REBALANCING) + .thenReturn(State.REBALANCING) + .thenReturn(State.RUNNING); + + final QueryableStoreType> storeType = + QueryableStoreTypes.sessionStore(); + + // When: + + store.store(storeType); + + // Then: + verify(kafkaStreams, atLeast(3)).state(); + } + + @Test + public void shouldThrowIfDoesNotFinishRebalanceBeforeTimeout() { + // Given: + when(kafkaStreams.state()).thenReturn(State.REBALANCING); + when(clock.currentTimeMs()).thenReturn(0L, 5L, TIMEOUT.toMillis() + 1); + + // When: + expectedException.expect(MaterializationTimeOutException.class); + expectedException.expectMessage( + "Store failed to rebalance within the configured timeout. timeout: 10ms"); + + // When: + store.store(QueryableStoreTypes.sessionStore()); + } + + @Test + public void shouldThrowIfNotRunningAfterRebalanced() { + // Given: + when(kafkaStreams.state()) + .thenReturn(State.REBALANCING) + .thenReturn(State.REBALANCING) + .thenReturn(State.NOT_RUNNING); + + // When: + expectedException.expect(NotRunningException.class); + expectedException.expectMessage("The query was not in a running state. state: NOT_RUNNING"); + + // When: + store.store(QueryableStoreTypes.sessionStore()); + } + + @Test + public void shouldThrowIfPendingShutdown() { + // Given: + when(kafkaStreams.state()) + .thenReturn(State.REBALANCING) + .thenReturn(State.REBALANCING) + .thenReturn(State.PENDING_SHUTDOWN); + + // When: + expectedException.expect(NotRunningException.class); + expectedException.expectMessage("The query was not in a running state. state: PENDING_SHUTDOWN"); + + // When: + store.store(QueryableStoreTypes.sessionStore()); + } + + @Test + public void shouldGetStoreOnceRunning() { + // Given: + when(kafkaStreams.state()).thenReturn(State.RUNNING); + + // When: + store.store(QueryableStoreTypes.sessionStore()); + + // Then: + final InOrder inOrder = Mockito.inOrder(kafkaStreams); + inOrder.verify(kafkaStreams, atLeast(1)).state(); + inOrder.verify(kafkaStreams).store(any(), any()); + } + + @Test + public void shouldRequestStore() { + // Given: + final QueryableStoreType> storeType = + QueryableStoreTypes.windowStore(); + + // When: + store.store(storeType); + + // Then: + verify(kafkaStreams).store(STORE_NAME, storeType); + } + + @Test + public void shouldThrowIfStoreNotAvailableWhenRequested() { + // Given: + when(kafkaStreams.store(any(), any())).thenThrow(new InvalidStateStoreException("boom")); + + // Then: + expectedException.expect(MaterializationException.class); + expectedException.expectMessage("State store currently unavailable: " + STORE_NAME); + expectedException.expectCause(instanceOf(InvalidStateStoreException.class)); + + // When: + store.store(QueryableStoreTypes.windowStore()); + } + + @Test + public void shouldReturnSessionStore() { + // Given: + final ReadOnlySessionStore sessionStore = mock(ReadOnlySessionStore.class); + when(kafkaStreams.store(any(), any())).thenReturn(sessionStore); + + // When: + final ReadOnlySessionStore result = store + .store(QueryableStoreTypes.sessionStore()); + + // Then: + assertThat(result, is(sessionStore)); + } + + @Test + public void shouldReturnWindowStore() { + // Given: + final ReadOnlyWindowStore windowStore = mock(ReadOnlyWindowStore.class); + when(kafkaStreams.store(any(), any())).thenReturn(windowStore); + + // When: + final ReadOnlyWindowStore result = store + .store(QueryableStoreTypes.windowStore()); + + // Then: + assertThat(result, is(windowStore)); + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java b/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java index ae80eed22be9..59e775a615a8 100644 --- a/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java +++ b/ksql-functional-tests/src/test/java/io/confluent/ksql/test/rest/RestTestExecutor.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import io.confluent.ksql.json.JsonMapper; import io.confluent.ksql.rest.client.KsqlRestClient; import io.confluent.ksql.rest.client.RestResponse; @@ -37,6 +38,7 @@ import io.confluent.ksql.util.RetryUtil; import java.io.Closeable; import java.net.URL; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -44,16 +46,22 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Deserializer; import org.hamcrest.Matcher; import org.hamcrest.StringDescription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RestTestExecutor implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(RestTestExecutor.class); + private static final String STATEMENT_MACRO = "\\{STATEMENT}"; + private static final Duration MAX_STATIC_WARMUP = Duration.ofSeconds(10); private final KsqlRestClient restClient; private final EmbeddedSingleNodeKafkaCluster kafkaCluster; @@ -80,7 +88,7 @@ void buildAndExecuteQuery(final RestTestCase testCase) { } verifyOutput(testCase); - verifyResponses(testCase, responses.get()); + verifyResponses(responses.get(), testCase.getExpectedResponses(), testCase.getStatements()); } public void close() { @@ -98,7 +106,7 @@ private void initializeTopics(final List topics) { // Test case could be trying to create a topic deleted by previous test. // Need to wait for previous topic to be deleted async, until then requests will fail RetryUtil.retryWithBackoff( - 10, + 12, 10, (int) TimeUnit.SECONDS.toMillis(10), createJob @@ -141,10 +149,65 @@ private void produceInputs(final Map> inputs) { private Optional> sendStatements(final RestTestCase testCase) { - final String statements = testCase.getStatements().stream() + final List allStatements = testCase.getStatements(); + + int firstStatic = 0; + for (; firstStatic < allStatements.size(); firstStatic++) { + final boolean isStatic = allStatements.get(firstStatic).startsWith("SELECT "); + if (isStatic) { + break; + } + } + + final List nonStatics = IntStream.range(0, firstStatic) + .mapToObj(allStatements::get) + .collect(Collectors.toList()); + + final Optional> results = sendStatements(testCase, nonStatics); + if (!results.isPresent()) { + return Optional.empty(); + } + + final List statics = IntStream.range(firstStatic, allStatements.size()) + .mapToObj(allStatements::get) + .collect(Collectors.toList()); + + if (statics.isEmpty()) { + failIfExpectingError(testCase); + return results; + } + + if (!testCase.expectedError().isPresent() + && testCase.getExpectedResponses().size() > firstStatic + ) { + final String firstStaticStatement = statics.get(0); + final Response firstStaticResponse = testCase.getExpectedResponses().get(firstStatic); + + waitForWarmStateStores(firstStaticStatement, firstStaticResponse); + } + + final Optional> moreResults = sendStatements(testCase, statics); + if (!moreResults.isPresent()) { + return Optional.empty(); + } + + failIfExpectingError(testCase); + + return moreResults + .map(ksqlEntities -> ImmutableList.builder() + .addAll(results.get()) + .addAll(ksqlEntities) + .build()); + } + + private Optional> sendStatements( + final RestTestCase testCase, + final List statements + ) { + final String sql = statements.stream() .collect(Collectors.joining(System.lineSeparator())); - final RestResponse resp = restClient.makeKsqlRequest(statements); + final RestResponse resp = restClient.makeKsqlRequest(sql); if (resp.isErroneous()) { final Optional>> expectedError = testCase.expectedError(); @@ -164,11 +227,6 @@ private Optional> sendStatements(final RestTestCase testCase) { return Optional.empty(); } - testCase.expectedError().map(ee -> { - throw new AssertionError("Expected last statement to return an error: " - + StringDescription.toString(ee)); - }); - return Optional.of(resp.getResponse()); } @@ -198,11 +256,10 @@ private void verifyOutput(final RestTestCase testCase) { } private static void verifyResponses( - final RestTestCase testCase, - final List actualResponses + final List actualResponses, + final List expectedResponses, + final List statements ) { - final List expectedResponses = testCase.getExpectedResponses(); - assertThat( "Not enough responses", actualResponses, @@ -216,7 +273,7 @@ private static void verifyResponses( // Expected does not need to include everything, only needs to be tested: for (final Entry e : expected.entrySet()) { final String key = e.getKey(); - final Object value = replaceMacros(e.getValue(), testCase.getStatements(), idx); + final Object value = replaceMacros(e.getValue(), statements, idx); final String baseReason = "Response mismatch at index " + idx; assertThat(baseReason, actual, hasKey(key)); assertThat(baseReason + " on key: " + key, actual.get(key), is(value)); @@ -224,6 +281,13 @@ private static void verifyResponses( } } + private static void failIfExpectingError(final RestTestCase testCase) { + testCase.expectedError().map(ee -> { + throw new AssertionError("Expected last statement to return an error: " + + StringDescription.toString(ee)); + }); + } + private static Object replaceMacros( final Object value, final List statements, @@ -260,11 +324,11 @@ private static void compareKeyValueTimestamp( + "with timestamp=" + actualTimestamp); if (!Objects.equals(actualKey, expectedKey)) { - throw error; + throw error; } if (!Objects.equals(actualValue, expectedValue)) { - throw error; + throw error; } if (actualTimestamp != expectedTimestamp) { @@ -282,4 +346,38 @@ private static Map asJsonMap(final KsqlEntity response) { throw new AssertionError("Failed to serialize response to JSON: " + response); } } + + private void waitForWarmStateStores( + final String firstStaticStatement, + final Response firstStaticResponse + ) { + // Special handling for static queries is required, as they depend on materialized state stores + // being warmed up. Initial requests may return null values. + + final ImmutableList expectedResponse = ImmutableList.of(firstStaticResponse); + final ImmutableList statements = ImmutableList.of(firstStaticStatement); + + final long threshold = System.currentTimeMillis() + MAX_STATIC_WARMUP.toMillis(); + while (System.currentTimeMillis() < threshold) { + final RestResponse resp = restClient.makeKsqlRequest(firstStaticStatement); + if (resp.isErroneous()) { + Thread.yield(); + LOG.info("Server responded with an error code to a static query. " + + "This could be because the materialized store is not yet warm."); + continue; + } + + final KsqlEntityList actualResponses = resp.getResponse(); + + try { + verifyResponses(actualResponses, expectedResponse, statements); + return; + } catch (final AssertionError e) { + // Potentially, state stores not warm yet + LOG.info("Server responded with incorrect result to a static query. " + + "This could be because the materialized store is not yet warm.", e); + Thread.yield(); + } + } + } } diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/hopping-windows.json b/ksql-functional-tests/src/test/resources/query-validation-tests/hopping-windows.json index cd5fe9afb208..6bdb4644013e 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/hopping-windows.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/hopping-windows.json @@ -108,6 +108,29 @@ {"topic": "S2", "key": 0, "value": {"ID":0,"TOPK":[100.0]}, "timestamp": 10000, "window": {"start": 10000, "end": 40000, "type": "time"}} ] }, + { + "name": "count", + "statements": [ + "CREATE STREAM TEST (ID INT) WITH (kafka_topic='test_topic', value_format='JSON', key='ID');", + "CREATE TABLE S2 as SELECT count(1) as count FROM test WINDOW HOPPING (SIZE 5 SECOND, ADVANCE BY 1 SECONDS) group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": {"id": 0}, "timestamp": 10345}, + {"topic": "test_topic", "key": 0, "value": {"id": 0}, "timestamp": 13251} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 10345, "window": {"start": 6000, "end": 11000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 10345, "window": {"start": 7000, "end": 12000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 10345, "window": {"start": 8000, "end": 13000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 10345, "window": {"start": 9000, "end": 14000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 10345, "window": {"start": 10000, "end": 15000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":2}, "timestamp": 13251, "window": {"start": 9000, "end": 14000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":2}, "timestamp": 13251, "window": {"start": 10000, "end": 15000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 13251, "window": {"start": 11000, "end": 16000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 13251, "window": {"start": 12000, "end": 17000, "type": "time"}}, + {"topic": "S2", "key": 0, "value": {"COUNT":1}, "timestamp": 13251, "window": {"start": 13000, "end": 18000, "type": "time"}} + ] + }, { "name": "import hopping table", "statements": [ diff --git a/ksql-functional-tests/src/test/resources/rest-query-validation-tests/materialized-aggregate-static-queries.json b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/materialized-aggregate-static-queries.json new file mode 100644 index 000000000000..9b33d7581318 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/rest-query-validation-tests/materialized-aggregate-static-queries.json @@ -0,0 +1,394 @@ +{ + "comments": [ + "Tests covering static queries of (materialized) aggregate tables" + ], + "tests": [ + { + "name": "non-windowed single key lookup", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10';", + "SELECT * FROM AGGREGATE WHERE ROWKEY='missing';" + ], + "inputs": [ + {"topic": "test_topic", "key": "11", "value": {}}, + {"topic": "test_topic", "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": null, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "tumbling windowed single key lookup with exact window start", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW TUMBLING(SIZE 1 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=12000;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=13000;" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 11345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 12345, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 12000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "hopping windowed single key lookup with exact window start", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW HOPPING(SIZE 10 SECOND, ADVANCE BY 1 SECONDS) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=12000;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=13000;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=14000;" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 10345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 10345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 13345, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 12000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": [ + {"window": {"start": 13000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "session windowed single key lookup with exact window start", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW SESSION(10 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=12345;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=12000;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart=13000;" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 12366, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 12345, "end": 12366}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 2}} + ]}, + {"@type": "row", "rows": []}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "tumbling windowed single key lookup with window start range", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW TUMBLING(SIZE 1 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE 11111 <= WindowStart AND WindowStart < 14144 AND ROWKEY='10';", + "SELECT * FROM AGGREGATE WHERE WindowStart >= 17000 AND 11234756356 > WindowStart AND ROWKEY='10';" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 12001, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 12211, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 14253, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 15364, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 12000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 2}}, + {"window": {"start": 14000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "hopping windowed single key lookup with window start range", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW HOPPING(SIZE 5 SECOND, ADVANCE BY 1 SECONDS) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND 6001 <= WindowStart AND WindowStart < 12000;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND 13001 <= WindowStart AND WindowStart < 11234756356;" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 10021, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 10345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 13251, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 7000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}}, + {"window": {"start": 8000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}}, + {"window": {"start": 9000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 2}}, + {"window": {"start": 10000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 2}}, + {"window": {"start": 11000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "session windowed single key lookup with window start range", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW SESSION(10 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND 8001 <= WindowStart AND WindowStart < 19444;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND 14000 <= WindowStart AND WindowStart < 11234756356;" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 12345, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 12345, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 13456, "key": "10", "value": {}}, + {"topic": "test_topic", "timestamp": 24456, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 12345, "end": 13456}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 2}} + ]}, + {"@type": "row", "rows": [ + {"window": {"start": 24456, "end": 24456}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]} + ] + }, + { + "name": "text datetime window bounds", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW TUMBLING(SIZE 1 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' AND WindowStart='2020-02-23T23:45:12.000';" + ], + "inputs": [ + {"topic": "test_topic", "timestamp": 1582501512456, "key": "11", "value": {}}, + {"topic": "test_topic", "timestamp": 1582501512456, "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": {"start": 1582501512000, "end": null}, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]} + ] + }, + { + "name": "aliased table", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE A WHERE '10'=ROWKEY;" + ], + "inputs": [ + {"topic": "test_topic", "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": null, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1}} + ]} + ] + }, + { + "name": "multiple aggregate columns", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT, SUM(CAST(ROWKEY AS INT)) AS SUM FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10';" + ], + "inputs": [ + {"topic": "test_topic", "key": "10", "value": {}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": null, "key": {"ROWKEY": "10"}, "value": {"COUNT": 1, "SUM": 10}} + ]} + ] + }, + { + "name": "having clause on aggregate", + "statements": [ + "CREATE STREAM INPUT (X INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT SUM(X) AS SUM FROM INPUT GROUP BY ROWKEY HAVING SUM(X) < 10;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10';", + "SELECT * FROM AGGREGATE WHERE ROWKEY='missing';" + ], + "inputs": [ + {"topic": "test_topic", "key": "10", "value": {"X": 9}}, + {"topic": "test_topic", "key": "missing", "value": {"X": 10}} + ], + "responses": [ + {"@type": "currentStatus"}, + {"@type": "currentStatus"}, + {"@type": "row", "rows": [ + {"window": null, "key": {"ROWKEY": "10"}, "value": {"SUM": 9}} + ]}, + {"@type": "row", "rows": []} + ] + }, + { + "name": "fail on unsupported query feature: non-select-star projections", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT COUNT FROM AGGREGATE WHERE ROWKEY='10';" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Static queries currently only support a 'SELECT *' projections", + "status": 400 + } + }, + { + "name": "fail on unsupported query feature: join", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE A JOIN INPUT I ON A.ROWKEY=I.ROWKEY WHERE A.ROWKEY='10';" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Static queries do not support joins.", + "status": 400 + } + }, + { + "name": "fail on unsupported query feature: window", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WINDOW TUMBLING (SIZE 1 SECOND) WHERE ROWKEY='10';" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Static queries do not support WINDOW clauses.", + "status": 400 + } + }, + { + "name": "fail on unsupported query feature: group by", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' GROUP BY ROWKEY;" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Static queries do not support GROUP BY clauses", + "status": 400 + } + }, + { + "name": "fail on unsupported query feature: where multiple rowkeys", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10' OR ROWKEY='11';" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "Only AND expressions are supported: ((AGGREGATE.ROWKEY = '10') OR (AGGREGATE.ROWKEY = '11'))", + "status": 400 + } + }, + { + "name": "fail on unsupported query feature: where rowkey range", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE '0' FROM GROUP BY ' style statement.", + "status": 400 + } + }, + { + "name": "fail on windowed table if no window start bounds provided", + "statements": [ + "CREATE STREAM INPUT (IGNORED INT) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE AGGREGATE AS SELECT COUNT(1) AS COUNT FROM INPUT WINDOW TUMBLING(SIZE 1 SECOND) GROUP BY ROWKEY;", + "SELECT * FROM AGGREGATE WHERE ROWKEY='10';" + ], + "expectedError": { + "type": "io.confluent.ksql.rest.entity.KsqlStatementErrorMessage", + "message": "WHERE clause missing WINDOWSTART", + "status": 400 + } + } + ] +} \ No newline at end of file diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/rewrite/StatementRewriteForRowtime.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/rewrite/StatementRewriteForRowtime.java index ddd97465ec46..bd2535d9577a 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/rewrite/StatementRewriteForRowtime.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/rewrite/StatementRewriteForRowtime.java @@ -15,6 +15,9 @@ package io.confluent.ksql.parser.rewrite; +import static io.confluent.ksql.util.KsqlConstants.DATE_TIME_PATTERN; +import static io.confluent.ksql.util.KsqlConstants.TIME_PATTERN; + import io.confluent.ksql.execution.expression.tree.BetweenPredicate; import io.confluent.ksql.execution.expression.tree.ComparisonExpression; import io.confluent.ksql.execution.expression.tree.DereferenceExpression; @@ -25,7 +28,6 @@ import io.confluent.ksql.parser.rewrite.ExpressionTreeRewriter.Context; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.timestamp.StringToTimestampParser; - import java.time.ZoneId; import java.util.Objects; import java.util.Optional; @@ -102,9 +104,7 @@ private static boolean expressionIsRowtime(final Expression node) { } private static LongLiteral rewriteTimestamp(final String timestamp) { - final String timePattern = "HH:mm:ss.SSS"; - final StringToTimestampParser parser = new StringToTimestampParser( - "yyyy-MM-dd'T'" + timePattern); + final StringToTimestampParser parser = new StringToTimestampParser(DATE_TIME_PATTERN); final String date; final String time; @@ -113,13 +113,13 @@ private static LongLiteral rewriteTimestamp(final String timestamp) { if (timestamp.contains("T")) { date = timestamp.substring(0, timestamp.indexOf('T')); final String withTimezone = completeTime( - timestamp.substring(timestamp.indexOf('T') + 1), - timePattern); + timestamp.substring(timestamp.indexOf('T') + 1) + ); timezone = getTimezone(withTimezone); - time = completeTime(withTimezone.substring(0, timezone.length()), timePattern); + time = completeTime(withTimezone.substring(0, timezone.length())); } else { date = completeDate(timestamp); - time = completeTime("", timePattern); + time = completeTime(""); timezone = ""; } @@ -159,10 +159,12 @@ private static String completeDate(final String date) { } } - private static String completeTime(final String time, final String timePattern) { - if (time.length() >= timePattern.length()) { + private static String completeTime(final String time) { + if (time.length() >= TIME_PATTERN.length()) { return time; } - return time + timePattern.substring(time.length()).replaceAll("[a-zA-Z]", "0"); + + return time + + TIME_PATTERN.substring(time.length()).replaceAll("[a-zA-Z]", "0"); } } \ No newline at end of file diff --git a/ksql-rest-app/pom.xml b/ksql-rest-app/pom.xml index a6f339373131..7afdd6f74831 100644 --- a/ksql-rest-app/pom.xml +++ b/ksql-rest-app/pom.xml @@ -42,6 +42,11 @@ ksql-rest-model + + io.confluent.ksql + ksql-rest-client + + io.confluent.ksql ksql-engine @@ -107,12 +112,6 @@ test - - io.confluent.ksql - ksql-rest-client - test - - diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntityFactory.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntityFactory.java new file mode 100644 index 000000000000..8a44c4d2fd67 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntityFactory.java @@ -0,0 +1,112 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.Window; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.schema.ksql.Column; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.SqlBaseType; +import java.time.Instant; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.stream.Collectors; +import org.apache.kafka.connect.data.Schema.Type; +import org.apache.kafka.connect.data.Struct; + +/** + * Factory class for {@link QueryResultEntity} + */ +public final class QueryResultEntityFactory { + + private QueryResultEntityFactory() { + } + + public static List createRows( + final Struct key, + final Map, GenericRow> rows, + final LogicalSchema schema + ) { + final LinkedHashMap keyFields = toOrderedMap(key); + + return rows.entrySet().stream() + .map(row -> new Row(window(row), keyFields, toOrderedMap(row.getValue(), schema.value()))) + .collect(Collectors.toList()); + } + + private static Optional window( + final Entry, GenericRow> row + ) { + return row.getKey() + .map(w -> new QueryResultEntity.Window( + w.start() + .toEpochMilli(), + w.end() + .map(Instant::toEpochMilli) + .map(OptionalLong::of) + .orElse(OptionalLong.empty()) + )); + } + + private static LinkedHashMap toOrderedMap(final Struct struct) { + if (struct == null) { + return null; + } + + final LinkedHashMap orderedMap = new LinkedHashMap<>(); + + for (int idx = 0; idx < struct.schema().fields().size(); idx++) { + final org.apache.kafka.connect.data.Field field = struct.schema().fields().get(idx); + final Object value = field.schema().type() == Type.STRUCT + ? toOrderedMap((Struct) struct.get(field)) + : struct.get(field); + + orderedMap.put(field.name(), value); + } + + return orderedMap; + } + + private static LinkedHashMap toOrderedMap( + final GenericRow row, + final List schema + ) { + if (row.getColumns().size() != schema.size()) { + throw new IllegalArgumentException("Column count mismatch." + + " expected: " + schema.size() + + ", got: " + row.getColumns().size() + ); + } + + final LinkedHashMap orderedMap = new LinkedHashMap<>(); + + for (int idx = 0; idx < row.getColumns().size(); idx++) { + final Column column = schema.get(idx); + final Object value = column.type().baseType() == SqlBaseType.STRUCT + ? toOrderedMap((Struct) row.getColumns().get(idx)) + : row.getColumns().get(idx); + + orderedMap.put(column.fullName(), value); + } + + return orderedMap; + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java index 72bb188964fd..d4688895bf41 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java @@ -32,6 +32,7 @@ import io.confluent.ksql.parser.tree.ListTables; import io.confluent.ksql.parser.tree.ListTopics; import io.confluent.ksql.parser.tree.ListTypes; +import io.confluent.ksql.parser.tree.Query; import io.confluent.ksql.parser.tree.SetProperty; import io.confluent.ksql.parser.tree.ShowColumns; import io.confluent.ksql.parser.tree.Statement; @@ -54,6 +55,8 @@ @SuppressWarnings({"unchecked", "rawtypes"}) public enum CustomExecutors { + POINT_QUERY(Query.class, StaticQueryExecutor::execute), + LIST_TOPICS(ListTopics.class, ListTopicsExecutor::execute), LIST_STREAMS(ListStreams.class, ListSourceExecutor::streams), LIST_TABLES(ListTables.class, ListSourceExecutor::tables), diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/StaticQueryExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/StaticQueryExecutor.java new file mode 100644 index 000000000000..284c6fc4dc10 --- /dev/null +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/StaticQueryExecutor.java @@ -0,0 +1,611 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import static java.util.Objects.requireNonNull; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMap.Builder; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.google.common.collect.Sets.SetView; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.execution.context.QueryContext.Stacker; +import io.confluent.ksql.execution.expression.tree.ComparisonExpression; +import io.confluent.ksql.execution.expression.tree.ComparisonExpression.Type; +import io.confluent.ksql.execution.expression.tree.DereferenceExpression; +import io.confluent.ksql.execution.expression.tree.Expression; +import io.confluent.ksql.execution.expression.tree.IntegerLiteral; +import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.execution.expression.tree.LogicalBinaryExpression; +import io.confluent.ksql.execution.expression.tree.LongLiteral; +import io.confluent.ksql.execution.expression.tree.StringLiteral; +import io.confluent.ksql.materialization.Locator; +import io.confluent.ksql.materialization.Locator.KsqlNode; +import io.confluent.ksql.materialization.Materialization; +import io.confluent.ksql.materialization.MaterializationTimeOutException; +import io.confluent.ksql.materialization.Window; +import io.confluent.ksql.metastore.MetaStore; +import io.confluent.ksql.metastore.model.DataSource; +import io.confluent.ksql.parser.tree.AliasedRelation; +import io.confluent.ksql.parser.tree.Join; +import io.confluent.ksql.parser.tree.Query; +import io.confluent.ksql.parser.tree.Relation; +import io.confluent.ksql.parser.tree.Select; +import io.confluent.ksql.parser.tree.SelectItem; +import io.confluent.ksql.parser.tree.SingleColumn; +import io.confluent.ksql.parser.tree.Table; +import io.confluent.ksql.query.QueryId; +import io.confluent.ksql.rest.Errors; +import io.confluent.ksql.rest.client.KsqlRestClient; +import io.confluent.ksql.rest.client.RestResponse; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.KsqlEntityList; +import io.confluent.ksql.rest.entity.QueryResultEntity; +import io.confluent.ksql.rest.entity.QueryResultEntityFactory; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlConstants; +import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.KsqlServerException; +import io.confluent.ksql.util.KsqlStatementException; +import io.confluent.ksql.util.PersistentQueryMetadata; +import io.confluent.ksql.util.SchemaUtil; +import io.confluent.ksql.util.timestamp.StringToTimestampParser; +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.kafka.connect.data.Struct; + +// CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling +public final class StaticQueryExecutor { + // CHECKSTYLE_RULES.ON: ClassDataAbstractionCoupling + + private static final Duration OWNERSHIP_TIMEOUT = Duration.ofSeconds(30); + private static final Set VALID_WINDOW_BOUNDS_TYPES = ImmutableSet.of( + Type.EQUAL, Type.GREATER_THAN_OR_EQUAL, Type.LESS_THAN + ); + private static final String VALID_WINDOW_BOUNDS_TYPES_STRING = + VALID_WINDOW_BOUNDS_TYPES.toString(); + + private StaticQueryExecutor() { + } + + public static void validate( + final ConfiguredStatement statement, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext + ) { + final Query queryStmt = statement.getStatement(); + + if (!queryStmt.isStatic()) { + throw new KsqlRestException(Errors.queryEndpoint(statement.getStatementText())); + } + + try { + validateSelects(queryStmt.getSelect()); + + final PersistentQueryMetadata query = findMaterializingQuery(executionContext, queryStmt); + + extractWhereInfo(queryStmt.getWhere(), query); + + if (queryStmt.getWindow().isPresent()) { + throw new KsqlException("Static queries do not support WINDOW clauses."); + } + + if (queryStmt.getGroupBy().isPresent()) { + throw new KsqlException("Static queries do not support GROUP BY clauses."); + } + + } catch (final Exception e) { + throw new KsqlStatementException( + e.getMessage(), + statement.getStatementText(), + e + ); + } + } + + public static Optional execute( + final ConfiguredStatement statement, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext + ) { + try { + final Query queryStmt = statement.getStatement(); + + final PersistentQueryMetadata query = findMaterializingQuery(executionContext, queryStmt); + + final WhereInfo whereInfo = extractWhereInfo(queryStmt.getWhere(), query); + + final QueryContext.Stacker contextStacker = new Stacker(new QueryId("static-query")); + + final Materialization mat = query + .getMaterialization(contextStacker) + .orElseThrow(() -> notMaterializedException(getSourceRelation(queryStmt.getFrom()))); + + final Struct rowKey = asKeyStruct(whereInfo.rowkey, query.getPhysicalSchema()); + + final KsqlNode owner = getOwner(rowKey, mat); + if (!owner.isLocal()) { + return Optional.of(proxyTo(owner, statement)); + } + + final Map, GenericRow> result; + if (whereInfo.windowBounds.isPresent()) { + final WindowBounds windowBounds = whereInfo.windowBounds.get(); + + final Builder, GenericRow> builder = ImmutableMap.builder(); + mat.windowed().get(rowKey, windowBounds.lower, windowBounds.upper) + .forEach((k, v) -> builder.put(Optional.of(k), v)); + + result = builder.build(); + } else { + result = mat.nonWindowed().get(rowKey) + .map(v -> ImmutableMap.of(Optional.empty(), v)) + .orElse(ImmutableMap.of()); + } + + final QueryResultEntity entity = new QueryResultEntity( + statement.getStatementText(), + mat.windowType(), + mat.schema(), + QueryResultEntityFactory.createRows(rowKey, result, mat.schema()) + ); + + return Optional.of(entity); + } catch (final Exception e) { + throw new KsqlStatementException( + e.getMessage(), + statement.getStatementText(), + e + ); + } + } + + private static final class WindowBounds { + + private final Instant lower; + private final Instant upper; + + private WindowBounds(final Instant lower, final Instant upper) { + this.lower = requireNonNull(lower, "lower"); + this.upper = requireNonNull(upper, "upper'"); + + if (lower.isAfter(upper)) { + throw new KsqlException("Lower window bound must be less than upper bound"); + } + } + } + + private static final class WhereInfo { + + private final Object rowkey; + private final Optional windowBounds; + + private WhereInfo( + final Object rowkey, + final Optional windowBounds + ) { + this.rowkey = rowkey; + this.windowBounds = windowBounds; + } + } + + private static WhereInfo extractWhereInfo( + final Optional possibleWhere, + final PersistentQueryMetadata query + ) { + final boolean windowed = query.getResultTopic().getKeyFormat().isWindowed(); + + final Expression where = possibleWhere + .orElseThrow(() -> invalidWhereClauseException("missing WHERE clause", windowed)); + + final Map> comparisons = extractComparisons(where); + + final List rowKeyComparison = comparisons.get(ComparisonTarget.ROWKEY); + if (rowKeyComparison == null) { + throw invalidWhereClauseException("WHERE clause missing ROWKEY", windowed); + } + + final Object rowKey = extractRowKeyWhereClause(rowKeyComparison, windowed); + + if (!windowed) { + if (comparisons.size() > 1) { + throw invalidWhereClauseException("Unsupported WHERE clause", windowed); + } + + return new WhereInfo(rowKey, Optional.empty()); + } + + final List windowBoundsComparison = + comparisons.get(ComparisonTarget.WINDOWSTART); + + if (windowBoundsComparison == null) { + throw invalidWhereClauseException( + "WHERE clause missing " + ComparisonTarget.WINDOWSTART, + windowed + ); + } + + final WindowBounds windowBounds = extractWhereClauseWindowBounds(windowBoundsComparison); + + return new WhereInfo(rowKey, Optional.of(windowBounds)); + } + + private static Object extractRowKeyWhereClause( + final List comparisons, + final boolean windowed + ) { + if (comparisons.size() != 1) { + throw invalidWhereClauseException("Multiple bounds on ROWKEY", windowed); + } + + final ComparisonExpression comparison = comparisons.get(0); + + final Expression other = comparison.getRight() instanceof DereferenceExpression + ? comparison.getLeft() + : comparison.getRight(); + + if (!(other instanceof StringLiteral)) { + throw invalidWhereClauseException("ROWKEY must be comparsed to STRING literal.", false); + } + + if (comparison.getType() != Type.EQUAL) { + throw invalidWhereClauseException("ROWKEY bound must currently be '='.", false); + } + + final Literal right = (Literal) other; + return right.getValue(); + } + + private static WindowBounds extractWhereClauseWindowBounds( + final List comparisons + ) { + final Map> byType = comparisons.stream() + .collect(Collectors.groupingBy(StaticQueryExecutor::getBoundType)); + + final SetView unsupported = Sets.difference(byType.keySet(), VALID_WINDOW_BOUNDS_TYPES); + if (!unsupported.isEmpty()) { + throw invalidWhereClauseException( + "Unsupported " + ComparisonTarget.WINDOWSTART + " bounds: " + unsupported, + true + ); + } + + final String duplicates = byType.entrySet().stream() + .filter(e -> e.getValue().size() > 1) + .map(e -> e.getKey() + ": " + e.getValue()) + .collect(Collectors.joining(System.lineSeparator())); + + if (!duplicates.isEmpty()) { + throw invalidWhereClauseException( + "Duplicate bounds on " + ComparisonTarget.WINDOWSTART + ": " + duplicates, + true + ); + } + + final Map singles = byType.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().get(0))); + + final ComparisonExpression equals = singles.get(Type.EQUAL); + if (equals != null) { + if (byType.size() > 1) { + throw invalidWhereClauseException( + "`" + equals + "` cannot be combined with other bounds on " + + ComparisonTarget.WINDOWSTART, + true + ); + } + + final Instant windowStart = extractWindowBound(equals); + return new WindowBounds(windowStart, windowStart); + } + + final ComparisonExpression upper = singles.get(Type.LESS_THAN); + if (upper == null) { + throw invalidWhereClauseException( + "Missing upper bound on " + ComparisonTarget.WINDOWSTART, + true + ); + } + + final ComparisonExpression lower = singles.get(Type.GREATER_THAN_OR_EQUAL); + if (lower == null) { + throw invalidWhereClauseException( + "Missing lower bound on " + ComparisonTarget.WINDOWSTART, + true + ); + } + + final Instant upperBound = extractWindowBound(upper); + final Instant lowerBound = extractWindowBound(lower); + return new WindowBounds(lowerBound, upperBound); + } + + private static Type getBoundType(final ComparisonExpression comparison) { + final Type type = comparison.getType(); + final boolean inverted = comparison.getRight() instanceof DereferenceExpression; + + switch (type) { + case LESS_THAN: + return inverted ? Type.GREATER_THAN : type; + case LESS_THAN_OR_EQUAL: + return inverted ? Type.GREATER_THAN_OR_EQUAL : type; + case GREATER_THAN: + return inverted ? Type.LESS_THAN : type; + case GREATER_THAN_OR_EQUAL: + return inverted ? Type.LESS_THAN_OR_EQUAL : type; + default: + return type; + } + } + + private static Instant extractWindowBound(final ComparisonExpression comparison) { + final Expression other = comparison.getRight() instanceof DereferenceExpression + ? comparison.getLeft() + : comparison.getRight(); + + final Instant bound = asInstant(other); + + switch (getBoundType(comparison)) { + case LESS_THAN: + return bound.minusMillis(1); + case GREATER_THAN: + return bound.plusMillis(1); + default: + return bound; + } + } + + private static Instant asInstant(final Expression other) { + if (other instanceof IntegerLiteral) { + return Instant.ofEpochMilli(((IntegerLiteral) other).getValue()); + } + + if (other instanceof LongLiteral) { + return Instant.ofEpochMilli(((LongLiteral) other).getValue()); + } + + if (other instanceof StringLiteral) { + final String text = ((StringLiteral) other).getValue(); + try { + final long timestamp = new StringToTimestampParser(KsqlConstants.DATE_TIME_PATTERN) + .parse(text); + + return Instant.ofEpochMilli(timestamp); + } catch (final Exception e) { + throw invalidWhereClauseException("Failed to parse datetime: " + text, true); + } + } + + throw invalidWhereClauseException( + ComparisonTarget.WINDOWSTART + " bounds must be BIGINT", + true + ); + } + + private enum ComparisonTarget { + ROWKEY, + WINDOWSTART + } + + private static Map> extractComparisons( + final Expression exp + ) { + if (exp instanceof ComparisonExpression) { + final ComparisonExpression comparison = (ComparisonExpression) exp; + final ComparisonTarget target = extractWhereClauseTarget(comparison); + return ImmutableMap.of(target, ImmutableList.of(comparison)); + } + + if (exp instanceof LogicalBinaryExpression) { + final LogicalBinaryExpression binary = (LogicalBinaryExpression) exp; + if (binary.getType() != LogicalBinaryExpression.Type.AND) { + throw invalidWhereClauseException("Only AND expressions are supported: " + exp, false); + } + + final Map> left = + extractComparisons(binary.getLeft()); + + final Map> right = + extractComparisons(binary.getRight()); + + return Stream + .concat(left.entrySet().stream(), right.entrySet().stream()) + .collect(Collectors.toMap(Entry::getKey, Entry::getValue, (l, r) -> + ImmutableList.builder().addAll(l).addAll(r).build() + )); + } + + throw invalidWhereClauseException("Unsupported expression: " + exp, false); + } + + private static ComparisonTarget extractWhereClauseTarget(final ComparisonExpression comparison) { + + final DereferenceExpression column; + if (comparison.getRight() instanceof DereferenceExpression) { + column = (DereferenceExpression) comparison.getRight(); + } else if (comparison.getLeft() instanceof DereferenceExpression) { + column = (DereferenceExpression) comparison.getLeft(); + } else { + throw invalidWhereClauseException("Invalid WHERE clause: " + comparison, false); + } + + final String fieldName = column.getFieldName(); + + try { + return ComparisonTarget.valueOf(fieldName.toUpperCase()); + } catch (final Exception e) { + throw invalidWhereClauseException("WHERE clause on unsupported field: " + fieldName, false); + } + } + + private static void validateSelects(final Select select) { + final List selectItems = select.getSelectItems(); + + if (selectItems.size() != 1 + || selectItems.get(0) instanceof SingleColumn + ) { + throw new KsqlException("Static queries currently only support a 'SELECT *' projections"); + } + } + + private static PersistentQueryMetadata findMaterializingQuery( + final KsqlExecutionContext executionContext, + final Query query + ) { + final MetaStore metaStore = executionContext.getMetaStore(); + + final Table sourceTable = getSourceRelation(query.getFrom()); + + final DataSource source = getSource(sourceTable, metaStore); + + final Set queries = metaStore.getQueriesWithSink(source.getName()); + if (queries.isEmpty()) { + throw notMaterializedException(sourceTable); + } + if (queries.size() > 1) { + throw new KsqlException("Multiple queries currently materialize '" + sourceTable + "'." + + " KSQL currently only supports static queries when the table has only been" + + " materialized once."); + } + + final QueryId queryId = new QueryId(Iterables.get(queries, 0)); + + return executionContext.getPersistentQuery(queryId) + .orElseThrow(() -> new KsqlException("Materializing query has been stopped")); + } + + private static DataSource getSource( + final Table sourceTable, + final MetaStore metaStore + ) { + final DataSource source = metaStore.getSource(sourceTable.getName().toString()); + if (source == null) { + throw new KsqlException("Unknown source: " + sourceTable.getName()); + } + + return source; + } + + private static Table getSourceRelation(final Relation from) { + if (from instanceof Join) { + throw new KsqlException("Static queries do not support joins."); + } + + if (from instanceof Table) { + return (Table) from; + } + + if (from instanceof AliasedRelation) { + return getSourceRelation(((AliasedRelation) from).getRelation()); + } + + throw new KsqlException("Unsupported source type: " + from.getClass().getSimpleName()); + } + + private static KsqlNode getOwner(final Struct rowKey, final Materialization mat) { + final Locator locator = mat.locator(); + + final long threshold = System.currentTimeMillis() + OWNERSHIP_TIMEOUT.toMillis(); + while (System.currentTimeMillis() < threshold) { + final Optional owner = locator.locate(rowKey); + if (owner.isPresent()) { + return owner.get(); + } + } + + throw new MaterializationTimeOutException( + "The owner of the key could not be determined within the configured timeout" + ); + } + + private static KsqlEntity proxyTo( + final KsqlNode owner, + final ConfiguredStatement statement + ) { + try (KsqlRestClient client = new KsqlRestClient(owner.location().toString())) { + + final RestResponse response = client + .makeKsqlRequest(statement.getStatementText()); + + if (response.isErroneous()) { + throw new KsqlServerException("Proxy attempt failed: " + response.getErrorMessage()); + } + + final KsqlEntityList entities = response.getResponse(); + if (entities.size() != 1) { + throw new RuntimeException("Boom - expected 1 entity, got: " + entities.size()); + } + + return entities.get(0); + } + } + + private static KsqlException notMaterializedException(final Table sourceTable) { + return new KsqlException( + "Table '" + sourceTable + "' is not materialized." + + " KSQL currently only supports static queries on materialized aggregate tables." + + " i.e. those created by a" + + " 'CREATE TABLE AS SELECT FROM GROUP BY ' style statement."); + } + + private static KsqlException invalidWhereClauseException( + final String msg, + final boolean windowed + ) { + final String additional = !windowed + ? "" + : System.lineSeparator() + + " - limits the time bounds of the windowed table. This can be: " + + System.lineSeparator() + + " + a single window lower bound, e.g. `WHERE WINDOWSTART = z`, or" + + System.lineSeparator() + + " + a range, e.g. `WHERE a < WINDOWSTART AND WINDOWSTART < b" + + System.lineSeparator() + + "WINDOWSTART currently supports operators: " + VALID_WINDOW_BOUNDS_TYPES_STRING + + System.lineSeparator() + + "WINDOWSTART currently comparison with epoch milliseconds " + + "or a datetime string in the form: " + KsqlConstants.DATE_TIME_PATTERN; + + return new KsqlException(msg + + System.lineSeparator() + + "Static queries currently require a WHERE clause that:" + + System.lineSeparator() + + " - limits the query to a single ROWKEY, e.g. `SELECT * FROM X WHERE ROWKEY=Y;`." + + additional + ); + } + + private static Struct asKeyStruct(final Object rowKey, final PhysicalSchema physicalSchema) { + final Struct key = new Struct(physicalSchema.keySchema().ksqlSchema()); + key.put(SchemaUtil.ROWKEY_NAME, rowKey); + return key; + } +} diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java index 05a3623a64c3..787cb164db45 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java @@ -44,6 +44,7 @@ import io.confluent.ksql.rest.server.execution.ExplainExecutor; import io.confluent.ksql.rest.server.execution.ListSourceExecutor; import io.confluent.ksql.rest.server.execution.PropertyExecutor; +import io.confluent.ksql.rest.server.execution.StaticQueryExecutor; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.KsqlException; @@ -61,7 +62,7 @@ @SuppressWarnings({"unchecked", "rawtypes"}) public enum CustomValidators { - QUERY_ENDPOINT(Query.class, QueryValidator::validate), + QUERY_ENDPOINT(Query.class, StaticQueryExecutor::validate), PRINT_TOPIC(PrintTopic.class, PrintTopicValidator::validate), LIST_TOPICS(ListTopics.class, StatementValidator.NO_VALIDATION), diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/QueryValidator.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/QueryValidator.java deleted file mode 100644 index 66fe30141ab9..000000000000 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/QueryValidator.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Copyright 2019 Confluent Inc. - * - * Licensed under the Confluent Community License (the "License"); you may not use - * this file except in compliance with the License. You may obtain a copy of the - * License at - * - * http://www.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.rest.server.validation; - -import io.confluent.ksql.KsqlExecutionContext; -import io.confluent.ksql.rest.Errors; -import io.confluent.ksql.rest.server.resources.KsqlRestException; -import io.confluent.ksql.services.ServiceContext; -import io.confluent.ksql.statement.ConfiguredStatement; - -public final class QueryValidator { - - private QueryValidator() { } - - public static void validate( - final ConfiguredStatement statement, - final KsqlExecutionContext context, - final ServiceContext serviceContext) { - throw new KsqlRestException(Errors.queryEndpoint(statement.getStatementText())); - } - -} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryDescriptionFactoryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryDescriptionFactoryTest.java index 71709d325746..959856bad57f 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryDescriptionFactoryTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryDescriptionFactoryTest.java @@ -37,6 +37,7 @@ import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Optional; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.Consumer; import org.apache.kafka.streams.KafkaStreams; @@ -110,6 +111,7 @@ public void setUp() { "execution plan", QUERY_ID, DataSourceType.KSTREAM, + Optional.empty(), "app id", sinkTopic, topology, diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityFactoryTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityFactoryTest.java new file mode 100644 index 000000000000..e7fd9dbf0393 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityFactoryTest.java @@ -0,0 +1,193 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.materialization.Window; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.SchemaConverters; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.Test; + +public class QueryResultEntityFactoryTest { + + private static final Schema STRUCT_SCHEMA = SchemaBuilder.struct() + .field("i0", Schema.OPTIONAL_INT32_SCHEMA) + .optional() + .build(); + + private static final Schema NESTED_SCHEMA = SchemaBuilder.struct() + .field("o0", STRUCT_SCHEMA) + .optional() + .build(); + + private static final Struct STRUCT = new Struct(STRUCT_SCHEMA) + .put("i0", 10); + + private static final Struct NESTED_STRUCT = new Struct(NESTED_SCHEMA) + .put("o0", STRUCT); + + private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() + .valueColumn("o0", SchemaConverters.connectToSqlConverter().toSqlType(STRUCT_SCHEMA)) + .build(); + + private static final GenericRow A_VALUE = new GenericRow(STRUCT); + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnValueFieldMismatch() { + // Given: + final GenericRow value = new GenericRow("text", 10); // <-- 2 fields vs 1 in schema + + // When: + QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.empty(), value), LOGICAL_SCHEMA); + } + + @Test + public void shouldHandleNestedStructsInKey() { + // When: + final List rows = QueryResultEntityFactory + .createRows(NESTED_STRUCT, ImmutableMap.of(Optional.empty(), A_VALUE), LOGICAL_SCHEMA); + + // Then: + final Map key = rows.get(0).getKey(); + assertThat(key.get("o0"), is(instanceOf(Map.class))); + assertThat(((Map) key.get("o0")), hasEntry("i0", 10)); + } + + @Test + public void shouldHandleValue() { + // Given: + final LogicalSchema logicalSchema = LogicalSchema.builder() + .valueColumn("c0", SqlTypes.STRING) + .valueColumn("c1", SqlTypes.INTEGER) + .build(); + + final GenericRow row = new GenericRow("text", 10); + + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.empty(), row), logicalSchema); + + // Then: + final Map value = rows.get(0).getValue(); + assertThat(value.get("c0"), is("text")); + assertThat(value.get("c1"), is(10)); + } + + @Test + public void shouldHandleNestedStructsInValue() { + // Given: + final GenericRow row = new GenericRow(STRUCT); + + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.empty(), row), LOGICAL_SCHEMA); + + // Then: + final Map value = rows.get(0).getValue(); + assertThat(value.get("o0"), is(instanceOf(Map.class))); + assertThat(((Map) value.get("o0")), hasEntry("i0", 10)); + } + + @Test + public void shouldHandleNoValue() { + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(), LOGICAL_SCHEMA); + + // Then: + assertThat(rows, is(empty())); + } + + @Test + public void shouldHandleNullStructInKey() { + // Given: + final Struct nestedStruct = new Struct(NESTED_SCHEMA) + .put("o0", null); + + // When: + final List rows = QueryResultEntityFactory + .createRows(nestedStruct, ImmutableMap.of(Optional.empty(), A_VALUE), LOGICAL_SCHEMA); + + // Then: + assertThat(rows.get(0).getKey().get("o0"), is(nullValue())); + } + + @Test + public void shouldHandleNullStructInValue() { + // Given: + final GenericRow row = new GenericRow((Object) null); + + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.empty(), row), LOGICAL_SCHEMA); + + // Then: + assertThat(rows.get(0).getValue().get("o0"), is(nullValue())); + } + + @Test + public void shouldHandleTimedWindow() { + // Given: + final Window window = Window.of(Instant.ofEpochMilli(10_123), Optional.empty()); + + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.of(window), A_VALUE), LOGICAL_SCHEMA); + + // Then: + final Optional actual = rows.get(0).getWindow(); + assertThat(actual, is(Optional.of(new QueryResultEntity.Window(10_123, OptionalLong.empty())))); + } + + @Test + public void shouldHandleSessionWindow() { + // Given: + final Window window = Window.of( + Instant.ofEpochMilli(10_123), + Optional.of(Instant.ofEpochMilli(32_101)) + ); + + // When: + final List rows = QueryResultEntityFactory + .createRows(STRUCT, ImmutableMap.of(Optional.of(window), A_VALUE), LOGICAL_SCHEMA); + + // Then: + final Optional actual = rows.get(0).getWindow(); + assertThat(actual, is(Optional.of(new QueryResultEntity.Window( + 10_123, + OptionalLong.of(32_101) + )))); + } +} diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/StaticQueryFunctionalTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/StaticQueryFunctionalTest.java new file mode 100644 index 000000000000..b3af29dcaf53 --- /dev/null +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/integration/StaticQueryFunctionalTest.java @@ -0,0 +1,226 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.integration; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import io.confluent.common.utils.IntegrationTest; +import io.confluent.ksql.integration.IntegrationTestHarness; +import io.confluent.ksql.integration.Retry; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.QueryResultEntity; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.rest.server.TestKsqlRestApp; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.Format; +import io.confluent.ksql.serde.SerdeOption; +import io.confluent.ksql.test.util.KsqlIdentifierTestUtil; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.SchemaUtil; +import io.confluent.ksql.util.TestDataProvider; +import io.confluent.ksql.util.UserDataProvider; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import kafka.zookeeper.ZooKeeperClientException; +import org.apache.kafka.streams.StreamsConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.RuleChain; + +/** + * Test to ensure static queries route across multiple KSQL nodes correctly. + * + *

For tests on general syntax and handled see RestQueryTranslationTest's + * materialized-aggregate-static-queries.json + */ +@Category({IntegrationTest.class}) +public class StaticQueryFunctionalTest { + + private static final String USER_TOPIC = "user_topic"; + private static final String USERS_STREAM = "users"; + + private static final TestDataProvider USER_PROVIDER = new UserDataProvider(); + private static final Format VALUE_FORMAT = Format.JSON; + + private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + + private static final int BASE_TIME = 1_000_000; + + private static final PhysicalSchema AGGREGATE_SCHEMA = PhysicalSchema.from( + LogicalSchema.builder() + .valueColumn("COUNT", SqlTypes.BIGINT) + .build(), + SerdeOption.none() + ); + + private static final TestKsqlRestApp REST_APP_0 = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KsqlConfig.KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withStaticServiceContext(TEST_HARNESS::getServiceContext) + .build(); + + private static final TestKsqlRestApp REST_APP_1 = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KsqlConfig.KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withStaticServiceContext(TEST_HARNESS::getServiceContext) + .build(); + + @ClassRule + public static final RuleChain CHAIN = RuleChain + .outerRule(Retry.of(3, ZooKeeperClientException.class, 3, TimeUnit.SECONDS)) + .around(TEST_HARNESS) + .around(REST_APP_0) + .around(REST_APP_1); + + private String output; + + @BeforeClass + public static void setUpClass() { + TEST_HARNESS.ensureTopics(2, USER_TOPIC); + + final AtomicLong timestampSupplier = new AtomicLong(BASE_TIME); + + TEST_HARNESS.produceRows( + USER_TOPIC, + USER_PROVIDER, + VALUE_FORMAT, + timestampSupplier::getAndIncrement + ); + + makeKsqlRequest( + REST_APP_0, + "CREATE STREAM " + USERS_STREAM + + " " + USER_PROVIDER.ksqlSchemaString() + + " WITH (" + + " kafka_topic='" + USER_TOPIC + "', " + + " key='" + USER_PROVIDER.key() + "', " + + " value_format='" + VALUE_FORMAT + "'" + + ");" + ); + } + + @Before + public void setUp() { + output = KsqlIdentifierTestUtil.uniqueIdentifierName(); + } + + @After + public void cleanUp() { + REST_APP_0.closePersistentQueries(); + REST_APP_0.dropSourcesExcept(USERS_STREAM); + REST_APP_1.closePersistentQueries(); + REST_APP_1.dropSourcesExcept(USERS_STREAM); + } + + @Test + public void shouldGetSingleKeyFromBothNodes() { + // Given: + final String key = Iterables.get(USER_PROVIDER.data().keySet(), 0); + + makeKsqlRequest( + REST_APP_0, + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(1) AS COUNT FROM " + USERS_STREAM + + " GROUP BY " + USER_PROVIDER.key() + ";" + ); + + waitForTableRows(); + + final String sql = "SELECT * FROM " + output + " WHERE ROWKEY = '" + key + "';"; + + // When: + + final List rows_0 = makeStaticQueryRequest(REST_APP_0, sql); + final List rows_1 = makeStaticQueryRequest(REST_APP_1, sql); + + // Then: + assertThat(rows_0, hasSize(1)); + assertThat(rows_0.get(0).getKey(), is(ImmutableMap.of(SchemaUtil.ROWKEY_NAME, key))); + assertThat(rows_0.get(0).getValue(), is(ImmutableMap.of("COUNT", 1))); + assertThat(rows_1, is(rows_0)); + } + + @Test + public void shouldGetSingleWindowedKeyFromBothNodes() { + // Given: + final String key = Iterables.get(USER_PROVIDER.data().keySet(), 0); + + makeKsqlRequest( + REST_APP_0, + "CREATE TABLE " + output + " AS" + + " SELECT COUNT(1) AS COUNT FROM " + USERS_STREAM + + " WINDOW TUMBLING (SIZE 1 SECOND)" + + " GROUP BY " + USER_PROVIDER.key() + ";" + ); + + waitForTableRows(); + + final String sql = "SELECT * FROM " + output + + " WHERE ROWKEY = '" + key + "'" + + " AND WINDOWSTART = " + BASE_TIME + ";"; + + // When: + final List rows_0 = makeStaticQueryRequest(REST_APP_0, sql); + final List rows_1 = makeStaticQueryRequest(REST_APP_1, sql); + + // Then: + assertThat(rows_0, hasSize(1)); + assertThat(rows_0.get(0).getKey(), is(ImmutableMap.of(SchemaUtil.ROWKEY_NAME, key))); + assertThat(rows_0.get(0).getValue(), is(ImmutableMap.of("COUNT", 1))); + assertThat(rows_1, is(rows_0)); + } + + private static List makeStaticQueryRequest( + final TestKsqlRestApp target, + final String sql + ) { + final List entities = makeKsqlRequest(target, sql); + assertThat(entities, hasSize(1)); + + final KsqlEntity entity = entities.get(0); + assertThat(entity, instanceOf(QueryResultEntity.class)); + return ((QueryResultEntity)entity).getRows(); + } + + private static List makeKsqlRequest( + final TestKsqlRestApp target, + final String sql + ) { + return RestIntegrationTestUtil.makeKsqlRequest(target, sql); + } + + private void waitForTableRows() { + TEST_HARNESS.verifyAvailableUniqueRows( + output.toUpperCase(), + USER_PROVIDER.data().size(), + VALUE_FORMAT, + AGGREGATE_SCHEMA + ); + } +} + diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java index 68ff40a70e96..d01065debf58 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/StatementExecutorTest.java @@ -116,7 +116,7 @@ public class StatementExecutorTest extends EasyMockSupport { public void setUp() { ksqlConfig = KsqlConfigTestUtil.create( CLUSTER, - ImmutableMap.of(StreamsConfig.APPLICATION_SERVER_CONFIG, "host:1234") + ImmutableMap.of(StreamsConfig.APPLICATION_SERVER_CONFIG, "http://host:1234") ); final FakeKafkaTopicClient fakeKafkaTopicClient = new FakeKafkaTopicClient(); diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 24bd0ad14790..b2ca0568730d 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -626,7 +626,7 @@ public void shouldFailCreateTableWithInferenceWithUnknownKey() { } @Test - public void shouldFailBareQuery() { + public void shouldFailBareContinuousQuery() { // Then: expectedException.expect(KsqlRestException.class); expectedException.expect(exceptionStatusCode(is(Code.BAD_REQUEST))); @@ -634,6 +634,20 @@ public void shouldFailBareQuery() { "RUN SCRIPT cannot be used with the following statements: \n" + "* PRINT\n" + "* SELECT")))); + expectedException.expect(exceptionStatementErrorMessage(statement(is( + "SELECT * FROM test_table EMIT CHANGES;")))); + + // When: + makeRequest("SELECT * FROM test_table EMIT CHANGES;"); + } + + @Test + public void shouldAllowBareStaticQuery() { + // Then: + expectedException.expect(KsqlRestException.class); + expectedException.expect(exceptionStatusCode(is(Code.BAD_REQUEST))); + expectedException.expect(exceptionStatementErrorMessage(errorMessage(containsString( + "Table 'TEST_TABLE' is not materialized")))); expectedException.expect(exceptionStatementErrorMessage(statement(is( "SELECT * FROM test_table;")))); diff --git a/ksql-rest-client/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityTest.java b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityTest.java new file mode 100644 index 000000000000..191d73cb25ca --- /dev/null +++ b/ksql-rest-client/src/test/java/io/confluent/ksql/rest/entity/QueryResultEntityTest.java @@ -0,0 +1,271 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.json.KsqlTypesSerializationModule; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.rest.client.json.KsqlTypesDeserializationModule; +import io.confluent.ksql.rest.entity.QueryResultEntity.Row; +import io.confluent.ksql.rest.entity.QueryResultEntity.Window; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import java.util.LinkedHashMap; +import java.util.Optional; +import java.util.OptionalLong; +import org.junit.Test; + +public class QueryResultEntityTest { + + private static final ObjectMapper MAPPER; + private static final String SOME_SQL = "some SQL"; + + private static final LogicalSchema LOGICAL_SCHEMA = LogicalSchema.builder() + .keyColumn("ROWKEY", SqlTypes.STRING) + .valueColumn("v0", SqlTypes.DOUBLE) + .valueColumn("v1", SqlTypes.STRING) + .build(); + + private static final Optional SESSION_WINDOW = Optional + .of(new Window(12_234, OptionalLong.of(43_234))); + + private static final Optional TIME_WINDOW = Optional + .of(new Window(12_234, OptionalLong.empty())); + + private static final LinkedHashMap A_KEY = + orderedMap("ROWKEY", "x"); + + private static final LinkedHashMap A_VALUE = + orderedMap("v0", 10.1D, "v1", "some text"); + + static { + MAPPER = new ObjectMapper(); + MAPPER.registerModule(new Jdk8Module()); + MAPPER.registerModule(new KsqlTypesSerializationModule()); + MAPPER.registerModule(new KsqlTypesDeserializationModule()); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnRowWindowTypeMismatch() { + new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.TUMBLING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnRowWindowTypeIfNoWindowTypeSupplied() { + new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnNoRowWindowIfWindowTypeSupplied() { + new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.TUMBLING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(Optional.empty(), A_KEY, A_VALUE)) + ); + } + + @Test + public void shouldNotThrowOnSessionRows() { + new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + } + + @Test + public void shouldNotThrowOnHoppingRows() { + new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.HOPPING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(TIME_WINDOW, A_KEY, A_VALUE)) + ); + } + + @Test + public void shouldNotThrowOnTumblingRows() { + new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.TUMBLING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(TIME_WINDOW, A_KEY, A_VALUE)) + ); + } + + @Test + public void shouldSerializeEntity() throws Exception { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final String json = MAPPER.writeValueAsString(entity); + + // Then: + assertThat(json, is("{" + + "\"@type\":\"row\"," + + "\"statementText\":\"some SQL\"," + + "\"windowType\":\"SESSION\"," + + "\"schema\":\"`ROWKEY` STRING KEY, `v0` DOUBLE, `v1` STRING\"," + + "\"rows\":[" + + "{" + + "\"window\":{\"start\":12234,\"end\":43234}," + + "\"key\":{\"ROWKEY\":\"x\"}," + + "\"value\":{\"v0\":10.1,\"v1\":\"some text\"}" + + "}" + + "]," + + "\"warnings\":[]}")); + + // When: + final KsqlEntity result = MAPPER.readValue(json, KsqlEntity.class); + + // Then: + assertThat(result, is(entity)); + } + + @Test + public void shouldSerializeNullValue() throws Exception { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, null)) + ); + + // When: + final String json = MAPPER.writeValueAsString(entity); + + // Then: + assertThat(json, containsString("\"value\":null")); + + // When: + final KsqlEntity result = MAPPER.readValue(json, KsqlEntity.class); + + // Then: + assertThat(result, is(entity)); + } + + @Test + public void shouldSerializeNullElements() throws Exception { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.SESSION), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(SESSION_WINDOW, A_KEY, orderedMap("v0", 10.1D, "v1", null))) + ); + + // When: + final String json = MAPPER.writeValueAsString(entity); + + // Then: + assertThat(json, containsString("\"value\":{\"v0\":10.1,\"v1\":null}")); + + // When: + final KsqlEntity result = MAPPER.readValue(json, KsqlEntity.class); + + // Then: + assertThat(result, is(entity)); + } + + @Test + public void shouldSerializeRowWithNoWindow() throws Exception { + // Given: + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.empty(), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(Optional.empty(), A_KEY, A_VALUE)) + ); + + // When: + final String json = MAPPER.writeValueAsString(entity); + + // Then: + assertThat(json, containsString("\"window\":null")); + + // When: + final KsqlEntity result = MAPPER.readValue(json, KsqlEntity.class); + + // Then: + assertThat(result, is(entity)); + } + + @Test + public void shouldSerializeRowWithTimeWindow() throws Exception { + // Given: + + final QueryResultEntity entity = new QueryResultEntity( + SOME_SQL, + Optional.of(WindowType.HOPPING), + LOGICAL_SCHEMA, + ImmutableList.of(new Row(TIME_WINDOW, A_KEY, A_VALUE)) + ); + + // When: + final String json = MAPPER.writeValueAsString(entity); + + // Then: + assertThat(json, containsString("\"window\":{\"start\":12234,\"end\":null}")); + + // When: + final KsqlEntity result = MAPPER.readValue(json, KsqlEntity.class); + + // Then: + assertThat(result, is(entity)); + } + + private static LinkedHashMap orderedMap(final Object... keysAndValues) { + assertThat("invalid test", keysAndValues.length % 2, is(0)); + + final LinkedHashMap orderedMap = new LinkedHashMap<>(); + + for (int idx = 0; idx < keysAndValues.length; idx = idx + 2) { + final Object key = keysAndValues[idx]; + final Object value = keysAndValues[idx + 1]; + + assertThat("invalid test", key, instanceOf(String.class)); + orderedMap.put((String) key, value); + } + + return orderedMap; + } +} \ No newline at end of file diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 8194b02f90bb..c7e36776de77 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -33,7 +33,7 @@ public final class Errors { public static final int ERROR_CODE_BAD_REQUEST = toErrorCode(BAD_REQUEST.getStatusCode()); public static final int ERROR_CODE_BAD_STATEMENT = toErrorCode(BAD_REQUEST.getStatusCode()) + 1; - private static final int ERROR_CODE_QUERY_ENDPOINT = toErrorCode(BAD_REQUEST.getStatusCode()) + 2; + public static final int ERROR_CODE_QUERY_ENDPOINT = toErrorCode(BAD_REQUEST.getStatusCode()) + 2; public static final int ERROR_CODE_UNAUTHORIZED = toErrorCode(UNAUTHORIZED.getStatusCode()); diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java index c3d8b67db3c9..b89d7be63985 100644 --- a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java @@ -48,7 +48,8 @@ @JsonSubTypes.Type(value = ConnectorList.class, name = "connector_list"), @JsonSubTypes.Type(value = ConnectorDescription.class, name = "connector_description"), @JsonSubTypes.Type(value = TypeList.class, name = "type_list"), - @JsonSubTypes.Type(value = ErrorEntity.class, name = "error_entity") + @JsonSubTypes.Type(value = ErrorEntity.class, name = "error_entity"), + @JsonSubTypes.Type(value = QueryResultEntity.class, name = "row") }) public abstract class KsqlEntity { private final String statementText; diff --git a/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntity.java b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntity.java new file mode 100644 index 000000000000..2c809538f3ab --- /dev/null +++ b/ksql-rest-model/src/main/java/io/confluent/ksql/rest/entity/QueryResultEntity.java @@ -0,0 +1,226 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import static java.util.Objects.requireNonNull; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.model.WindowType; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalLong; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class QueryResultEntity extends KsqlEntity { + + private final Optional windowType; + private final LogicalSchema schema; + private final ImmutableList rows; + + public QueryResultEntity( + @JsonProperty("statementText") final String statementText, + @JsonProperty("windowType") final Optional windowType, + @JsonProperty("schema") final LogicalSchema schema, + @JsonProperty("rows") final List rows + ) { + super(statementText); + this.windowType = requireNonNull(windowType, "windowType"); + this.schema = requireNonNull(schema, "schema"); + this.rows = ImmutableList.copyOf(requireNonNull(rows, "rows")); + + rows.forEach(row -> row.validate(windowType, schema)); + } + + public Optional getWindowType() { + return windowType; + } + + public LogicalSchema getSchema() { + return schema; + } + + public List getRows() { + return rows; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (!(o instanceof QueryResultEntity)) { + return false; + } + final QueryResultEntity that = (QueryResultEntity) o; + return Objects.equals(schema, that.schema) + && Objects.equals(rows, that.rows); + } + + @Override + public int hashCode() { + return Objects.hash(schema, rows); + } + + @JsonIgnoreProperties(ignoreUnknown = true) + public static final class Window { + + private final long start; + private final OptionalLong end; + + public Window( + @JsonProperty("start") final long start, + @JsonProperty("end") final OptionalLong end + ) { + this.start = start; + this.end = requireNonNull(end, "end"); + } + + public long getStart() { + return start; + } + + public OptionalLong getEnd() { + return end; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Window window = (Window) o; + return start == window.start + && Objects.equals(end, window.end); + } + + @Override + public int hashCode() { + return Objects.hash(start, end); + } + + @Override + public String toString() { + return "Window{" + + "start=" + start + + ", end=" + end + + '}'; + } + } + + @JsonIgnoreProperties(ignoreUnknown = true) + public static final class Row { + + private final Optional window; + private final Map key; + private final Optional> value; + + @JsonCreator + public Row( + @JsonProperty("window") final Optional window, + @JsonProperty("key") final LinkedHashMap key, + @JsonProperty("value") final LinkedHashMap value + ) { + this.window = requireNonNull(window, "window"); + this.key = new LinkedHashMap<>(requireNonNull(key, "key")); + this.value = Optional.ofNullable(value).map(LinkedHashMap::new); + } + + public Optional getWindow() { + return window; + } + + public Map getKey() { + return Collections.unmodifiableMap(key); + } + + public Map getValue() { + return value + .map(Collections::unmodifiableMap) + .orElse(null); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final Row that = (Row) o; + return Objects.equals(this.window, that.window) + && Objects.equals(this.key, that.key) + && Objects.equals(this.value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(window, key, value); + } + + void validate( + final Optional windowType, + final LogicalSchema schema + ) { + if (window.isPresent() != windowType.isPresent()) { + throw new IllegalArgumentException("window mismatch." + + " expected: " + windowType + + ", got: " + window + ); + } + + window.ifPresent(w -> { + final boolean rowIsSession = w.end.isPresent(); + final boolean schemaIsSession = windowType.get() == WindowType.SESSION; + if (rowIsSession != schemaIsSession) { + throw new IllegalArgumentException("window mismtach." + + " expected: " + schemaIsSession + + ", got: " + rowIsSession + ); + } + }); + + if (schema.key().size() != key.size()) { + throw new IllegalArgumentException("key field count mismatch." + + " expected: " + schema.key().size() + + ", got: " + key.size() + ); + } + + value.ifPresent(v -> { + if (schema.value().size() != v.size()) { + throw new IllegalArgumentException("value field count mismatch." + + " expected: " + schema.value().size() + + ", got: " + v.size() + ); + } + }); + } + } +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SelectValueMapper.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SelectValueMapper.java index 49bb2d565d23..b98b91a53c10 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SelectValueMapper.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SelectValueMapper.java @@ -33,7 +33,7 @@ public class SelectValueMapper implements ValueMapper { private final ImmutableList selects; private final ProcessingLogger processingLogger; - SelectValueMapper( + public SelectValueMapper( final List selects, final ProcessingLogger processingLogger ) {